You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by zh...@apache.org on 2019/01/04 06:40:27 UTC
[01/15] hbase git commit: HBASE-21547 Precommit uses master flaky
list for other branches [Forced Update!]
Repository: hbase
Updated Branches:
refs/heads/HBASE-21512 44462a48e -> 04e6909ad (forced update)
HBASE-21547 Precommit uses master flaky list for other branches
Signed-off-by: Sean Busbey <bu...@apache.org>
Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/f0b50a8f
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/f0b50a8f
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/f0b50a8f
Branch: refs/heads/HBASE-21512
Commit: f0b50a8f9b6db266c1e0f307fda6a4ad7b852135
Parents: ec948f5
Author: Peter Somogyi <ps...@apache.org>
Authored: Fri Dec 28 15:16:48 2018 +0100
Committer: Peter Somogyi <ps...@apache.org>
Committed: Wed Jan 2 08:26:57 2019 +0100
----------------------------------------------------------------------
dev-support/hbase-personality.sh | 14 ++++++++++++++
1 file changed, 14 insertions(+)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/hbase/blob/f0b50a8f/dev-support/hbase-personality.sh
----------------------------------------------------------------------
diff --git a/dev-support/hbase-personality.sh b/dev-support/hbase-personality.sh
index 3856a29..8dd24be 100755
--- a/dev-support/hbase-personality.sh
+++ b/dev-support/hbase-personality.sh
@@ -272,6 +272,20 @@ function get_include_exclude_tests_arg
yetus_error "Wget error $? in fetching includes file from url" \
"${INCLUDE_TESTS_URL}. Ignoring and proceeding."
fi
+ else
+ # Use branch specific exclude list when EXCLUDE_TESTS_URL and INCLUDE_TESTS_URL are empty
+ FLAKY_URL="https://builds.apache.org/job/HBase-Find-Flaky-Tests/job/${PATCH_BRANCH}/lastSuccessfulBuild/artifact/excludes/"
+ if wget "${FLAKY_URL}" -O "excludes"; then
+ excludes=$(cat excludes)
+ yetus_debug "excludes=${excludes}"
+ if [[ -n "${excludes}" ]]; then
+ eval "${__resultvar}='-Dtest.exclude.pattern=${excludes}'"
+ fi
+ rm excludes
+ else
+ yetus_error "Wget error $? in fetching excludes file from url" \
+ "${FLAKY_URL}. Ignoring and proceeding."
+ fi
fi
}
[05/15] hbase git commit: HBASE-21652 Refactor ThriftServer making
thrift2 server inherited from thrift1 server
Posted by zh...@apache.org.
HBASE-21652 Refactor ThriftServer making thrift2 server inherited from thrift1 server
Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/e4b6b4af
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/e4b6b4af
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/e4b6b4af
Branch: refs/heads/HBASE-21512
Commit: e4b6b4afb933a961f543537875f87a2dc62d3757
Parents: f0b50a8
Author: Allan Yang <al...@apache.org>
Authored: Wed Jan 2 16:13:17 2019 +0800
Committer: Allan Yang <al...@apache.org>
Committed: Wed Jan 2 16:13:57 2019 +0800
----------------------------------------------------------------------
.../apache/hadoop/hbase/thrift/Constants.java | 151 ++
.../hbase/thrift/HBaseServiceHandler.java | 90 +
.../hbase/thrift/HbaseHandlerMetricsProxy.java | 20 +-
.../apache/hadoop/hbase/thrift/ImplType.java | 143 ++
.../hadoop/hbase/thrift/IncrementCoalescer.java | 6 +-
.../hbase/thrift/ThriftHBaseServiceHandler.java | 1347 ++++++++++++
.../hadoop/hbase/thrift/ThriftHttpServlet.java | 12 +-
.../hadoop/hbase/thrift/ThriftServer.java | 698 +++++-
.../hadoop/hbase/thrift/ThriftServerRunner.java | 2031 ------------------
.../thrift2/ThriftHBaseServiceHandler.java | 69 +-
.../hadoop/hbase/thrift2/ThriftServer.java | 594 +----
.../resources/hbase-webapps/thrift/thrift.jsp | 2 +-
.../hbase/thrift/TestThriftHttpServer.java | 28 +-
.../hadoop/hbase/thrift/TestThriftServer.java | 58 +-
.../hbase/thrift/TestThriftServerCmdLine.java | 48 +-
.../thrift/TestThriftSpnegoHttpServer.java | 21 +-
.../hbase/thrift2/TestThrift2HttpServer.java | 90 +
.../hbase/thrift2/TestThrift2ServerCmdLine.java | 99 +
.../thrift2/TestThriftHBaseServiceHandler.java | 15 +-
19 files changed, 2711 insertions(+), 2811 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/hbase/blob/e4b6b4af/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/Constants.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/Constants.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/Constants.java
new file mode 100644
index 0000000..8e3d004
--- /dev/null
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/Constants.java
@@ -0,0 +1,151 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.thrift;
+
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * Thrift related constants
+ */
+@InterfaceAudience.Private
+public final class Constants {
+ private Constants(){}
+
+ public static final int DEFAULT_HTTP_MAX_HEADER_SIZE = 64 * 1024; // 64k
+
+ public static final String SERVER_TYPE_CONF_KEY =
+ "hbase.regionserver.thrift.server.type";
+
+ public static final String COMPACT_CONF_KEY = "hbase.regionserver.thrift.compact";
+ public static final boolean COMPACT_CONF_DEFAULT = false;
+
+ public static final String FRAMED_CONF_KEY = "hbase.regionserver.thrift.framed";
+ public static final boolean FRAMED_CONF_DEFAULT = false;
+
+ public static final String MAX_FRAME_SIZE_CONF_KEY =
+ "hbase.regionserver.thrift.framed.max_frame_size_in_mb";
+ public static final int MAX_FRAME_SIZE_CONF_DEFAULT = 2;
+
+ public static final String COALESCE_INC_KEY = "hbase.regionserver.thrift.coalesceIncrement";
+ public static final String USE_HTTP_CONF_KEY = "hbase.regionserver.thrift.http";
+
+ public static final String HTTP_MIN_THREADS_KEY = "hbase.thrift.http_threads.min";
+ public static final int HTTP_MIN_THREADS_KEY_DEFAULT = 2;
+
+ public static final String HTTP_MAX_THREADS_KEY = "hbase.thrift.http_threads.max";
+ public static final int HTTP_MAX_THREADS_KEY_DEFAULT = 100;
+
+ // ssl related configs
+ public static final String THRIFT_SSL_ENABLED_KEY = "hbase.thrift.ssl.enabled";
+ public static final String THRIFT_SSL_KEYSTORE_STORE_KEY = "hbase.thrift.ssl.keystore.store";
+ public static final String THRIFT_SSL_KEYSTORE_PASSWORD_KEY =
+ "hbase.thrift.ssl.keystore.password";
+ public static final String THRIFT_SSL_KEYSTORE_KEYPASSWORD_KEY
+ = "hbase.thrift.ssl.keystore.keypassword";
+ public static final String THRIFT_SSL_EXCLUDE_CIPHER_SUITES_KEY =
+ "hbase.thrift.ssl.exclude.cipher.suites";
+ public static final String THRIFT_SSL_INCLUDE_CIPHER_SUITES_KEY =
+ "hbase.thrift.ssl.include.cipher.suites";
+ public static final String THRIFT_SSL_EXCLUDE_PROTOCOLS_KEY =
+ "hbase.thrift.ssl.exclude.protocols";
+ public static final String THRIFT_SSL_INCLUDE_PROTOCOLS_KEY =
+ "hbase.thrift.ssl.include.protocols";
+
+
+ public static final String THRIFT_SUPPORT_PROXYUSER_KEY = "hbase.thrift.support.proxyuser";
+
+ //kerberos related configs
+ public static final String THRIFT_DNS_INTERFACE_KEY = "hbase.thrift.dns.interface";
+ public static final String THRIFT_DNS_NAMESERVER_KEY = "hbase.thrift.dns.nameserver";
+ public static final String THRIFT_KERBEROS_PRINCIPAL_KEY = "hbase.thrift.kerberos.principal";
+ public static final String THRIFT_KEYTAB_FILE_KEY = "hbase.thrift.keytab.file";
+ public static final String THRIFT_SPNEGO_PRINCIPAL_KEY = "hbase.thrift.spnego.principal";
+ public static final String THRIFT_SPNEGO_KEYTAB_FILE_KEY = "hbase.thrift.spnego.keytab.file";
+
+ /**
+ * Amount of time in milliseconds before a server thread will timeout
+ * waiting for client to send data on a connected socket. Currently,
+ * applies only to TBoundedThreadPoolServer
+ */
+ public static final String THRIFT_SERVER_SOCKET_READ_TIMEOUT_KEY =
+ "hbase.thrift.server.socket.read.timeout";
+ public static final int THRIFT_SERVER_SOCKET_READ_TIMEOUT_DEFAULT = 60000;
+
+
+ /**
+ * Thrift quality of protection configuration key. Valid values can be:
+ * auth-conf: authentication, integrity and confidentiality checking
+ * auth-int: authentication and integrity checking
+ * auth: authentication only
+ *
+ * This is used to authenticate the callers and support impersonation.
+ * The thrift server and the HBase cluster must run in secure mode.
+ */
+ public static final String THRIFT_QOP_KEY = "hbase.thrift.security.qop";
+
+ public static final String BACKLOG_CONF_KEY = "hbase.regionserver.thrift.backlog";
+ public static final int BACKLOG_CONF_DEAFULT = 0;
+
+ public static final String BIND_CONF_KEY = "hbase.regionserver.thrift.ipaddress";
+ public static final String DEFAULT_BIND_ADDR = "0.0.0.0";
+
+ public static final String PORT_CONF_KEY = "hbase.regionserver.thrift.port";
+ public static final int DEFAULT_LISTEN_PORT = 9090;
+
+ public static final String THRIFT_HTTP_ALLOW_OPTIONS_METHOD =
+ "hbase.thrift.http.allow.options.method";
+ public static final boolean THRIFT_HTTP_ALLOW_OPTIONS_METHOD_DEFAULT = false;
+
+ public static final String THRIFT_INFO_SERVER_PORT = "hbase.thrift.info.port";
+ public static final int THRIFT_INFO_SERVER_PORT_DEFAULT = 9095;
+
+ public static final String THRIFT_INFO_SERVER_BINDING_ADDRESS = "hbase.thrift.info.bindAddress";
+ public static final String THRIFT_INFO_SERVER_BINDING_ADDRESS_DEFAULT = "0.0.0.0";
+
+ public static final String THRIFT_QUEUE_SIZE = "hbase.thrift.queue.size";
+ public static final int THRIFT_QUEUE_SIZE_DEFAULT = Integer.MAX_VALUE;
+
+ public static final String THRIFT_SELECTOR_NUM = "hbase.thrift.selector.num";
+
+ public static final String THRIFT_FILTERS = "hbase.thrift.filters";
+
+ // Command line options
+
+ public static final String READ_TIMEOUT_OPTION = "readTimeout";
+ public static final String MIN_WORKERS_OPTION = "minWorkers";
+ public static final String MAX_WORKERS_OPTION = "workers";
+ public static final String MAX_QUEUE_SIZE_OPTION = "queue";
+ public static final String SELECTOR_NUM_OPTION = "selectors";
+ public static final String KEEP_ALIVE_SEC_OPTION = "keepAliveSec";
+ public static final String BIND_OPTION = "bind";
+ public static final String COMPACT_OPTION = "compact";
+ public static final String FRAMED_OPTION = "framed";
+ public static final String PORT_OPTION = "port";
+ public static final String INFOPORT_OPTION = "infoport";
+
+ //for thrift2 server
+ public static final String READONLY_OPTION ="readonly";
+
+ public static final String THRIFT_READONLY_ENABLED = "hbase.thrift.readonly";
+ public static final boolean THRIFT_READONLY_ENABLED_DEFAULT = false;
+
+
+
+
+
+}
http://git-wip-us.apache.org/repos/asf/hbase/blob/e4b6b4af/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/HBaseServiceHandler.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/HBaseServiceHandler.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/HBaseServiceHandler.java
new file mode 100644
index 0000000..7990871
--- /dev/null
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/HBaseServiceHandler.java
@@ -0,0 +1,90 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.thrift;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseInterfaceAudience;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.security.UserProvider;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.ConnectionCache;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * abstract class for HBase handler
+ * providing a Connection cache and get table/admin method
+ */
+@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.TOOLS)
+public abstract class HBaseServiceHandler {
+ public static final String CLEANUP_INTERVAL = "hbase.thrift.connection.cleanup-interval";
+ public static final String MAX_IDLETIME = "hbase.thrift.connection.max-idletime";
+
+ protected Configuration conf;
+
+ protected final ConnectionCache connectionCache;
+
+ public HBaseServiceHandler(final Configuration c,
+ final UserProvider userProvider) throws IOException {
+ this.conf = c;
+ int cleanInterval = conf.getInt(CLEANUP_INTERVAL, 10 * 1000);
+ int maxIdleTime = conf.getInt(MAX_IDLETIME, 10 * 60 * 1000);
+ connectionCache = new ConnectionCache(
+ conf, userProvider, cleanInterval, maxIdleTime);
+ }
+
+ protected ThriftMetrics metrics = null;
+
+ public void initMetrics(ThriftMetrics metrics) {
+ this.metrics = metrics;
+ }
+
+ public void setEffectiveUser(String effectiveUser) {
+ connectionCache.setEffectiveUser(effectiveUser);
+ }
+
+ /**
+ * Obtain HBaseAdmin. Creates the instance if it is not already created.
+ */
+ protected Admin getAdmin() throws IOException {
+ return connectionCache.getAdmin();
+ }
+
+ /**
+ * Creates and returns a Table instance from a given table name.
+ *
+ * @param tableName
+ * name of table
+ * @return Table object
+ * @throws IOException if getting the table fails
+ */
+ protected Table getTable(final byte[] tableName) throws IOException {
+ String table = Bytes.toString(tableName);
+ return connectionCache.getTable(table);
+ }
+
+ protected Table getTable(final ByteBuffer tableName) throws IOException {
+ return getTable(Bytes.getBytes(tableName));
+ }
+
+
+}
http://git-wip-us.apache.org/repos/asf/hbase/blob/e4b6b4af/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/HbaseHandlerMetricsProxy.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/HbaseHandlerMetricsProxy.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/HbaseHandlerMetricsProxy.java
index 5a6e436..1402f86 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/HbaseHandlerMetricsProxy.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/HbaseHandlerMetricsProxy.java
@@ -25,9 +25,8 @@ import java.lang.reflect.Proxy;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.thrift.generated.Hbase;
+import org.apache.hadoop.hbase.thrift2.generated.THBaseService;
import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
/**
* Converts a Hbase.Iface using InvocationHandler so that it reports process
@@ -36,10 +35,7 @@ import org.slf4j.LoggerFactory;
@InterfaceAudience.Private
public final class HbaseHandlerMetricsProxy implements InvocationHandler {
- private static final Logger LOG = LoggerFactory.getLogger(
- HbaseHandlerMetricsProxy.class);
-
- private final Hbase.Iface handler;
+ private final Object handler;
private final ThriftMetrics metrics;
public static Hbase.Iface newInstance(Hbase.Iface handler,
@@ -51,8 +47,18 @@ public final class HbaseHandlerMetricsProxy implements InvocationHandler {
new HbaseHandlerMetricsProxy(handler, metrics, conf));
}
+ // for thrift 2
+ public static THBaseService.Iface newInstance(THBaseService.Iface handler,
+ ThriftMetrics metrics,
+ Configuration conf) {
+ return (THBaseService.Iface) Proxy.newProxyInstance(
+ handler.getClass().getClassLoader(),
+ new Class[]{THBaseService.Iface.class},
+ new HbaseHandlerMetricsProxy(handler, metrics, conf));
+ }
+
private HbaseHandlerMetricsProxy(
- Hbase.Iface handler, ThriftMetrics metrics, Configuration conf) {
+ Object handler, ThriftMetrics metrics, Configuration conf) {
this.handler = handler;
this.metrics = metrics;
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/e4b6b4af/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ImplType.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ImplType.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ImplType.java
new file mode 100644
index 0000000..7108115
--- /dev/null
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ImplType.java
@@ -0,0 +1,143 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.thrift;
+
+import static org.apache.hadoop.hbase.thrift.Constants.SERVER_TYPE_CONF_KEY;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.thrift.server.THsHaServer;
+import org.apache.thrift.server.TNonblockingServer;
+import org.apache.thrift.server.TServer;
+import org.apache.thrift.server.TThreadedSelectorServer;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLine;
+import org.apache.hbase.thirdparty.org.apache.commons.cli.Option;
+import org.apache.hbase.thirdparty.org.apache.commons.cli.OptionGroup;
+
+/** An enum of server implementation selections */
+@InterfaceAudience.Private
+public enum ImplType {
+ HS_HA("hsha", true, THsHaServer.class, true),
+ NONBLOCKING("nonblocking", true, TNonblockingServer.class, true),
+ THREAD_POOL("threadpool", false, TBoundedThreadPoolServer.class, true),
+ THREADED_SELECTOR("threadedselector", true, TThreadedSelectorServer.class, true);
+
+ private static final Logger LOG = LoggerFactory.getLogger(ImplType.class);
+ public static final ImplType DEFAULT = THREAD_POOL;
+
+
+ final String option;
+ final boolean isAlwaysFramed;
+ final Class<? extends TServer> serverClass;
+ final boolean canSpecifyBindIP;
+
+ private ImplType(String option, boolean isAlwaysFramed,
+ Class<? extends TServer> serverClass, boolean canSpecifyBindIP) {
+ this.option = option;
+ this.isAlwaysFramed = isAlwaysFramed;
+ this.serverClass = serverClass;
+ this.canSpecifyBindIP = canSpecifyBindIP;
+ }
+
+ /**
+ * @return <code>-option</code>
+ */
+ @Override
+ public String toString() {
+ return "-" + option;
+ }
+
+ public String getOption() {
+ return option;
+ }
+
+ public boolean isAlwaysFramed() {
+ return isAlwaysFramed;
+ }
+
+ public String getDescription() {
+ StringBuilder sb = new StringBuilder("Use the " +
+ serverClass.getSimpleName());
+ if (isAlwaysFramed) {
+ sb.append(" This implies the framed transport.");
+ }
+ if (this == DEFAULT) {
+ sb.append("This is the default.");
+ }
+ return sb.toString();
+ }
+
+ static OptionGroup createOptionGroup() {
+ OptionGroup group = new OptionGroup();
+ for (ImplType t : values()) {
+ group.addOption(new Option(t.option, t.getDescription()));
+ }
+ return group;
+ }
+
+ public static ImplType getServerImpl(Configuration conf) {
+ String confType = conf.get(SERVER_TYPE_CONF_KEY, THREAD_POOL.option);
+ for (ImplType t : values()) {
+ if (confType.equals(t.option)) {
+ return t;
+ }
+ }
+ throw new AssertionError("Unknown server ImplType.option:" + confType);
+ }
+
+ static void setServerImpl(CommandLine cmd, Configuration conf) {
+ ImplType chosenType = null;
+ int numChosen = 0;
+ for (ImplType t : values()) {
+ if (cmd.hasOption(t.option)) {
+ chosenType = t;
+ ++numChosen;
+ }
+ }
+ if (numChosen < 1) {
+ LOG.info("Using default thrift server type");
+ chosenType = DEFAULT;
+ } else if (numChosen > 1) {
+ throw new AssertionError("Exactly one option out of " +
+ Arrays.toString(values()) + " has to be specified");
+ }
+ LOG.info("Using thrift server type " + chosenType.option);
+ conf.set(SERVER_TYPE_CONF_KEY, chosenType.option);
+ }
+
+ public String simpleClassName() {
+ return serverClass.getSimpleName();
+ }
+
+ public static List<String> serversThatCannotSpecifyBindIP() {
+ List<String> l = new ArrayList<>();
+ for (ImplType t : values()) {
+ if (!t.canSpecifyBindIP) {
+ l.add(t.simpleClassName());
+ }
+ }
+ return l;
+ }
+}
http://git-wip-us.apache.org/repos/asf/hbase/blob/e4b6b4af/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/IncrementCoalescer.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/IncrementCoalescer.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/IncrementCoalescer.java
index e36d639..971cd17 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/IncrementCoalescer.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/IncrementCoalescer.java
@@ -33,7 +33,6 @@ import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.LongAdder;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.client.Table;
-import org.apache.hadoop.hbase.thrift.ThriftServerRunner.HBaseHandler;
import org.apache.hadoop.hbase.thrift.generated.TIncrement;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Threads;
@@ -180,7 +179,7 @@ public class IncrementCoalescer implements IncrementCoalescerMBean {
private final ConcurrentMap<FullyQualifiedRow, Long> countersMap =
new ConcurrentHashMap<>(100000, 0.75f, 1500);
private final ThreadPoolExecutor pool;
- private final HBaseHandler handler;
+ private final ThriftHBaseServiceHandler handler;
private int maxQueueSize = 500000;
private static final int CORE_POOL_SIZE = 1;
@@ -188,7 +187,7 @@ public class IncrementCoalescer implements IncrementCoalescerMBean {
private static final Logger LOG = LoggerFactory.getLogger(FullyQualifiedRow.class);
@SuppressWarnings("deprecation")
- public IncrementCoalescer(HBaseHandler hand) {
+ public IncrementCoalescer(ThriftHBaseServiceHandler hand) {
this.handler = hand;
LinkedBlockingQueue<Runnable> queue = new LinkedBlockingQueue<>();
pool =
@@ -230,6 +229,7 @@ public class IncrementCoalescer implements IncrementCoalescerMBean {
inc.getAmmount());
}
+ @SuppressWarnings("FutureReturnValueIgnored")
private boolean internalQueueIncrement(byte[] tableName, byte[] rowKey, byte[] fam,
byte[] qual, long ammount) throws TException {
int countersMapSize = countersMap.size();
http://git-wip-us.apache.org/repos/asf/hbase/blob/e4b6b4af/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftHBaseServiceHandler.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftHBaseServiceHandler.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftHBaseServiceHandler.java
new file mode 100644
index 0000000..34bf5e8
--- /dev/null
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftHBaseServiceHandler.java
@@ -0,0 +1,1347 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.thrift;
+
+import static org.apache.hadoop.hbase.thrift.Constants.COALESCE_INC_KEY;
+import static org.apache.hadoop.hbase.util.Bytes.getBytes;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellBuilder;
+import org.apache.hadoop.hbase.CellBuilderFactory;
+import org.apache.hadoop.hbase.CellBuilderType;
+import org.apache.hadoop.hbase.CellUtil;
+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.KeyValue;
+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.hadoop.hbase.client.Append;
+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.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;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.filter.Filter;
+import org.apache.hadoop.hbase.filter.ParseFilter;
+import org.apache.hadoop.hbase.filter.PrefixFilter;
+import org.apache.hadoop.hbase.filter.WhileMatchFilter;
+import org.apache.hadoop.hbase.security.UserProvider;
+import org.apache.hadoop.hbase.thrift.generated.AlreadyExists;
+import org.apache.hadoop.hbase.thrift.generated.BatchMutation;
+import org.apache.hadoop.hbase.thrift.generated.ColumnDescriptor;
+import org.apache.hadoop.hbase.thrift.generated.Hbase;
+import org.apache.hadoop.hbase.thrift.generated.IOError;
+import org.apache.hadoop.hbase.thrift.generated.IllegalArgument;
+import org.apache.hadoop.hbase.thrift.generated.Mutation;
+import org.apache.hadoop.hbase.thrift.generated.TAppend;
+import org.apache.hadoop.hbase.thrift.generated.TCell;
+import org.apache.hadoop.hbase.thrift.generated.TIncrement;
+import org.apache.hadoop.hbase.thrift.generated.TRegionInfo;
+import org.apache.hadoop.hbase.thrift.generated.TRowResult;
+import org.apache.hadoop.hbase.thrift.generated.TScan;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.thrift.TException;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hbase.thirdparty.com.google.common.base.Throwables;
+
+/**
+ * The HBaseServiceHandler is a glue object that connects Thrift RPC calls to the
+ * HBase client API primarily defined in the Admin and Table objects.
+ */
+@InterfaceAudience.Private
+@SuppressWarnings("deprecation")
+public class ThriftHBaseServiceHandler extends HBaseServiceHandler implements Hbase.Iface {
+ private static final Logger LOG = LoggerFactory.getLogger(ThriftHBaseServiceHandler.class);
+
+ public static final int HREGION_VERSION = 1;
+
+ // nextScannerId and scannerMap are used to manage scanner state
+ private int nextScannerId = 0;
+ private HashMap<Integer, ResultScannerWrapper> scannerMap;
+ IncrementCoalescer coalescer;
+
+ /**
+ * Returns a list of all the column families for a given Table.
+ *
+ * @param table table
+ * @throws IOException
+ */
+ byte[][] getAllColumns(Table table) throws IOException {
+ HColumnDescriptor[] cds = table.getTableDescriptor().getColumnFamilies();
+ byte[][] columns = new byte[cds.length][];
+ for (int i = 0; i < cds.length; i++) {
+ columns[i] = Bytes.add(cds[i].getName(),
+ KeyValue.COLUMN_FAMILY_DELIM_ARRAY);
+ }
+ return columns;
+ }
+
+
+ /**
+ * Assigns a unique ID to the scanner and adds the mapping to an internal
+ * hash-map.
+ *
+ * @param scanner the {@link ResultScanner} to add
+ * @return integer scanner id
+ */
+ protected synchronized int addScanner(ResultScanner scanner, boolean sortColumns) {
+ int id = nextScannerId++;
+ ResultScannerWrapper resultScannerWrapper =
+ new ResultScannerWrapper(scanner, sortColumns);
+ scannerMap.put(id, resultScannerWrapper);
+ return id;
+ }
+
+ /**
+ * Returns the scanner associated with the specified ID.
+ *
+ * @param id the ID of the scanner to get
+ * @return a Scanner, or null if ID was invalid.
+ */
+ private synchronized ResultScannerWrapper getScanner(int id) {
+ return scannerMap.get(id);
+ }
+
+ /**
+ * Removes the scanner associated with the specified ID from the internal
+ * id->scanner hash-map.
+ *
+ * @param id the ID of the scanner to remove
+ * @return a Scanner, or null if ID was invalid.
+ */
+ private synchronized ResultScannerWrapper removeScanner(int id) {
+ return scannerMap.remove(id);
+ }
+
+ protected ThriftHBaseServiceHandler(final Configuration c,
+ final UserProvider userProvider) throws IOException {
+ super(c, userProvider);
+ scannerMap = new HashMap<>();
+ this.coalescer = new IncrementCoalescer(this);
+ }
+
+
+ @Override
+ public void enableTable(ByteBuffer tableName) throws IOError {
+ try{
+ getAdmin().enableTable(getTableName(tableName));
+ } catch (IOException e) {
+ LOG.warn(e.getMessage(), e);
+ throw getIOError(e);
+ }
+ }
+
+ @Override
+ public void disableTable(ByteBuffer tableName) throws IOError{
+ try{
+ getAdmin().disableTable(getTableName(tableName));
+ } catch (IOException e) {
+ LOG.warn(e.getMessage(), e);
+ throw getIOError(e);
+ }
+ }
+
+ @Override
+ public boolean isTableEnabled(ByteBuffer tableName) throws IOError {
+ try {
+ return this.connectionCache.getAdmin().isTableEnabled(getTableName(tableName));
+ } catch (IOException e) {
+ LOG.warn(e.getMessage(), e);
+ throw getIOError(e);
+ }
+ }
+
+ // ThriftServerRunner.compact should be deprecated and replaced with methods specific to
+ // table and region.
+ @Override
+ public void compact(ByteBuffer tableNameOrRegionName) throws IOError {
+ try {
+ try {
+ getAdmin().compactRegion(getBytes(tableNameOrRegionName));
+ } catch (IllegalArgumentException e) {
+ // Invalid region, try table
+ getAdmin().compact(TableName.valueOf(getBytes(tableNameOrRegionName)));
+ }
+ } catch (IOException e) {
+ LOG.warn(e.getMessage(), e);
+ throw getIOError(e);
+ }
+ }
+
+ // ThriftServerRunner.majorCompact should be deprecated and replaced with methods specific
+ // to table and region.
+ @Override
+ public void majorCompact(ByteBuffer tableNameOrRegionName) throws IOError {
+ try {
+ try {
+ getAdmin().compactRegion(getBytes(tableNameOrRegionName));
+ } catch (IllegalArgumentException e) {
+ // Invalid region, try table
+ getAdmin().compact(TableName.valueOf(getBytes(tableNameOrRegionName)));
+ }
+ } catch (IOException e) {
+ LOG.warn(e.getMessage(), e);
+ throw getIOError(e);
+ }
+ }
+
+ @Override
+ public List<ByteBuffer> getTableNames() throws IOError {
+ try {
+ TableName[] tableNames = this.getAdmin().listTableNames();
+ ArrayList<ByteBuffer> list = new ArrayList<>(tableNames.length);
+ for (TableName tableName : tableNames) {
+ list.add(ByteBuffer.wrap(tableName.getName()));
+ }
+ return list;
+ } catch (IOException e) {
+ LOG.warn(e.getMessage(), e);
+ throw getIOError(e);
+ }
+ }
+
+ /**
+ * @return the list of regions in the given table, or an empty list if the table does not exist
+ */
+ @Override
+ public List<TRegionInfo> getTableRegions(ByteBuffer tableName) throws IOError {
+ try (RegionLocator locator = connectionCache.getRegionLocator(getBytes(tableName))) {
+ List<HRegionLocation> regionLocations = locator.getAllRegionLocations();
+ List<TRegionInfo> results = new ArrayList<>(regionLocations.size());
+ for (HRegionLocation regionLocation : regionLocations) {
+ RegionInfo info = regionLocation.getRegionInfo();
+ ServerName serverName = regionLocation.getServerName();
+ TRegionInfo region = new TRegionInfo();
+ region.serverName = ByteBuffer.wrap(
+ Bytes.toBytes(serverName.getHostname()));
+ region.port = serverName.getPort();
+ region.startKey = ByteBuffer.wrap(info.getStartKey());
+ region.endKey = ByteBuffer.wrap(info.getEndKey());
+ region.id = info.getRegionId();
+ region.name = ByteBuffer.wrap(info.getRegionName());
+ region.version = HREGION_VERSION; // HRegion now not versioned, PB encoding used
+ results.add(region);
+ }
+ return results;
+ } catch (TableNotFoundException e) {
+ // Return empty list for non-existing table
+ return Collections.emptyList();
+ } catch (IOException e){
+ LOG.warn(e.getMessage(), e);
+ throw getIOError(e);
+ }
+ }
+
+ @Override
+ public List<TCell> get(
+ ByteBuffer tableName, ByteBuffer row, ByteBuffer column,
+ Map<ByteBuffer, ByteBuffer> attributes)
+ throws IOError {
+ byte [][] famAndQf = CellUtil.parseColumn(getBytes(column));
+ if (famAndQf.length == 1) {
+ return get(tableName, row, famAndQf[0], null, attributes);
+ }
+ if (famAndQf.length == 2) {
+ return get(tableName, row, famAndQf[0], famAndQf[1], attributes);
+ }
+ throw new IllegalArgumentException("Invalid familyAndQualifier provided.");
+ }
+
+ /**
+ * Note: this internal interface is slightly different from public APIs in regard to handling
+ * of the qualifier. Here we differ from the public Java API in that null != byte[0]. Rather,
+ * we respect qual == null as a request for the entire column family. The caller (
+ * {@link #get(ByteBuffer, ByteBuffer, ByteBuffer, Map)}) interface IS consistent in that the
+ * column is parse like normal.
+ */
+ protected List<TCell> get(ByteBuffer tableName,
+ ByteBuffer row,
+ byte[] family,
+ byte[] qualifier,
+ Map<ByteBuffer, ByteBuffer> attributes) throws IOError {
+ Table table = null;
+ try {
+ table = getTable(tableName);
+ Get get = new Get(getBytes(row));
+ addAttributes(get, attributes);
+ if (qualifier == null) {
+ get.addFamily(family);
+ } else {
+ get.addColumn(family, qualifier);
+ }
+ Result result = table.get(get);
+ return ThriftUtilities.cellFromHBase(result.rawCells());
+ } catch (IOException e) {
+ LOG.warn(e.getMessage(), e);
+ throw getIOError(e);
+ } finally {
+ closeTable(table);
+ }
+ }
+
+ @Override
+ public List<TCell> getVer(ByteBuffer tableName, ByteBuffer row, ByteBuffer column,
+ int numVersions, Map<ByteBuffer, ByteBuffer> attributes) throws IOError {
+ byte [][] famAndQf = CellUtil.parseColumn(getBytes(column));
+ if(famAndQf.length == 1) {
+ return getVer(tableName, row, famAndQf[0], null, numVersions, attributes);
+ }
+ if (famAndQf.length == 2) {
+ return getVer(tableName, row, famAndQf[0], famAndQf[1], numVersions, attributes);
+ }
+ throw new IllegalArgumentException("Invalid familyAndQualifier provided.");
+
+ }
+
+ /**
+ * Note: this public interface is slightly different from public Java APIs in regard to
+ * handling of the qualifier. Here we differ from the public Java API in that null != byte[0].
+ * Rather, we respect qual == null as a request for the entire column family. If you want to
+ * access the entire column family, use
+ * {@link #getVer(ByteBuffer, ByteBuffer, ByteBuffer, int, Map)} with a {@code column} value
+ * that lacks a {@code ':'}.
+ */
+ 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);
+ Get get = new Get(getBytes(row));
+ addAttributes(get, attributes);
+ if (null == qualifier) {
+ get.addFamily(family);
+ } else {
+ get.addColumn(family, qualifier);
+ }
+ get.setMaxVersions(numVersions);
+ Result result = table.get(get);
+ return ThriftUtilities.cellFromHBase(result.rawCells());
+ } catch (IOException e) {
+ LOG.warn(e.getMessage(), e);
+ throw getIOError(e);
+ } finally{
+ closeTable(table);
+ }
+ }
+
+ @Override
+ public List<TCell> getVerTs(ByteBuffer tableName, ByteBuffer row, ByteBuffer column,
+ long timestamp, int numVersions, Map<ByteBuffer, ByteBuffer> attributes) throws IOError {
+ byte [][] famAndQf = CellUtil.parseColumn(getBytes(column));
+ if (famAndQf.length == 1) {
+ return getVerTs(tableName, row, famAndQf[0], null, timestamp, numVersions, attributes);
+ }
+ if (famAndQf.length == 2) {
+ return getVerTs(tableName, row, famAndQf[0], famAndQf[1], timestamp, numVersions,
+ attributes);
+ }
+ throw new IllegalArgumentException("Invalid familyAndQualifier provided.");
+ }
+
+ /**
+ * Note: this internal interface is slightly different from public APIs in regard to handling
+ * of the qualifier. Here we differ from the public Java API in that null != byte[0]. Rather,
+ * we respect qual == null as a request for the entire column family. The caller (
+ * {@link #getVerTs(ByteBuffer, ByteBuffer, ByteBuffer, long, int, Map)}) interface IS
+ * consistent in that the column is parse like normal.
+ */
+ 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);
+ Get get = new Get(getBytes(row));
+ addAttributes(get, attributes);
+ if (null == qualifier) {
+ get.addFamily(family);
+ } else {
+ get.addColumn(family, qualifier);
+ }
+ get.setTimeRange(0, timestamp);
+ get.setMaxVersions(numVersions);
+ Result result = table.get(get);
+ return ThriftUtilities.cellFromHBase(result.rawCells());
+ } catch (IOException e) {
+ LOG.warn(e.getMessage(), e);
+ throw getIOError(e);
+ } finally{
+ closeTable(table);
+ }
+ }
+
+ @Override
+ public List<TRowResult> getRow(ByteBuffer tableName, ByteBuffer row,
+ Map<ByteBuffer, ByteBuffer> attributes) throws IOError {
+ return getRowWithColumnsTs(tableName, row, null,
+ HConstants.LATEST_TIMESTAMP,
+ attributes);
+ }
+
+ @Override
+ public List<TRowResult> getRowWithColumns(ByteBuffer tableName,
+ ByteBuffer row,
+ List<ByteBuffer> columns,
+ Map<ByteBuffer, ByteBuffer> attributes) throws IOError {
+ return getRowWithColumnsTs(tableName, row, columns,
+ HConstants.LATEST_TIMESTAMP,
+ attributes);
+ }
+
+ @Override
+ public List<TRowResult> getRowTs(ByteBuffer tableName, ByteBuffer row,
+ long timestamp, Map<ByteBuffer, ByteBuffer> attributes) throws IOError {
+ return getRowWithColumnsTs(tableName, row, null,
+ timestamp, attributes);
+ }
+
+ @Override
+ 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);
+ if (columns == null) {
+ Get get = new Get(getBytes(row));
+ addAttributes(get, attributes);
+ get.setTimeRange(0, timestamp);
+ Result result = table.get(get);
+ return ThriftUtilities.rowResultFromHBase(result);
+ }
+ Get get = new Get(getBytes(row));
+ addAttributes(get, attributes);
+ for(ByteBuffer column : columns) {
+ byte [][] famAndQf = CellUtil.parseColumn(getBytes(column));
+ if (famAndQf.length == 1) {
+ get.addFamily(famAndQf[0]);
+ } else {
+ get.addColumn(famAndQf[0], famAndQf[1]);
+ }
+ }
+ get.setTimeRange(0, timestamp);
+ Result result = table.get(get);
+ return ThriftUtilities.rowResultFromHBase(result);
+ } catch (IOException e) {
+ LOG.warn(e.getMessage(), e);
+ throw getIOError(e);
+ } finally{
+ closeTable(table);
+ }
+ }
+
+ @Override
+ public List<TRowResult> getRows(ByteBuffer tableName,
+ List<ByteBuffer> rows,
+ Map<ByteBuffer, ByteBuffer> attributes)
+ throws IOError {
+ return getRowsWithColumnsTs(tableName, rows, null,
+ HConstants.LATEST_TIMESTAMP,
+ attributes);
+ }
+
+ @Override
+ public List<TRowResult> getRowsWithColumns(ByteBuffer tableName,
+ List<ByteBuffer> rows,
+ List<ByteBuffer> columns,
+ Map<ByteBuffer, ByteBuffer> attributes) throws IOError {
+ return getRowsWithColumnsTs(tableName, rows, columns,
+ HConstants.LATEST_TIMESTAMP,
+ attributes);
+ }
+
+ @Override
+ public List<TRowResult> getRowsTs(ByteBuffer tableName,
+ List<ByteBuffer> rows,
+ long timestamp,
+ Map<ByteBuffer, ByteBuffer> attributes) throws IOError {
+ return getRowsWithColumnsTs(tableName, rows, null,
+ timestamp, attributes);
+ }
+
+ @Override
+ public List<TRowResult> getRowsWithColumnsTs(ByteBuffer tableName,
+ 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());
+ table = getTable(tableName);
+ if (metrics != null) {
+ metrics.incNumRowKeysInBatchGet(rows.size());
+ }
+ for (ByteBuffer row : rows) {
+ Get get = new Get(getBytes(row));
+ addAttributes(get, attributes);
+ if (columns != null) {
+
+ for(ByteBuffer column : columns) {
+ byte [][] famAndQf = CellUtil.parseColumn(getBytes(column));
+ if (famAndQf.length == 1) {
+ get.addFamily(famAndQf[0]);
+ } else {
+ get.addColumn(famAndQf[0], famAndQf[1]);
+ }
+ }
+ }
+ get.setTimeRange(0, timestamp);
+ gets.add(get);
+ }
+ Result[] result = table.get(gets);
+ return ThriftUtilities.rowResultFromHBase(result);
+ } catch (IOException e) {
+ LOG.warn(e.getMessage(), e);
+ throw getIOError(e);
+ } finally{
+ closeTable(table);
+ }
+ }
+
+ @Override
+ public void deleteAll(
+ ByteBuffer tableName, ByteBuffer row, ByteBuffer column,
+ Map<ByteBuffer, ByteBuffer> attributes)
+ throws IOError {
+ deleteAllTs(tableName, row, column, HConstants.LATEST_TIMESTAMP,
+ attributes);
+ }
+
+ @Override
+ public void deleteAllTs(ByteBuffer tableName,
+ ByteBuffer row,
+ ByteBuffer column,
+ long timestamp, Map<ByteBuffer, ByteBuffer> attributes) throws IOError {
+ Table table = null;
+ try {
+ table = getTable(tableName);
+ Delete delete = new Delete(getBytes(row));
+ addAttributes(delete, attributes);
+ byte [][] famAndQf = CellUtil.parseColumn(getBytes(column));
+ if (famAndQf.length == 1) {
+ delete.addFamily(famAndQf[0], timestamp);
+ } else {
+ delete.addColumns(famAndQf[0], famAndQf[1], timestamp);
+ }
+ table.delete(delete);
+
+ } catch (IOException e) {
+ LOG.warn(e.getMessage(), e);
+ throw getIOError(e);
+ } finally {
+ closeTable(table);
+ }
+ }
+
+ @Override
+ public void deleteAllRow(
+ ByteBuffer tableName, ByteBuffer row,
+ Map<ByteBuffer, ByteBuffer> attributes) throws IOError {
+ deleteAllRowTs(tableName, row, HConstants.LATEST_TIMESTAMP, attributes);
+ }
+
+ @Override
+ public void deleteAllRowTs(
+ ByteBuffer tableName, ByteBuffer row, long timestamp,
+ Map<ByteBuffer, ByteBuffer> attributes) throws IOError {
+ Table table = null;
+ try {
+ table = getTable(tableName);
+ Delete delete = new Delete(getBytes(row), timestamp);
+ addAttributes(delete, attributes);
+ table.delete(delete);
+ } catch (IOException e) {
+ LOG.warn(e.getMessage(), e);
+ throw getIOError(e);
+ } finally {
+ closeTable(table);
+ }
+ }
+
+ @Override
+ public void createTable(ByteBuffer in_tableName,
+ List<ColumnDescriptor> columnFamilies) throws IOError, IllegalArgument, AlreadyExists {
+ TableName tableName = getTableName(in_tableName);
+ try {
+ if (getAdmin().tableExists(tableName)) {
+ throw new AlreadyExists("table name already in use");
+ }
+ HTableDescriptor desc = new HTableDescriptor(tableName);
+ for (ColumnDescriptor col : columnFamilies) {
+ HColumnDescriptor colDesc = ThriftUtilities.colDescFromThrift(col);
+ desc.addFamily(colDesc);
+ }
+ getAdmin().createTable(desc);
+ } catch (IOException e) {
+ LOG.warn(e.getMessage(), e);
+ throw getIOError(e);
+ } catch (IllegalArgumentException e) {
+ LOG.warn(e.getMessage(), e);
+ throw new IllegalArgument(Throwables.getStackTraceAsString(e));
+ }
+ }
+
+ private static TableName getTableName(ByteBuffer buffer) {
+ return TableName.valueOf(getBytes(buffer));
+ }
+
+ @Override
+ public void deleteTable(ByteBuffer in_tableName) throws IOError {
+ TableName tableName = getTableName(in_tableName);
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("deleteTable: table={}", tableName);
+ }
+ try {
+ if (!getAdmin().tableExists(tableName)) {
+ throw new IOException("table does not exist");
+ }
+ getAdmin().deleteTable(tableName);
+ } catch (IOException e) {
+ LOG.warn(e.getMessage(), e);
+ throw getIOError(e);
+ }
+ }
+
+ @Override
+ public void mutateRow(ByteBuffer tableName, ByteBuffer row,
+ List<Mutation> mutations, Map<ByteBuffer, ByteBuffer> attributes)
+ throws IOError, IllegalArgument {
+ mutateRowTs(tableName, row, mutations, HConstants.LATEST_TIMESTAMP, attributes);
+ }
+
+ @Override
+ public void mutateRowTs(ByteBuffer tableName, ByteBuffer row,
+ List<Mutation> mutations, long timestamp,
+ Map<ByteBuffer, ByteBuffer> attributes)
+ throws IOError, IllegalArgument {
+ Table table = null;
+ try {
+ table = getTable(tableName);
+ Put put = new Put(getBytes(row), timestamp);
+ addAttributes(put, attributes);
+
+ Delete delete = new Delete(getBytes(row));
+ addAttributes(delete, attributes);
+ if (metrics != null) {
+ metrics.incNumRowKeysInBatchMutate(mutations.size());
+ }
+
+ // I apologize for all this mess :)
+ CellBuilder builder = CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY);
+ for (Mutation m : mutations) {
+ byte[][] famAndQf = CellUtil.parseColumn(getBytes(m.column));
+ if (m.isDelete) {
+ if (famAndQf.length == 1) {
+ delete.addFamily(famAndQf[0], timestamp);
+ } else {
+ delete.addColumns(famAndQf[0], famAndQf[1], timestamp);
+ }
+ delete.setDurability(m.writeToWAL ? Durability.SYNC_WAL : Durability.SKIP_WAL);
+ } else {
+ if(famAndQf.length == 1) {
+ LOG.warn("No column qualifier specified. Delete is the only mutation supported "
+ + "over the whole column family.");
+ } else {
+ put.add(builder.clear()
+ .setRow(put.getRow())
+ .setFamily(famAndQf[0])
+ .setQualifier(famAndQf[1])
+ .setTimestamp(put.getTimestamp())
+ .setType(Cell.Type.Put)
+ .setValue(m.value != null ? getBytes(m.value)
+ : HConstants.EMPTY_BYTE_ARRAY)
+ .build());
+ }
+ put.setDurability(m.writeToWAL ? Durability.SYNC_WAL : Durability.SKIP_WAL);
+ }
+ }
+ if (!delete.isEmpty()) {
+ table.delete(delete);
+ }
+ if (!put.isEmpty()) {
+ table.put(put);
+ }
+ } catch (IOException e) {
+ LOG.warn(e.getMessage(), e);
+ throw getIOError(e);
+ } catch (IllegalArgumentException e) {
+ LOG.warn(e.getMessage(), e);
+ throw new IllegalArgument(Throwables.getStackTraceAsString(e));
+ } finally{
+ closeTable(table);
+ }
+ }
+
+ @Override
+ public void mutateRows(ByteBuffer tableName, List<BatchMutation> rowBatches,
+ Map<ByteBuffer, ByteBuffer> attributes)
+ throws IOError, IllegalArgument, TException {
+ mutateRowsTs(tableName, rowBatches, HConstants.LATEST_TIMESTAMP, attributes);
+ }
+
+ @Override
+ public void mutateRowsTs(
+ ByteBuffer tableName, List<BatchMutation> rowBatches, long timestamp,
+ Map<ByteBuffer, ByteBuffer> attributes)
+ throws IOError, IllegalArgument, TException {
+ List<Put> puts = new ArrayList<>();
+ List<Delete> deletes = new ArrayList<>();
+ CellBuilder builder = CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY);
+ for (BatchMutation batch : rowBatches) {
+ byte[] row = getBytes(batch.row);
+ List<Mutation> mutations = batch.mutations;
+ Delete delete = new Delete(row);
+ addAttributes(delete, attributes);
+ Put put = new Put(row, timestamp);
+ addAttributes(put, attributes);
+ for (Mutation m : mutations) {
+ byte[][] famAndQf = CellUtil.parseColumn(getBytes(m.column));
+ if (m.isDelete) {
+ // no qualifier, family only.
+ if (famAndQf.length == 1) {
+ delete.addFamily(famAndQf[0], timestamp);
+ } else {
+ delete.addColumns(famAndQf[0], famAndQf[1], timestamp);
+ }
+ delete.setDurability(m.writeToWAL ? Durability.SYNC_WAL
+ : Durability.SKIP_WAL);
+ } else {
+ if (famAndQf.length == 1) {
+ LOG.warn("No column qualifier specified. Delete is the only mutation supported "
+ + "over the whole column family.");
+ }
+ if (famAndQf.length == 2) {
+ try {
+ put.add(builder.clear()
+ .setRow(put.getRow())
+ .setFamily(famAndQf[0])
+ .setQualifier(famAndQf[1])
+ .setTimestamp(put.getTimestamp())
+ .setType(Cell.Type.Put)
+ .setValue(m.value != null ? getBytes(m.value)
+ : HConstants.EMPTY_BYTE_ARRAY)
+ .build());
+ } catch (IOException e) {
+ throw new IllegalArgumentException(e);
+ }
+ } else {
+ throw new IllegalArgumentException("Invalid famAndQf provided.");
+ }
+ put.setDurability(m.writeToWAL ? Durability.SYNC_WAL : Durability.SKIP_WAL);
+ }
+ }
+ if (!delete.isEmpty()) {
+ deletes.add(delete);
+ }
+ if (!put.isEmpty()) {
+ puts.add(put);
+ }
+ }
+
+ Table table = null;
+ try {
+ table = getTable(tableName);
+ if (!puts.isEmpty()) {
+ table.put(puts);
+ }
+ if (!deletes.isEmpty()) {
+ table.delete(deletes);
+ }
+ } catch (IOException e) {
+ LOG.warn(e.getMessage(), e);
+ throw getIOError(e);
+ } catch (IllegalArgumentException e) {
+ LOG.warn(e.getMessage(), e);
+ throw new IllegalArgument(Throwables.getStackTraceAsString(e));
+ } finally{
+ closeTable(table);
+ }
+ }
+
+ @Override
+ public long atomicIncrement(
+ ByteBuffer tableName, ByteBuffer row, ByteBuffer column, long amount)
+ throws IOError, IllegalArgument, TException {
+ byte [][] famAndQf = CellUtil.parseColumn(getBytes(column));
+ if(famAndQf.length == 1) {
+ return atomicIncrement(tableName, row, famAndQf[0], HConstants.EMPTY_BYTE_ARRAY, amount);
+ }
+ return atomicIncrement(tableName, row, famAndQf[0], famAndQf[1], amount);
+ }
+
+ protected long atomicIncrement(ByteBuffer tableName, ByteBuffer row,
+ byte [] family, byte [] qualifier, long amount)
+ throws IOError, IllegalArgument, TException {
+ Table table = null;
+ try {
+ table = getTable(tableName);
+ return table.incrementColumnValue(
+ getBytes(row), family, qualifier, amount);
+ } catch (IOException e) {
+ LOG.warn(e.getMessage(), e);
+ throw getIOError(e);
+ } finally {
+ closeTable(table);
+ }
+ }
+
+ @Override
+ public void scannerClose(int id) throws IOError, IllegalArgument {
+ LOG.debug("scannerClose: id={}", id);
+ ResultScannerWrapper resultScannerWrapper = getScanner(id);
+ if (resultScannerWrapper == null) {
+ LOG.warn("scanner ID is invalid");
+ throw new IllegalArgument("scanner ID is invalid");
+ }
+ resultScannerWrapper.getScanner().close();
+ removeScanner(id);
+ }
+
+ @Override
+ public List<TRowResult> scannerGetList(int id,int nbRows)
+ throws IllegalArgument, IOError {
+ LOG.debug("scannerGetList: id={}", id);
+ ResultScannerWrapper resultScannerWrapper = getScanner(id);
+ if (null == resultScannerWrapper) {
+ String message = "scanner ID is invalid";
+ LOG.warn(message);
+ throw new IllegalArgument("scanner ID is invalid");
+ }
+
+ Result [] results;
+ try {
+ results = resultScannerWrapper.getScanner().next(nbRows);
+ if (null == results) {
+ return new ArrayList<>();
+ }
+ } catch (IOException e) {
+ LOG.warn(e.getMessage(), e);
+ throw getIOError(e);
+ }
+ return ThriftUtilities.rowResultFromHBase(results, resultScannerWrapper.isColumnSorted());
+ }
+
+ @Override
+ public List<TRowResult> scannerGet(int id) throws IllegalArgument, IOError {
+ return scannerGetList(id,1);
+ }
+
+ @Override
+ public int scannerOpenWithScan(ByteBuffer tableName, TScan tScan,
+ Map<ByteBuffer, ByteBuffer> attributes)
+ throws IOError {
+
+ Table table = null;
+ try {
+ table = getTable(tableName);
+ Scan scan = new Scan();
+ addAttributes(scan, attributes);
+ if (tScan.isSetStartRow()) {
+ scan.setStartRow(tScan.getStartRow());
+ }
+ if (tScan.isSetStopRow()) {
+ scan.setStopRow(tScan.getStopRow());
+ }
+ if (tScan.isSetTimestamp()) {
+ scan.setTimeRange(0, tScan.getTimestamp());
+ }
+ if (tScan.isSetCaching()) {
+ scan.setCaching(tScan.getCaching());
+ }
+ if (tScan.isSetBatchSize()) {
+ scan.setBatch(tScan.getBatchSize());
+ }
+ if (tScan.isSetColumns() && !tScan.getColumns().isEmpty()) {
+ for(ByteBuffer column : tScan.getColumns()) {
+ byte [][] famQf = CellUtil.parseColumn(getBytes(column));
+ if(famQf.length == 1) {
+ scan.addFamily(famQf[0]);
+ } else {
+ scan.addColumn(famQf[0], famQf[1]);
+ }
+ }
+ }
+ if (tScan.isSetFilterString()) {
+ ParseFilter parseFilter = new ParseFilter();
+ scan.setFilter(
+ parseFilter.parseFilterString(tScan.getFilterString()));
+ }
+ if (tScan.isSetReversed()) {
+ scan.setReversed(tScan.isReversed());
+ }
+ if (tScan.isSetCacheBlocks()) {
+ scan.setCacheBlocks(tScan.isCacheBlocks());
+ }
+ return addScanner(table.getScanner(scan), tScan.sortColumns);
+ } catch (IOException e) {
+ LOG.warn(e.getMessage(), e);
+ throw getIOError(e);
+ } finally{
+ closeTable(table);
+ }
+ }
+
+ @Override
+ public int scannerOpen(ByteBuffer tableName, ByteBuffer startRow,
+ List<ByteBuffer> columns,
+ Map<ByteBuffer, ByteBuffer> attributes) throws IOError {
+
+ Table table = null;
+ try {
+ table = getTable(tableName);
+ Scan scan = new Scan(getBytes(startRow));
+ addAttributes(scan, attributes);
+ if(columns != null && !columns.isEmpty()) {
+ for(ByteBuffer column : columns) {
+ byte [][] famQf = CellUtil.parseColumn(getBytes(column));
+ if(famQf.length == 1) {
+ scan.addFamily(famQf[0]);
+ } else {
+ scan.addColumn(famQf[0], famQf[1]);
+ }
+ }
+ }
+ return addScanner(table.getScanner(scan), false);
+ } catch (IOException e) {
+ LOG.warn(e.getMessage(), e);
+ throw getIOError(e);
+ } finally{
+ closeTable(table);
+ }
+ }
+
+ @Override
+ public int scannerOpenWithStop(ByteBuffer tableName, ByteBuffer startRow,
+ ByteBuffer stopRow, List<ByteBuffer> columns,
+ Map<ByteBuffer, ByteBuffer> attributes)
+ throws IOError, TException {
+
+ Table table = null;
+ try {
+ table = getTable(tableName);
+ Scan scan = new Scan(getBytes(startRow), getBytes(stopRow));
+ addAttributes(scan, attributes);
+ if(columns != null && !columns.isEmpty()) {
+ for(ByteBuffer column : columns) {
+ byte [][] famQf = CellUtil.parseColumn(getBytes(column));
+ if(famQf.length == 1) {
+ scan.addFamily(famQf[0]);
+ } else {
+ scan.addColumn(famQf[0], famQf[1]);
+ }
+ }
+ }
+ return addScanner(table.getScanner(scan), false);
+ } catch (IOException e) {
+ LOG.warn(e.getMessage(), e);
+ throw getIOError(e);
+ } finally{
+ closeTable(table);
+ }
+ }
+
+ @Override
+ public int scannerOpenWithPrefix(ByteBuffer tableName,
+ ByteBuffer startAndPrefix,
+ List<ByteBuffer> columns,
+ Map<ByteBuffer, ByteBuffer> attributes)
+ throws IOError, TException {
+
+ Table table = null;
+ try {
+ table = getTable(tableName);
+ Scan scan = new Scan(getBytes(startAndPrefix));
+ addAttributes(scan, attributes);
+ Filter f = new WhileMatchFilter(
+ new PrefixFilter(getBytes(startAndPrefix)));
+ scan.setFilter(f);
+ if (columns != null && !columns.isEmpty()) {
+ for(ByteBuffer column : columns) {
+ byte [][] famQf = CellUtil.parseColumn(getBytes(column));
+ if(famQf.length == 1) {
+ scan.addFamily(famQf[0]);
+ } else {
+ scan.addColumn(famQf[0], famQf[1]);
+ }
+ }
+ }
+ return addScanner(table.getScanner(scan), false);
+ } catch (IOException e) {
+ LOG.warn(e.getMessage(), e);
+ throw getIOError(e);
+ } finally{
+ closeTable(table);
+ }
+ }
+
+ @Override
+ 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);
+ Scan scan = new Scan(getBytes(startRow));
+ addAttributes(scan, attributes);
+ scan.setTimeRange(0, timestamp);
+ if (columns != null && !columns.isEmpty()) {
+ for (ByteBuffer column : columns) {
+ byte [][] famQf = CellUtil.parseColumn(getBytes(column));
+ if(famQf.length == 1) {
+ scan.addFamily(famQf[0]);
+ } else {
+ scan.addColumn(famQf[0], famQf[1]);
+ }
+ }
+ }
+ return addScanner(table.getScanner(scan), false);
+ } catch (IOException e) {
+ LOG.warn(e.getMessage(), e);
+ throw getIOError(e);
+ } finally{
+ closeTable(table);
+ }
+ }
+
+ @Override
+ public int scannerOpenWithStopTs(ByteBuffer tableName, ByteBuffer startRow,
+ ByteBuffer stopRow, List<ByteBuffer> columns, long timestamp,
+ Map<ByteBuffer, ByteBuffer> attributes)
+ throws IOError, TException {
+
+ Table table = null;
+ try {
+ table = getTable(tableName);
+ Scan scan = new Scan(getBytes(startRow), getBytes(stopRow));
+ addAttributes(scan, attributes);
+ scan.setTimeRange(0, timestamp);
+ if (columns != null && !columns.isEmpty()) {
+ for (ByteBuffer column : columns) {
+ byte [][] famQf = CellUtil.parseColumn(getBytes(column));
+ if(famQf.length == 1) {
+ scan.addFamily(famQf[0]);
+ } else {
+ scan.addColumn(famQf[0], famQf[1]);
+ }
+ }
+ }
+ scan.setTimeRange(0, timestamp);
+ return addScanner(table.getScanner(scan), false);
+ } catch (IOException e) {
+ LOG.warn(e.getMessage(), e);
+ throw getIOError(e);
+ } finally{
+ closeTable(table);
+ }
+ }
+
+ @Override
+ public Map<ByteBuffer, ColumnDescriptor> getColumnDescriptors(
+ ByteBuffer tableName) throws IOError, TException {
+
+ Table table = null;
+ try {
+ TreeMap<ByteBuffer, ColumnDescriptor> columns = new TreeMap<>();
+
+ table = getTable(tableName);
+ HTableDescriptor desc = table.getTableDescriptor();
+
+ for (HColumnDescriptor e : desc.getFamilies()) {
+ ColumnDescriptor col = ThriftUtilities.colDescFromHbase(e);
+ columns.put(col.name, col);
+ }
+ return columns;
+ } catch (IOException e) {
+ LOG.warn(e.getMessage(), e);
+ throw getIOError(e);
+ } finally {
+ closeTable(table);
+ }
+ }
+
+ private void closeTable(Table table) throws IOError {
+ try{
+ if(table != null){
+ table.close();
+ }
+ } catch (IOException e){
+ LOG.error(e.getMessage(), e);
+ throw getIOError(e);
+ }
+ }
+
+ @Override
+ public TRegionInfo getRegionInfo(ByteBuffer searchRow) throws IOError {
+ try {
+ byte[] row = getBytes(searchRow);
+ Result startRowResult = getReverseScanResult(TableName.META_TABLE_NAME.getName(), row,
+ HConstants.CATALOG_FAMILY);
+
+ if (startRowResult == null) {
+ throw new IOException("Cannot find row in "+ TableName.META_TABLE_NAME+", row="
+ + Bytes.toStringBinary(row));
+ }
+
+ // find region start and end keys
+ RegionInfo regionInfo = MetaTableAccessor.getRegionInfo(startRowResult);
+ if (regionInfo == null) {
+ throw new IOException("RegionInfo REGIONINFO was null or " +
+ " empty in Meta for row="
+ + Bytes.toStringBinary(row));
+ }
+ TRegionInfo region = new TRegionInfo();
+ region.setStartKey(regionInfo.getStartKey());
+ region.setEndKey(regionInfo.getEndKey());
+ region.id = regionInfo.getRegionId();
+ region.setName(regionInfo.getRegionName());
+ region.version = HREGION_VERSION; // version not used anymore, PB encoding used.
+
+ // find region assignment to server
+ ServerName serverName = MetaTableAccessor.getServerName(startRowResult, 0);
+ if (serverName != null) {
+ region.setServerName(Bytes.toBytes(serverName.getHostname()));
+ region.port = serverName.getPort();
+ }
+ return region;
+ } catch (IOException e) {
+ LOG.warn(e.getMessage(), e);
+ throw getIOError(e);
+ }
+ }
+
+ private Result getReverseScanResult(byte[] tableName, byte[] row, byte[] family)
+ throws IOException {
+ Scan scan = new Scan(row);
+ scan.setReversed(true);
+ scan.addFamily(family);
+ scan.setStartRow(row);
+ try (Table table = getTable(tableName);
+ ResultScanner scanner = table.getScanner(scan)) {
+ return scanner.next();
+ }
+ }
+
+ @Override
+ public void increment(TIncrement tincrement) throws IOError, TException {
+
+ if (tincrement.getRow().length == 0 || tincrement.getTable().length == 0) {
+ throw new TException("Must supply a table and a row key; can't increment");
+ }
+
+ if (conf.getBoolean(COALESCE_INC_KEY, false)) {
+ this.coalescer.queueIncrement(tincrement);
+ return;
+ }
+
+ Table table = null;
+ try {
+ table = getTable(tincrement.getTable());
+ Increment inc = ThriftUtilities.incrementFromThrift(tincrement);
+ table.increment(inc);
+ } catch (IOException e) {
+ LOG.warn(e.getMessage(), e);
+ throw getIOError(e);
+ } finally{
+ closeTable(table);
+ }
+ }
+
+ @Override
+ public void incrementRows(List<TIncrement> tincrements) throws IOError, TException {
+ if (conf.getBoolean(COALESCE_INC_KEY, false)) {
+ this.coalescer.queueIncrements(tincrements);
+ return;
+ }
+ for (TIncrement tinc : tincrements) {
+ increment(tinc);
+ }
+ }
+
+ @Override
+ public List<TCell> append(TAppend tappend) throws IOError, TException {
+ if (tappend.getRow().length == 0 || tappend.getTable().length == 0) {
+ throw new TException("Must supply a table and a row key; can't append");
+ }
+
+ Table table = null;
+ try {
+ table = getTable(tappend.getTable());
+ Append append = ThriftUtilities.appendFromThrift(tappend);
+ Result result = table.append(append);
+ return ThriftUtilities.cellFromHBase(result.rawCells());
+ } catch (IOException e) {
+ LOG.warn(e.getMessage(), e);
+ throw getIOError(e);
+ } finally{
+ closeTable(table);
+ }
+ }
+
+ @Override
+ public boolean checkAndPut(ByteBuffer tableName, ByteBuffer row, ByteBuffer column,
+ ByteBuffer value, Mutation mput, Map<ByteBuffer, ByteBuffer> attributes) throws IOError,
+ IllegalArgument, TException {
+ Put put;
+ try {
+ put = new Put(getBytes(row), HConstants.LATEST_TIMESTAMP);
+ addAttributes(put, attributes);
+
+ byte[][] famAndQf = CellUtil.parseColumn(getBytes(mput.column));
+ put.add(CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY)
+ .setRow(put.getRow())
+ .setFamily(famAndQf[0])
+ .setQualifier(famAndQf[1])
+ .setTimestamp(put.getTimestamp())
+ .setType(Cell.Type.Put)
+ .setValue(mput.value != null ? getBytes(mput.value)
+ : HConstants.EMPTY_BYTE_ARRAY)
+ .build());
+ put.setDurability(mput.writeToWAL ? Durability.SYNC_WAL : Durability.SKIP_WAL);
+ } catch (IOException | IllegalArgumentException e) {
+ LOG.warn(e.getMessage(), e);
+ throw new IllegalArgument(Throwables.getStackTraceAsString(e));
+ }
+
+ Table table = null;
+ try {
+ table = getTable(tableName);
+ byte[][] famAndQf = CellUtil.parseColumn(getBytes(column));
+ Table.CheckAndMutateBuilder mutateBuilder =
+ table.checkAndMutate(getBytes(row), famAndQf[0]).qualifier(famAndQf[1]);
+ if (value != null) {
+ return mutateBuilder.ifEquals(getBytes(value)).thenPut(put);
+ } else {
+ return mutateBuilder.ifNotExists().thenPut(put);
+ }
+ } catch (IOException e) {
+ LOG.warn(e.getMessage(), e);
+ throw getIOError(e);
+ } catch (IllegalArgumentException e) {
+ LOG.warn(e.getMessage(), e);
+ throw new IllegalArgument(Throwables.getStackTraceAsString(e));
+ } finally {
+ closeTable(table);
+ }
+ }
+
+ private static IOError getIOError(Throwable throwable) {
+ IOError error = new IOErrorWithCause(throwable);
+ error.setMessage(Throwables.getStackTraceAsString(throwable));
+ return error;
+ }
+
+ /**
+ * Adds all the attributes into the Operation object
+ */
+ private static void addAttributes(OperationWithAttributes op,
+ Map<ByteBuffer, ByteBuffer> attributes) {
+ if (attributes == null || attributes.isEmpty()) {
+ return;
+ }
+ for (Map.Entry<ByteBuffer, ByteBuffer> entry : attributes.entrySet()) {
+ String name = Bytes.toStringBinary(getBytes(entry.getKey()));
+ byte[] value = getBytes(entry.getValue());
+ op.setAttribute(name, value);
+ }
+ }
+
+ protected static class ResultScannerWrapper {
+
+ private final ResultScanner scanner;
+ private final boolean sortColumns;
+ public ResultScannerWrapper(ResultScanner resultScanner,
+ boolean sortResultColumns) {
+ scanner = resultScanner;
+ sortColumns = sortResultColumns;
+ }
+
+ public ResultScanner getScanner() {
+ return scanner;
+ }
+
+ public boolean isColumnSorted() {
+ return sortColumns;
+ }
+ }
+
+ public static class IOErrorWithCause extends IOError {
+ private final Throwable cause;
+ public IOErrorWithCause(Throwable cause) {
+ this.cause = cause;
+ }
+
+ @Override
+ public synchronized Throwable getCause() {
+ return cause;
+ }
+
+ @Override
+ public boolean equals(Object other) {
+ if (super.equals(other) &&
+ other instanceof IOErrorWithCause) {
+ Throwable otherCause = ((IOErrorWithCause) other).getCause();
+ if (this.getCause() != null) {
+ return otherCause != null && this.getCause().equals(otherCause);
+ } else {
+ return otherCause == null;
+ }
+ }
+ return false;
+ }
+
+ @Override
+ public int hashCode() {
+ int result = super.hashCode();
+ result = 31 * result + (cause != null ? cause.hashCode() : 0);
+ return result;
+ }
+ }
+
+
+}
http://git-wip-us.apache.org/repos/asf/hbase/blob/e4b6b4af/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftHttpServlet.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftHttpServlet.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftHttpServlet.java
index 4c9a35b..7f85152 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftHttpServlet.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftHttpServlet.java
@@ -18,8 +18,8 @@
package org.apache.hadoop.hbase.thrift;
-import static org.apache.hadoop.hbase.thrift.ThriftServerRunner.THRIFT_SPNEGO_KEYTAB_FILE_KEY;
-import static org.apache.hadoop.hbase.thrift.ThriftServerRunner.THRIFT_SPNEGO_PRINCIPAL_KEY;
+import static org.apache.hadoop.hbase.thrift.Constants.THRIFT_SPNEGO_KEYTAB_FILE_KEY;
+import static org.apache.hadoop.hbase.thrift.Constants.THRIFT_SPNEGO_PRINCIPAL_KEY;
import java.io.IOException;
import java.security.PrivilegedExceptionAction;
@@ -58,7 +58,7 @@ public class ThriftHttpServlet extends TServlet {
private static final Logger LOG = LoggerFactory.getLogger(ThriftHttpServlet.class.getName());
private final transient UserGroupInformation serviceUGI;
private final transient UserGroupInformation httpUGI;
- private final transient ThriftServerRunner.HBaseHandler hbaseHandler;
+ private final transient HBaseServiceHandler handler;
private final boolean doAsEnabled;
private final boolean securityEnabled;
@@ -67,11 +67,11 @@ public class ThriftHttpServlet extends TServlet {
public ThriftHttpServlet(TProcessor processor, TProtocolFactory protocolFactory,
UserGroupInformation serviceUGI, Configuration conf,
- ThriftServerRunner.HBaseHandler hbaseHandler, boolean securityEnabled, boolean doAsEnabled)
+ HBaseServiceHandler handler, boolean securityEnabled, boolean doAsEnabled)
throws IOException {
super(processor, protocolFactory);
this.serviceUGI = serviceUGI;
- this.hbaseHandler = hbaseHandler;
+ this.handler = handler;
this.securityEnabled = securityEnabled;
this.doAsEnabled = doAsEnabled;
@@ -146,7 +146,7 @@ public class ThriftHttpServlet extends TServlet {
}
effectiveUser = doAsUserFromQuery;
}
- hbaseHandler.setEffectiveUser(effectiveUser);
+ handler.setEffectiveUser(effectiveUser);
super.doPost(request, response);
}
[11/15] hbase git commit: HBASE-21630 [shell] Define ENDKEY ==
STOPROW (we have ENDROW)
Posted by zh...@apache.org.
HBASE-21630 [shell] Define ENDKEY == STOPROW (we have ENDROW)
Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/3fbdd5bb
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/3fbdd5bb
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/3fbdd5bb
Branch: refs/heads/HBASE-21512
Commit: 3fbdd5bbe9937bdb12984275dafced9cb6746a63
Parents: c300565
Author: Nihal Jain <ni...@gmail.com>
Authored: Wed Jan 2 01:24:08 2019 +0530
Committer: stack <st...@apache.org>
Committed: Thu Jan 3 21:58:28 2019 -0800
----------------------------------------------------------------------
hbase-shell/src/main/ruby/hbase_constants.rb | 2 ++
hbase-shell/src/test/ruby/hbase/table_test.rb | 36 ++++++++++++++++++++--
2 files changed, 35 insertions(+), 3 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/hbase/blob/3fbdd5bb/hbase-shell/src/main/ruby/hbase_constants.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/hbase_constants.rb b/hbase-shell/src/main/ruby/hbase_constants.rb
index 4c1ad22..9871685 100644
--- a/hbase-shell/src/main/ruby/hbase_constants.rb
+++ b/hbase-shell/src/main/ruby/hbase_constants.rb
@@ -43,8 +43,10 @@ module HBaseConstants
METADATA = org.apache.hadoop.hbase.HConstants::METADATA
STOPROW = 'STOPROW'.freeze
STARTROW = 'STARTROW'.freeze
+ STARTKEY = STARTROW
ROWPREFIXFILTER = 'ROWPREFIXFILTER'.freeze
ENDROW = STOPROW
+ ENDKEY = STOPROW
RAW = 'RAW'.freeze
LIMIT = 'LIMIT'.freeze
METHOD = 'METHOD'.freeze
http://git-wip-us.apache.org/repos/asf/hbase/blob/3fbdd5bb/hbase-shell/src/test/ruby/hbase/table_test.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/test/ruby/hbase/table_test.rb b/hbase-shell/src/test/ruby/hbase/table_test.rb
index 5ec317a..b3343e2 100644
--- a/hbase-shell/src/test/ruby/hbase/table_test.rb
+++ b/hbase-shell/src/test/ruby/hbase/table_test.rb
@@ -476,6 +476,16 @@ module Hbase
assert_not_nil(res['2']['x:b'])
end
+ define_test "scan should support STARTKEY parameter" do
+ res = @test_table._scan_internal STARTKEY => '2'
+ assert_not_nil(res)
+ assert_kind_of(Hash, res)
+ assert_nil(res['1'])
+ assert_not_nil(res['2'])
+ assert_not_nil(res['2']['x:a'])
+ assert_not_nil(res['2']['x:b'])
+ end
+
define_test "scan should support STOPROW parameter" do
res = @test_table._scan_internal STOPROW => '2'
assert_not_nil(res)
@@ -486,7 +496,27 @@ module Hbase
assert_nil(res['2'])
end
- define_test "scan should support ROWPREFIXFILTER parameter (test 1)" do
+ define_test "scan should support ENDROW parameter" do
+ res = @test_table._scan_internal ENDROW => '2'
+ assert_not_nil(res)
+ assert_kind_of(Hash, res)
+ assert_not_nil(res['1'])
+ assert_not_nil(res['1']['x:a'])
+ assert_not_nil(res['1']['x:b'])
+ assert_nil(res['2'])
+ end
+
+ define_test "scan should support ENDKEY parameter" do
+ res = @test_table._scan_internal ENDKEY => '2'
+ assert_not_nil(res)
+ assert_kind_of(Hash, res)
+ assert_not_nil(res['1'])
+ assert_not_nil(res['1']['x:a'])
+ assert_not_nil(res['1']['x:b'])
+ assert_nil(res['2'])
+ end
+
+ define_test 'scan should support ROWPREFIXFILTER parameter (test 1)' do
res = @test_table._scan_internal ROWPREFIXFILTER => '1'
assert_not_nil(res)
assert_kind_of(Hash, res)
@@ -496,7 +526,7 @@ module Hbase
assert_nil(res['2'])
end
- define_test "scan should support ROWPREFIXFILTER parameter (test 2)" do
+ define_test 'scan should support ROWPREFIXFILTER parameter (test 2)' do
res = @test_table._scan_internal ROWPREFIXFILTER => '2'
assert_not_nil(res)
assert_kind_of(Hash, res)
@@ -506,7 +536,7 @@ module Hbase
assert_not_nil(res['2']['x:b'])
end
- define_test "scan should support LIMIT parameter" do
+ define_test 'scan should support LIMIT parameter' do
res = @test_table._scan_internal LIMIT => 1
assert_not_nil(res)
assert_kind_of(Hash, res)
[06/15] hbase git commit: HBASE-17356 Add replica get support
Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/db66e6cc/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 d705d7c..28db7e8 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
@@ -20,9 +20,9 @@ package org.apache.hadoop.hbase.client;
import static java.util.stream.Collectors.toList;
import static org.apache.hadoop.hbase.client.ConnectionUtils.checkHasFamilies;
import static org.apache.hadoop.hbase.client.ConnectionUtils.isEmptyStopRow;
+import static org.apache.hadoop.hbase.util.FutureUtils.addListener;
import com.google.protobuf.RpcChannel;
-
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
@@ -32,11 +32,11 @@ import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.function.Function;
-
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.CompareOperator;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.RegionLocations;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.AsyncRpcRetryingCallerFactory.SingleRequestCallerBuilder;
import org.apache.hadoop.hbase.filter.BinaryComparator;
@@ -45,9 +45,12 @@ 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.slf4j.Logger;
+import org.slf4j.LoggerFactory;
import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
+
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
import org.apache.hadoop.hbase.shaded.protobuf.ResponseConverter;
@@ -63,7 +66,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.CompareType
/**
* The implementation of RawAsyncTable.
- * <p>
+ * <p/>
* The word 'Raw' means that this is a low level class. The returned {@link CompletableFuture} will
* be finished inside the rpc framework thread, which means that the callbacks registered to the
* {@link CompletableFuture} will also be executed inside the rpc framework thread. So users who use
@@ -74,6 +77,8 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.CompareType
@InterfaceAudience.Private
class RawAsyncTableImpl implements AsyncTable<AdvancedScanResultConsumer> {
+ private static final Logger LOG = LoggerFactory.getLogger(RawAsyncTableImpl.class);
+
private final AsyncConnectionImpl conn;
private final TableName tableName;
@@ -204,58 +209,126 @@ class RawAsyncTableImpl implements AsyncTable<AdvancedScanResultConsumer> {
private <T> SingleRequestCallerBuilder<T> newCaller(byte[] row, long rpcTimeoutNs) {
return conn.callerFactory.<T> single().table(tableName).row(row)
- .rpcTimeout(rpcTimeoutNs, TimeUnit.NANOSECONDS)
- .operationTimeout(operationTimeoutNs, TimeUnit.NANOSECONDS)
- .pause(pauseNs, TimeUnit.NANOSECONDS).maxAttempts(maxAttempts)
- .startLogErrorsCnt(startLogErrorsCnt);
+ .rpcTimeout(rpcTimeoutNs, TimeUnit.NANOSECONDS)
+ .operationTimeout(operationTimeoutNs, TimeUnit.NANOSECONDS)
+ .pause(pauseNs, TimeUnit.NANOSECONDS).maxAttempts(maxAttempts)
+ .startLogErrorsCnt(startLogErrorsCnt);
}
private <T> SingleRequestCallerBuilder<T> newCaller(Row row, long rpcTimeoutNs) {
return newCaller(row.getRow(), rpcTimeoutNs);
}
+ private CompletableFuture<Result> get(Get get, int replicaId, long timeoutNs) {
+ return this.<Result> newCaller(get, timeoutNs)
+ .action((controller, loc, stub) -> RawAsyncTableImpl
+ .<Get, GetRequest, GetResponse, Result> call(controller, loc, stub, get,
+ RequestConverter::buildGetRequest, (s, c, req, done) -> s.get(c, req, done),
+ (c, resp) -> ProtobufUtil.toResult(resp.getResult(), c.cellScanner())))
+ .replicaId(replicaId).call();
+ }
+
+ // Connect the two futures, if the src future is done, then mark the dst future as done. And if
+ // the dst future is done, then cancel the src future. This is used for timeline consistent read.
+ private <T> void connect(CompletableFuture<T> srcFuture, CompletableFuture<T> dstFuture) {
+ addListener(srcFuture, (r, e) -> {
+ if (e != null) {
+ dstFuture.completeExceptionally(e);
+ } else {
+ dstFuture.complete(r);
+ }
+ });
+ // The cancellation may be a dummy one as the dstFuture may be completed by this srcFuture.
+ // Notice that this is a bit tricky, as the execution chain maybe 'complete src -> complete dst
+ // -> cancel src', for now it seems to be fine, as the will use CAS to set the result first in
+ // CompletableFuture. If later this causes problems, we could use whenCompleteAsync to break the
+ // tie.
+ addListener(dstFuture, (r, e) -> srcFuture.cancel(false));
+ }
+
+ private void timelineConsistentGet(Get get, RegionLocations locs,
+ CompletableFuture<Result> future) {
+ if (future.isDone()) {
+ // do not send requests to secondary replicas if the future is done, i.e, the primary request
+ // has already been finished.
+ return;
+ }
+ for (int replicaId = 1, n = locs.size(); replicaId < n; replicaId++) {
+ CompletableFuture<Result> secondaryFuture = get(get, replicaId, readRpcTimeoutNs);
+ connect(secondaryFuture, future);
+ }
+ }
+
@Override
public CompletableFuture<Result> get(Get get) {
- return this.<Result> newCaller(get, readRpcTimeoutNs)
- .action((controller, loc, stub) -> RawAsyncTableImpl
- .<Get, GetRequest, GetResponse, Result> call(controller, loc, stub, get,
- RequestConverter::buildGetRequest, (s, c, req, done) -> s.get(c, req, done),
- (c, resp) -> ProtobufUtil.toResult(resp.getResult(), c.cellScanner())))
- .call();
+ CompletableFuture<Result> primaryFuture =
+ get(get, RegionReplicaUtil.DEFAULT_REPLICA_ID, readRpcTimeoutNs);
+ if (get.getConsistency() == Consistency.STRONG) {
+ return primaryFuture;
+ }
+ // Timeline consistent read, where we will send requests to other region replicas
+ CompletableFuture<Result> future = new CompletableFuture<>();
+ connect(primaryFuture, future);
+ long primaryCallTimeoutNs = conn.connConf.getPrimaryCallTimeoutNs();
+ long startNs = System.nanoTime();
+ addListener(conn.getLocator().getRegionLocations(tableName, get.getRow(),
+ RegionLocateType.CURRENT, false, readRpcTimeoutNs), (locs, error) -> {
+ if (error != null) {
+ LOG.warn(
+ "Failed to locate all the replicas for table={}, row='{}'," +
+ " give up timeline consistent read",
+ tableName, Bytes.toStringBinary(get.getRow()), error);
+ return;
+ }
+ if (locs.size() <= 1) {
+ LOG.warn(
+ "There are no secondary replicas for region {}," + " give up timeline consistent read",
+ locs.getDefaultRegionLocation().getRegion());
+ return;
+ }
+ long delayNs = primaryCallTimeoutNs - (System.nanoTime() - startNs);
+ if (delayNs <= 0) {
+ timelineConsistentGet(get, locs, future);
+ } else {
+ AsyncConnectionImpl.RETRY_TIMER.newTimeout(
+ timeout -> timelineConsistentGet(get, locs, future), delayNs, TimeUnit.NANOSECONDS);
+ }
+ });
+ return future;
}
@Override
public CompletableFuture<Void> put(Put put) {
return this.<Void> newCaller(put, writeRpcTimeoutNs)
- .action((controller, loc, stub) -> RawAsyncTableImpl.<Put> voidMutate(controller, loc, stub,
- put, RequestConverter::buildMutateRequest))
- .call();
+ .action((controller, loc, stub) -> RawAsyncTableImpl.<Put> voidMutate(controller, loc, stub,
+ put, RequestConverter::buildMutateRequest))
+ .call();
}
@Override
public CompletableFuture<Void> delete(Delete delete) {
return this.<Void> newCaller(delete, writeRpcTimeoutNs)
- .action((controller, loc, stub) -> RawAsyncTableImpl.<Delete> voidMutate(controller, loc,
- stub, delete, RequestConverter::buildMutateRequest))
- .call();
+ .action((controller, loc, stub) -> RawAsyncTableImpl.<Delete> voidMutate(controller, loc,
+ stub, delete, RequestConverter::buildMutateRequest))
+ .call();
}
@Override
public CompletableFuture<Result> append(Append append) {
checkHasFamilies(append);
return this.<Result> newCaller(append, rpcTimeoutNs)
- .action((controller, loc, stub) -> this.<Append, Result> noncedMutate(controller, loc, stub,
- append, RequestConverter::buildMutateRequest, RawAsyncTableImpl::toResult))
- .call();
+ .action((controller, loc, stub) -> this.<Append, Result> noncedMutate(controller, loc, stub,
+ append, RequestConverter::buildMutateRequest, RawAsyncTableImpl::toResult))
+ .call();
}
@Override
public CompletableFuture<Result> increment(Increment increment) {
checkHasFamilies(increment);
return this.<Result> newCaller(increment, rpcTimeoutNs)
- .action((controller, loc, stub) -> this.<Increment, Result> noncedMutate(controller, loc,
- stub, increment, RequestConverter::buildMutateRequest, RawAsyncTableImpl::toResult))
- .call();
+ .action((controller, loc, stub) -> this.<Increment, Result> noncedMutate(controller, loc,
+ stub, increment, RequestConverter::buildMutateRequest, RawAsyncTableImpl::toResult))
+ .call();
}
private final class CheckAndMutateBuilderImpl implements CheckAndMutateBuilder {
@@ -313,36 +386,36 @@ class RawAsyncTableImpl implements AsyncTable<AdvancedScanResultConsumer> {
public CompletableFuture<Boolean> thenPut(Put put) {
preCheck();
return RawAsyncTableImpl.this.<Boolean> newCaller(row, rpcTimeoutNs)
- .action((controller, loc, stub) -> RawAsyncTableImpl.<Put, Boolean> mutate(controller,
- loc, stub, put,
- (rn, p) -> RequestConverter.buildMutateRequest(rn, row, family, qualifier,
- new BinaryComparator(value), CompareType.valueOf(op.name()), timeRange, p),
- (c, r) -> r.getProcessed()))
- .call();
+ .action((controller, loc, stub) -> RawAsyncTableImpl.<Put, Boolean> mutate(controller, loc,
+ stub, put,
+ (rn, p) -> RequestConverter.buildMutateRequest(rn, row, family, qualifier,
+ new BinaryComparator(value), CompareType.valueOf(op.name()), timeRange, p),
+ (c, r) -> r.getProcessed()))
+ .call();
}
@Override
public CompletableFuture<Boolean> thenDelete(Delete delete) {
preCheck();
return RawAsyncTableImpl.this.<Boolean> newCaller(row, rpcTimeoutNs)
- .action((controller, loc, stub) -> RawAsyncTableImpl.<Delete, Boolean> mutate(controller,
- loc, stub, delete,
- (rn, d) -> RequestConverter.buildMutateRequest(rn, row, family, qualifier,
- new BinaryComparator(value), CompareType.valueOf(op.name()), timeRange, d),
- (c, r) -> r.getProcessed()))
- .call();
+ .action((controller, loc, stub) -> RawAsyncTableImpl.<Delete, Boolean> mutate(controller,
+ loc, stub, delete,
+ (rn, d) -> RequestConverter.buildMutateRequest(rn, row, family, qualifier,
+ new BinaryComparator(value), CompareType.valueOf(op.name()), timeRange, d),
+ (c, r) -> r.getProcessed()))
+ .call();
}
@Override
public CompletableFuture<Boolean> thenMutate(RowMutations mutation) {
preCheck();
return RawAsyncTableImpl.this.<Boolean> newCaller(mutation, rpcTimeoutNs)
- .action((controller, loc, stub) -> RawAsyncTableImpl.<Boolean> mutateRow(controller, loc,
- stub, mutation,
- (rn, rm) -> RequestConverter.buildMutateRequest(rn, row, family, qualifier,
- new BinaryComparator(value), CompareType.valueOf(op.name()), timeRange, rm),
- resp -> resp.getExists()))
- .call();
+ .action((controller, loc, stub) -> RawAsyncTableImpl.<Boolean> mutateRow(controller, loc,
+ stub, mutation,
+ (rn, rm) -> RequestConverter.buildMutateRequest(rn, row, family, qualifier,
+ new BinaryComparator(value), CompareType.valueOf(op.name()), timeRange, rm),
+ resp -> resp.getExists()))
+ .call();
}
}
@@ -375,10 +448,10 @@ class RawAsyncTableImpl implements AsyncTable<AdvancedScanResultConsumer> {
if (ex != null) {
future.completeExceptionally(ex instanceof IOException ? ex
: new IOException(
- "Failed to mutate row: " + Bytes.toStringBinary(mutation.getRow()), ex));
+ "Failed to mutate row: " + Bytes.toStringBinary(mutation.getRow()), ex));
} else {
future.complete(respConverter
- .apply((Result) multiResp.getResults().get(regionName).result.get(0)));
+ .apply((Result) multiResp.getResults().get(regionName).result.get(0)));
}
} catch (IOException e) {
future.completeExceptionally(e);
@@ -399,7 +472,8 @@ class RawAsyncTableImpl implements AsyncTable<AdvancedScanResultConsumer> {
RegionAction.Builder regionMutationBuilder = RequestConverter.buildRegionAction(rn, rm);
regionMutationBuilder.setAtomic(true);
return MultiRequest.newBuilder().addRegionAction(regionMutationBuilder.build()).build();
- }, resp -> null)).call();
+ }, resp -> null))
+ .call();
}
private Scan setDefaultScanConfig(Scan scan) {
@@ -416,7 +490,7 @@ class RawAsyncTableImpl implements AsyncTable<AdvancedScanResultConsumer> {
public void scan(Scan scan, AdvancedScanResultConsumer consumer) {
new AsyncClientScanner(setDefaultScanConfig(scan), consumer, tableName, conn, pauseNs,
- maxAttempts, scanTimeoutNs, readRpcTimeoutNs, startLogErrorsCnt).start();
+ maxAttempts, scanTimeoutNs, readRpcTimeoutNs, startLogErrorsCnt).start();
}
private long resultSize2CacheSize(long maxResultSize) {
@@ -427,8 +501,8 @@ class RawAsyncTableImpl implements AsyncTable<AdvancedScanResultConsumer> {
@Override
public ResultScanner getScanner(Scan scan) {
return new AsyncTableResultScanner(this, ReflectionUtils.newInstance(scan.getClass(), scan),
- resultSize2CacheSize(
- scan.getMaxResultSize() > 0 ? scan.getMaxResultSize() : defaultScannerMaxResultSize));
+ resultSize2CacheSize(
+ scan.getMaxResultSize() > 0 ? scan.getMaxResultSize() : defaultScannerMaxResultSize));
}
@Override
@@ -477,14 +551,14 @@ class RawAsyncTableImpl implements AsyncTable<AdvancedScanResultConsumer> {
private List<CompletableFuture<Void>> voidMutate(List<? extends Row> actions) {
return this.<Object> batch(actions, writeRpcTimeoutNs).stream()
- .map(f -> f.<Void> thenApply(r -> null)).collect(toList());
+ .map(f -> f.<Void> thenApply(r -> null)).collect(toList());
}
private <T> List<CompletableFuture<T>> batch(List<? extends Row> actions, long rpcTimeoutNs) {
return conn.callerFactory.batch().table(tableName).actions(actions)
- .operationTimeout(operationTimeoutNs, TimeUnit.NANOSECONDS)
- .rpcTimeout(rpcTimeoutNs, TimeUnit.NANOSECONDS).pause(pauseNs, TimeUnit.NANOSECONDS)
- .maxAttempts(maxAttempts).startLogErrorsCnt(startLogErrorsCnt).call();
+ .operationTimeout(operationTimeoutNs, TimeUnit.NANOSECONDS)
+ .rpcTimeout(rpcTimeoutNs, TimeUnit.NANOSECONDS).pause(pauseNs, TimeUnit.NANOSECONDS)
+ .maxAttempts(maxAttempts).startLogErrorsCnt(startLogErrorsCnt).call();
}
@Override
@@ -515,7 +589,7 @@ class RawAsyncTableImpl implements AsyncTable<AdvancedScanResultConsumer> {
private <S, R> CompletableFuture<R> coprocessorService(Function<RpcChannel, S> stubMaker,
ServiceCaller<S, R> callable, RegionInfo region, byte[] row) {
RegionCoprocessorRpcChannelImpl channel = new RegionCoprocessorRpcChannelImpl(conn, tableName,
- region, row, rpcTimeoutNs, operationTimeoutNs);
+ region, row, rpcTimeoutNs, operationTimeoutNs);
S stub = stubMaker.apply(channel);
CompletableFuture<R> future = new CompletableFuture<>();
ClientCoprocessorRpcController controller = new ClientCoprocessorRpcController();
@@ -553,10 +627,9 @@ class RawAsyncTableImpl implements AsyncTable<AdvancedScanResultConsumer> {
}
private <S, R> void onLocateComplete(Function<RpcChannel, S> stubMaker,
- ServiceCaller<S, R> callable, CoprocessorCallback<R> callback,
- List<HRegionLocation> locs, byte[] endKey, boolean endKeyInclusive,
- AtomicBoolean locateFinished, AtomicInteger unfinishedRequest, HRegionLocation loc,
- Throwable error) {
+ ServiceCaller<S, R> callable, CoprocessorCallback<R> callback, List<HRegionLocation> locs,
+ byte[] endKey, boolean endKeyInclusive, AtomicBoolean locateFinished,
+ AtomicInteger unfinishedRequest, HRegionLocation loc, Throwable error) {
if (error != null) {
callback.onError(error);
return;
@@ -566,11 +639,11 @@ class RawAsyncTableImpl implements AsyncTable<AdvancedScanResultConsumer> {
if (locateFinished(region, endKey, endKeyInclusive)) {
locateFinished.set(true);
} else {
- conn.getLocator()
- .getRegionLocation(tableName, region.getEndKey(), RegionLocateType.CURRENT,
- operationTimeoutNs)
- .whenComplete((l, e) -> onLocateComplete(stubMaker, callable, callback, locs, endKey,
- endKeyInclusive, locateFinished, unfinishedRequest, l, e));
+ addListener(
+ conn.getLocator().getRegionLocation(tableName, region.getEndKey(), RegionLocateType.CURRENT,
+ operationTimeoutNs),
+ (l, e) -> onLocateComplete(stubMaker, callable, callback, locs, endKey, endKeyInclusive,
+ locateFinished, unfinishedRequest, l, e));
}
coprocessorService(stubMaker, callable, region, region.getStartKey()).whenComplete((r, e) -> {
if (e != null) {
@@ -630,11 +703,10 @@ class RawAsyncTableImpl implements AsyncTable<AdvancedScanResultConsumer> {
@Override
public void execute() {
- conn.getLocator().getRegionLocation(tableName, startKey,
- startKeyInclusive ? RegionLocateType.CURRENT : RegionLocateType.AFTER, operationTimeoutNs)
- .whenComplete(
- (loc, error) -> onLocateComplete(stubMaker, callable, callback, new ArrayList<>(),
- endKey, endKeyInclusive, new AtomicBoolean(false), new AtomicInteger(0), loc, error));
+ addListener(conn.getLocator().getRegionLocation(tableName, startKey,
+ startKeyInclusive ? RegionLocateType.CURRENT : RegionLocateType.AFTER, operationTimeoutNs),
+ (loc, error) -> onLocateComplete(stubMaker, callable, callback, new ArrayList<>(), endKey,
+ endKeyInclusive, new AtomicBoolean(false), new AtomicInteger(0), loc, error));
}
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/db66e6cc/hbase-common/src/main/java/org/apache/hadoop/hbase/util/FutureUtils.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/FutureUtils.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/FutureUtils.java
new file mode 100644
index 0000000..067e66b
--- /dev/null
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/FutureUtils.java
@@ -0,0 +1,60 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.util;
+
+import java.util.concurrent.CompletableFuture;
+import java.util.function.BiConsumer;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Helper class for processing futures.
+ */
+@InterfaceAudience.Private
+public final class FutureUtils {
+
+ private static final Logger LOG = LoggerFactory.getLogger(FutureUtils.class);
+
+ private FutureUtils() {
+ }
+
+ /**
+ * This is method is used when you just want to add a listener to the given future. We will call
+ * {@link CompletableFuture#whenComplete(BiConsumer)} to register the {@code action} to the
+ * {@code future}. Ignoring the return value of a Future is considered as a bad practice as it may
+ * suppress exceptions thrown from the code that completes the future, and this method will catch
+ * all the exception thrown from the {@code action} to catch possible code bugs.
+ * <p/>
+ * And the error phone check will always report FutureReturnValueIgnored because every method in
+ * the {@link CompletableFuture} class will return a new {@link CompletableFuture}, so you always
+ * have one future that has not been checked. So we introduce this method and add a suppress
+ * warnings annotation here.
+ */
+ @SuppressWarnings("FutureReturnValueIgnored")
+ public static <T> void addListener(CompletableFuture<T> future,
+ BiConsumer<? super T, ? super Throwable> action) {
+ future.whenComplete((resp, error) -> {
+ try {
+ action.accept(resp, error);
+ } catch (Throwable t) {
+ LOG.error("Unexpected error caught when processing CompletableFuture", t);
+ }
+ });
+ }
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/hbase/blob/db66e6cc/hbase-server/src/test/java/org/apache/hadoop/hbase/client/RegionReplicaTestHelper.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/RegionReplicaTestHelper.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/RegionReplicaTestHelper.java
new file mode 100644
index 0000000..c14f69f
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/RegionReplicaTestHelper.java
@@ -0,0 +1,161 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.client;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+
+import java.io.IOException;
+import java.util.Optional;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.NotServingRegionException;
+import org.apache.hadoop.hbase.RegionLocations;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.Waiter.ExplainingPredicate;
+import org.apache.hadoop.hbase.util.Bytes;
+
+final class RegionReplicaTestHelper {
+
+ private RegionReplicaTestHelper() {
+ }
+
+ // waits for all replicas to have region location
+ static void waitUntilAllMetaReplicasHavingRegionLocation(AsyncRegistry registry,
+ int regionReplication) throws IOException {
+ TestZKAsyncRegistry.TEST_UTIL.waitFor(
+ TestZKAsyncRegistry.TEST_UTIL.getConfiguration()
+ .getLong("hbase.client.sync.wait.timeout.msec", 60000),
+ 200, true, new ExplainingPredicate<IOException>() {
+ @Override
+ public String explainFailure() throws IOException {
+ return "Not all meta replicas get assigned";
+ }
+
+ @Override
+ public boolean evaluate() throws IOException {
+ try {
+ RegionLocations locs = registry.getMetaRegionLocation().get();
+ if (locs.size() < regionReplication) {
+ return false;
+ }
+ for (int i = 0; i < regionReplication; i++) {
+ if (locs.getRegionLocation(i) == null) {
+ return false;
+ }
+ }
+ return true;
+ } catch (Exception e) {
+ TestZKAsyncRegistry.LOG.warn("Failed to get meta region locations", e);
+ return false;
+ }
+ }
+ });
+ }
+
+ static Optional<ServerName> getRSCarryingReplica(HBaseTestingUtility util, TableName tableName,
+ int replicaId) {
+ return util.getHBaseCluster().getRegionServerThreads().stream().map(t -> t.getRegionServer())
+ .filter(rs -> rs.getRegions(tableName).stream()
+ .anyMatch(r -> r.getRegionInfo().getReplicaId() == replicaId))
+ .findAny().map(rs -> rs.getServerName());
+ }
+
+ /**
+ * Return the new location.
+ */
+ static ServerName moveRegion(HBaseTestingUtility util, HRegionLocation currentLoc)
+ throws Exception {
+ ServerName serverName = currentLoc.getServerName();
+ RegionInfo regionInfo = currentLoc.getRegion();
+ TableName tableName = regionInfo.getTable();
+ int replicaId = regionInfo.getReplicaId();
+ ServerName newServerName = util.getHBaseCluster().getRegionServerThreads().stream()
+ .map(t -> t.getRegionServer().getServerName()).filter(sn -> !sn.equals(serverName)).findAny()
+ .get();
+ util.getAdmin().move(regionInfo.getEncodedNameAsBytes(),
+ Bytes.toBytes(newServerName.getServerName()));
+ util.waitFor(30000, new ExplainingPredicate<Exception>() {
+
+ @Override
+ public boolean evaluate() throws Exception {
+ Optional<ServerName> newServerName = getRSCarryingReplica(util, tableName, replicaId);
+ return newServerName.isPresent() && !newServerName.get().equals(serverName);
+ }
+
+ @Override
+ public String explainFailure() throws Exception {
+ return regionInfo.getRegionNameAsString() + " is still on " + serverName;
+ }
+ });
+ return newServerName;
+ }
+
+ interface Locator {
+ RegionLocations getRegionLocations(TableName tableName, int replicaId, boolean reload)
+ throws Exception;
+
+ void updateCachedLocationOnError(HRegionLocation loc, Throwable error) throws Exception;
+ }
+
+ static void testLocator(HBaseTestingUtility util, TableName tableName, Locator locator)
+ throws Exception {
+ RegionLocations locs =
+ locator.getRegionLocations(tableName, RegionReplicaUtil.DEFAULT_REPLICA_ID, false);
+ assertEquals(3, locs.size());
+ for (int i = 0; i < 3; i++) {
+ HRegionLocation loc = locs.getRegionLocation(i);
+ assertNotNull(loc);
+ ServerName serverName = getRSCarryingReplica(util, tableName, i).get();
+ assertEquals(serverName, loc.getServerName());
+ }
+ ServerName newServerName = moveRegion(util, locs.getDefaultRegionLocation());
+ // The cached location should not be changed
+ assertEquals(locs.getDefaultRegionLocation().getServerName(),
+ locator.getRegionLocations(tableName, RegionReplicaUtil.DEFAULT_REPLICA_ID, false)
+ .getDefaultRegionLocation().getServerName());
+ // should get the new location when reload = true
+ assertEquals(newServerName,
+ locator.getRegionLocations(tableName, RegionReplicaUtil.DEFAULT_REPLICA_ID, true)
+ .getDefaultRegionLocation().getServerName());
+ // the cached location should be replaced
+ assertEquals(newServerName,
+ locator.getRegionLocations(tableName, RegionReplicaUtil.DEFAULT_REPLICA_ID, false)
+ .getDefaultRegionLocation().getServerName());
+
+ ServerName newServerName1 = moveRegion(util, locs.getRegionLocation(1));
+ ServerName newServerName2 = moveRegion(util, locs.getRegionLocation(2));
+
+ // The cached location should not be change
+ assertEquals(locs.getRegionLocation(1).getServerName(),
+ locator.getRegionLocations(tableName, 1, false).getRegionLocation(1).getServerName());
+ // clear the cached location for replica 1
+ locator.updateCachedLocationOnError(locs.getRegionLocation(1), new NotServingRegionException());
+ // the cached location for replica 2 should not be changed
+ assertEquals(locs.getRegionLocation(2).getServerName(),
+ locator.getRegionLocations(tableName, 2, false).getRegionLocation(2).getServerName());
+ // should get the new location as we have cleared the old location
+ assertEquals(newServerName1,
+ locator.getRegionLocations(tableName, 1, false).getRegionLocation(1).getServerName());
+ // as we will get the new location for replica 2 at once, we should also get the new location
+ // for replica 2
+ assertEquals(newServerName2,
+ locator.getRegionLocations(tableName, 2, false).getRegionLocation(2).getServerName());
+ }
+}
http://git-wip-us.apache.org/repos/asf/hbase/blob/db66e6cc/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncMetaRegionLocator.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncMetaRegionLocator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncMetaRegionLocator.java
index 7c08d6d..df1fe08 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncMetaRegionLocator.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncMetaRegionLocator.java
@@ -17,20 +17,19 @@
*/
package org.apache.hadoop.hbase.client;
-import static org.junit.Assert.assertEquals;
+import static org.apache.hadoop.hbase.client.RegionReplicaTestHelper.testLocator;
-import java.util.Optional;
import org.apache.commons.io.IOUtils;
import org.apache.hadoop.hbase.HBaseClassTestRule;
import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.RegionLocations;
import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.Waiter.ExplainingPredicate;
+import org.apache.hadoop.hbase.client.RegionReplicaTestHelper.Locator;
import org.apache.hadoop.hbase.master.balancer.BaseLoadBalancer;
import org.apache.hadoop.hbase.testclassification.ClientTests;
import org.apache.hadoop.hbase.testclassification.MediumTests;
-import org.apache.hadoop.hbase.util.Bytes;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.ClassRule;
@@ -42,7 +41,7 @@ public class TestAsyncMetaRegionLocator {
@ClassRule
public static final HBaseClassTestRule CLASS_RULE =
- HBaseClassTestRule.forClass(TestAsyncMetaRegionLocator.class);
+ HBaseClassTestRule.forClass(TestAsyncMetaRegionLocator.class);
private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
@@ -53,10 +52,11 @@ public class TestAsyncMetaRegionLocator {
@BeforeClass
public static void setUp() throws Exception {
TEST_UTIL.getConfiguration().set(BaseLoadBalancer.TABLES_ON_MASTER, "none");
+ TEST_UTIL.getConfiguration().setInt(HConstants.META_REPLICAS_NUM, 3);
TEST_UTIL.startMiniCluster(3);
- TEST_UTIL.waitUntilAllSystemRegionsAssigned();
- TEST_UTIL.getAdmin().setBalancerRunning(false, true);
REGISTRY = AsyncRegistryFactory.getRegistry(TEST_UTIL.getConfiguration());
+ RegionReplicaTestHelper.waitUntilAllMetaReplicasHavingRegionLocation(REGISTRY, 3);
+ TEST_UTIL.getAdmin().balancerSwitch(false, true);
LOCATOR = new AsyncMetaRegionLocator(REGISTRY);
}
@@ -66,42 +66,21 @@ public class TestAsyncMetaRegionLocator {
TEST_UTIL.shutdownMiniCluster();
}
- private Optional<ServerName> getRSCarryingMeta() {
- return TEST_UTIL.getHBaseCluster().getRegionServerThreads().stream()
- .map(t -> t.getRegionServer())
- .filter(rs -> !rs.getRegions(TableName.META_TABLE_NAME).isEmpty()).findAny()
- .map(rs -> rs.getServerName());
- }
-
@Test
- public void testReload() throws Exception {
- ServerName serverName = getRSCarryingMeta().get();
- assertEquals(serverName, LOCATOR.getRegionLocation(false).get().getServerName());
-
- ServerName newServerName = TEST_UTIL.getHBaseCluster().getRegionServerThreads().stream()
- .map(t -> t.getRegionServer().getServerName()).filter(sn -> !sn.equals(serverName))
- .findAny().get();
- TEST_UTIL.getAdmin().move(HRegionInfo.FIRST_META_REGIONINFO.getEncodedNameAsBytes(),
- Bytes.toBytes(newServerName.getServerName()));
- TEST_UTIL.waitFor(30000, new ExplainingPredicate<Exception>() {
+ public void test() throws Exception {
+ testLocator(TEST_UTIL, TableName.META_TABLE_NAME, new Locator() {
@Override
- public boolean evaluate() throws Exception {
- Optional<ServerName> newServerName = getRSCarryingMeta();
- return newServerName.isPresent() && !newServerName.get().equals(serverName);
+ public void updateCachedLocationOnError(HRegionLocation loc, Throwable error)
+ throws Exception {
+ LOCATOR.updateCachedLocationOnError(loc, error);
}
@Override
- public String explainFailure() throws Exception {
- return HRegionInfo.FIRST_META_REGIONINFO.getRegionNameAsString() + " is still on " +
- serverName;
+ public RegionLocations getRegionLocations(TableName tableName, int replicaId, boolean reload)
+ throws Exception {
+ return LOCATOR.getRegionLocations(replicaId, reload).get();
}
});
- // The cached location will not change
- assertEquals(serverName, LOCATOR.getRegionLocation(false).get().getServerName());
- // should get the new location when reload = true
- assertEquals(newServerName, LOCATOR.getRegionLocation(true).get().getServerName());
- // the cached location should be replaced
- assertEquals(newServerName, LOCATOR.getRegionLocation(false).get().getServerName());
}
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/db66e6cc/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocator.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocator.java
index 38dc78d..eeaf99f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocator.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocator.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.client;
import static java.util.stream.Collectors.toList;
import static org.apache.hadoop.hbase.HConstants.EMPTY_END_ROW;
import static org.apache.hadoop.hbase.HConstants.EMPTY_START_ROW;
+import static org.apache.hadoop.hbase.client.RegionReplicaTestHelper.testLocator;
import static org.hamcrest.CoreMatchers.instanceOf;
import static org.junit.Assert.assertArrayEquals;
import static org.junit.Assert.assertEquals;
@@ -38,10 +39,12 @@ import org.apache.hadoop.hbase.HBaseClassTestRule;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.NotServingRegionException;
+import org.apache.hadoop.hbase.RegionLocations;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableNotFoundException;
import org.apache.hadoop.hbase.Waiter.ExplainingPredicate;
+import org.apache.hadoop.hbase.client.RegionReplicaTestHelper.Locator;
import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.testclassification.ClientTests;
import org.apache.hadoop.hbase.testclassification.MediumTests;
@@ -58,7 +61,7 @@ public class TestAsyncNonMetaRegionLocator {
@ClassRule
public static final HBaseClassTestRule CLASS_RULE =
- HBaseClassTestRule.forClass(TestAsyncNonMetaRegionLocator.class);
+ HBaseClassTestRule.forClass(TestAsyncNonMetaRegionLocator.class);
private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
@@ -78,7 +81,7 @@ public class TestAsyncNonMetaRegionLocator {
TEST_UTIL.getAdmin().balancerSwitch(false, true);
AsyncRegistry registry = AsyncRegistryFactory.getRegistry(TEST_UTIL.getConfiguration());
CONN = new AsyncConnectionImpl(TEST_UTIL.getConfiguration(), registry,
- registry.getClusterId().get(), User.getCurrent());
+ registry.getClusterId().get(), User.getCurrent());
LOCATOR = new AsyncNonMetaRegionLocator(CONN);
SPLIT_KEYS = new byte[8][];
for (int i = 111; i < 999; i += 111) {
@@ -109,11 +112,18 @@ public class TestAsyncNonMetaRegionLocator {
TEST_UTIL.waitTableAvailable(TABLE_NAME);
}
+ private CompletableFuture<HRegionLocation> getDefaultRegionLocation(TableName tableName,
+ byte[] row, RegionLocateType locateType, boolean reload) {
+ return LOCATOR
+ .getRegionLocations(tableName, row, RegionReplicaUtil.DEFAULT_REPLICA_ID, locateType, reload)
+ .thenApply(RegionLocations::getDefaultRegionLocation);
+ }
+
@Test
public void testNoTable() throws InterruptedException {
for (RegionLocateType locateType : RegionLocateType.values()) {
try {
- LOCATOR.getRegionLocation(TABLE_NAME, EMPTY_START_ROW, locateType, false).get();
+ getDefaultRegionLocation(TABLE_NAME, EMPTY_START_ROW, locateType, false).get();
} catch (ExecutionException e) {
assertThat(e.getCause(), instanceOf(TableNotFoundException.class));
}
@@ -126,7 +136,7 @@ public class TestAsyncNonMetaRegionLocator {
TEST_UTIL.getAdmin().disableTable(TABLE_NAME);
for (RegionLocateType locateType : RegionLocateType.values()) {
try {
- LOCATOR.getRegionLocation(TABLE_NAME, EMPTY_START_ROW, locateType, false).get();
+ getDefaultRegionLocation(TABLE_NAME, EMPTY_START_ROW, locateType, false).get();
} catch (ExecutionException e) {
assertThat(e.getCause(), instanceOf(TableNotFoundException.class));
}
@@ -148,13 +158,13 @@ public class TestAsyncNonMetaRegionLocator {
ServerName serverName = TEST_UTIL.getRSForFirstRegionInTable(TABLE_NAME).getServerName();
for (RegionLocateType locateType : RegionLocateType.values()) {
assertLocEquals(EMPTY_START_ROW, EMPTY_END_ROW, serverName,
- LOCATOR.getRegionLocation(TABLE_NAME, EMPTY_START_ROW, locateType, false).get());
+ getDefaultRegionLocation(TABLE_NAME, EMPTY_START_ROW, locateType, false).get());
}
byte[] randKey = new byte[ThreadLocalRandom.current().nextInt(128)];
ThreadLocalRandom.current().nextBytes(randKey);
for (RegionLocateType locateType : RegionLocateType.values()) {
assertLocEquals(EMPTY_START_ROW, EMPTY_END_ROW, serverName,
- LOCATOR.getRegionLocation(TABLE_NAME, randKey, locateType, false).get());
+ getDefaultRegionLocation(TABLE_NAME, randKey, locateType, false).get());
}
}
@@ -179,12 +189,12 @@ public class TestAsyncNonMetaRegionLocator {
private ServerName[] getLocations(byte[][] startKeys) {
ServerName[] serverNames = new ServerName[startKeys.length];
TEST_UTIL.getHBaseCluster().getRegionServerThreads().stream().map(t -> t.getRegionServer())
- .forEach(rs -> {
- rs.getRegions(TABLE_NAME).forEach(r -> {
- serverNames[Arrays.binarySearch(startKeys, r.getRegionInfo().getStartKey(),
- Bytes::compareTo)] = rs.getServerName();
- });
+ .forEach(rs -> {
+ rs.getRegions(TABLE_NAME).forEach(r -> {
+ serverNames[Arrays.binarySearch(startKeys, r.getRegionInfo().getStartKey(),
+ Bytes::compareTo)] = rs.getServerName();
});
+ });
return serverNames;
}
@@ -196,8 +206,9 @@ public class TestAsyncNonMetaRegionLocator {
IntStream.range(0, 2).forEach(n -> IntStream.range(0, startKeys.length).forEach(i -> {
try {
assertLocEquals(startKeys[i], i == startKeys.length - 1 ? EMPTY_END_ROW : startKeys[i + 1],
- serverNames[i], LOCATOR
- .getRegionLocation(TABLE_NAME, startKeys[i], RegionLocateType.CURRENT, false).get());
+ serverNames[i],
+ getDefaultRegionLocation(TABLE_NAME, startKeys[i], RegionLocateType.CURRENT, false)
+ .get());
} catch (InterruptedException | ExecutionException e) {
throw new RuntimeException(e);
}
@@ -208,7 +219,7 @@ public class TestAsyncNonMetaRegionLocator {
try {
assertLocEquals(startKeys[i], i == startKeys.length - 1 ? EMPTY_END_ROW : startKeys[i + 1],
serverNames[i],
- LOCATOR.getRegionLocation(TABLE_NAME, startKeys[i], RegionLocateType.AFTER, false).get());
+ getDefaultRegionLocation(TABLE_NAME, startKeys[i], RegionLocateType.AFTER, false).get());
} catch (InterruptedException | ExecutionException e) {
throw new RuntimeException(e);
}
@@ -220,8 +231,7 @@ public class TestAsyncNonMetaRegionLocator {
n -> IntStream.range(0, endKeys.length).map(i -> endKeys.length - 1 - i).forEach(i -> {
try {
assertLocEquals(i == 0 ? EMPTY_START_ROW : endKeys[i - 1], endKeys[i], serverNames[i],
- LOCATOR.getRegionLocation(TABLE_NAME, endKeys[i], RegionLocateType.BEFORE, false)
- .get());
+ getDefaultRegionLocation(TABLE_NAME, endKeys[i], RegionLocateType.BEFORE, false).get());
} catch (InterruptedException | ExecutionException e) {
throw new RuntimeException(e);
}
@@ -232,29 +242,29 @@ public class TestAsyncNonMetaRegionLocator {
public void testRegionMove() throws IOException, InterruptedException, ExecutionException {
createSingleRegionTable();
ServerName serverName = TEST_UTIL.getRSForFirstRegionInTable(TABLE_NAME).getServerName();
- HRegionLocation loc = LOCATOR
- .getRegionLocation(TABLE_NAME, EMPTY_START_ROW, RegionLocateType.CURRENT, false).get();
+ HRegionLocation loc =
+ getDefaultRegionLocation(TABLE_NAME, EMPTY_START_ROW, RegionLocateType.CURRENT, false).get();
assertLocEquals(EMPTY_START_ROW, EMPTY_END_ROW, serverName, loc);
ServerName newServerName = TEST_UTIL.getHBaseCluster().getRegionServerThreads().stream()
- .map(t -> t.getRegionServer().getServerName()).filter(sn -> !sn.equals(serverName))
- .findAny().get();
+ .map(t -> t.getRegionServer().getServerName()).filter(sn -> !sn.equals(serverName)).findAny()
+ .get();
TEST_UTIL.getAdmin().move(Bytes.toBytes(loc.getRegion().getEncodedName()),
Bytes.toBytes(newServerName.getServerName()));
while (!TEST_UTIL.getRSForFirstRegionInTable(TABLE_NAME).getServerName()
- .equals(newServerName)) {
+ .equals(newServerName)) {
Thread.sleep(100);
}
// Should be same as it is in cache
- assertSame(loc, LOCATOR
- .getRegionLocation(TABLE_NAME, EMPTY_START_ROW, RegionLocateType.CURRENT, false).get());
- LOCATOR.updateCachedLocation(loc, null);
+ assertSame(loc,
+ getDefaultRegionLocation(TABLE_NAME, EMPTY_START_ROW, RegionLocateType.CURRENT, false).get());
+ LOCATOR.updateCachedLocationOnError(loc, null);
// null error will not trigger a cache cleanup
- assertSame(loc, LOCATOR
- .getRegionLocation(TABLE_NAME, EMPTY_START_ROW, RegionLocateType.CURRENT, false).get());
- LOCATOR.updateCachedLocation(loc, new NotServingRegionException());
- assertLocEquals(EMPTY_START_ROW, EMPTY_END_ROW, newServerName, LOCATOR
- .getRegionLocation(TABLE_NAME, EMPTY_START_ROW, RegionLocateType.CURRENT, false).get());
+ assertSame(loc,
+ getDefaultRegionLocation(TABLE_NAME, EMPTY_START_ROW, RegionLocateType.CURRENT, false).get());
+ LOCATOR.updateCachedLocationOnError(loc, new NotServingRegionException());
+ assertLocEquals(EMPTY_START_ROW, EMPTY_END_ROW, newServerName,
+ getDefaultRegionLocation(TABLE_NAME, EMPTY_START_ROW, RegionLocateType.CURRENT, false).get());
}
// usually locate after will return the same result, so we add a test to make it return different
@@ -266,21 +276,21 @@ public class TestAsyncNonMetaRegionLocator {
TEST_UTIL.createTable(TABLE_NAME, FAMILY, new byte[][] { splitKey });
TEST_UTIL.waitTableAvailable(TABLE_NAME);
HRegionLocation currentLoc =
- LOCATOR.getRegionLocation(TABLE_NAME, row, RegionLocateType.CURRENT, false).get();
+ getDefaultRegionLocation(TABLE_NAME, row, RegionLocateType.CURRENT, false).get();
ServerName currentServerName = TEST_UTIL.getRSForFirstRegionInTable(TABLE_NAME).getServerName();
assertLocEquals(EMPTY_START_ROW, splitKey, currentServerName, currentLoc);
HRegionLocation afterLoc =
- LOCATOR.getRegionLocation(TABLE_NAME, row, RegionLocateType.AFTER, false).get();
+ getDefaultRegionLocation(TABLE_NAME, row, RegionLocateType.AFTER, false).get();
ServerName afterServerName =
- TEST_UTIL.getHBaseCluster().getRegionServerThreads().stream().map(t -> t.getRegionServer())
- .filter(rs -> rs.getRegions(TABLE_NAME).stream()
- .anyMatch(r -> Bytes.equals(splitKey, r.getRegionInfo().getStartKey())))
- .findAny().get().getServerName();
+ TEST_UTIL.getHBaseCluster().getRegionServerThreads().stream().map(t -> t.getRegionServer())
+ .filter(rs -> rs.getRegions(TABLE_NAME).stream()
+ .anyMatch(r -> Bytes.equals(splitKey, r.getRegionInfo().getStartKey())))
+ .findAny().get().getServerName();
assertLocEquals(splitKey, EMPTY_END_ROW, afterServerName, afterLoc);
assertSame(afterLoc,
- LOCATOR.getRegionLocation(TABLE_NAME, row, RegionLocateType.AFTER, false).get());
+ getDefaultRegionLocation(TABLE_NAME, row, RegionLocateType.AFTER, false).get());
}
// For HBASE-17402
@@ -292,9 +302,9 @@ public class TestAsyncNonMetaRegionLocator {
ServerName[] serverNames = getLocations(startKeys);
for (int i = 0; i < 100; i++) {
LOCATOR.clearCache(TABLE_NAME);
- List<CompletableFuture<HRegionLocation>> futures = IntStream.range(0, 1000)
- .mapToObj(n -> String.format("%03d", n)).map(s -> Bytes.toBytes(s))
- .map(r -> LOCATOR.getRegionLocation(TABLE_NAME, r, RegionLocateType.CURRENT, false))
+ List<CompletableFuture<HRegionLocation>> futures =
+ IntStream.range(0, 1000).mapToObj(n -> String.format("%03d", n)).map(s -> Bytes.toBytes(s))
+ .map(r -> getDefaultRegionLocation(TABLE_NAME, r, RegionLocateType.CURRENT, false))
.collect(toList());
for (int j = 0; j < 1000; j++) {
int index = Math.min(8, j / 111);
@@ -309,11 +319,11 @@ public class TestAsyncNonMetaRegionLocator {
ServerName serverName = TEST_UTIL.getRSForFirstRegionInTable(TABLE_NAME).getServerName();
for (RegionLocateType locateType : RegionLocateType.values()) {
assertLocEquals(EMPTY_START_ROW, EMPTY_END_ROW, serverName,
- LOCATOR.getRegionLocation(TABLE_NAME, EMPTY_START_ROW, locateType, false).get());
+ getDefaultRegionLocation(TABLE_NAME, EMPTY_START_ROW, locateType, false).get());
}
ServerName newServerName = TEST_UTIL.getHBaseCluster().getRegionServerThreads().stream()
- .map(t -> t.getRegionServer().getServerName()).filter(sn -> !sn.equals(serverName))
- .findAny().get();
+ .map(t -> t.getRegionServer().getServerName()).filter(sn -> !sn.equals(serverName)).findAny()
+ .get();
Admin admin = TEST_UTIL.getAdmin();
RegionInfo region = admin.getRegions(TABLE_NAME).stream().findAny().get();
admin.move(region.getEncodedNameAsBytes(), Bytes.toBytes(newServerName.getServerName()));
@@ -334,15 +344,15 @@ public class TestAsyncNonMetaRegionLocator {
// The cached location will not change
for (RegionLocateType locateType : RegionLocateType.values()) {
assertLocEquals(EMPTY_START_ROW, EMPTY_END_ROW, serverName,
- LOCATOR.getRegionLocation(TABLE_NAME, EMPTY_START_ROW, locateType, false).get());
+ getDefaultRegionLocation(TABLE_NAME, EMPTY_START_ROW, locateType, false).get());
}
// should get the new location when reload = true
assertLocEquals(EMPTY_START_ROW, EMPTY_END_ROW, newServerName,
- LOCATOR.getRegionLocation(TABLE_NAME, EMPTY_START_ROW, RegionLocateType.CURRENT, true).get());
+ getDefaultRegionLocation(TABLE_NAME, EMPTY_START_ROW, RegionLocateType.CURRENT, true).get());
// the cached location should be replaced
for (RegionLocateType locateType : RegionLocateType.values()) {
assertLocEquals(EMPTY_START_ROW, EMPTY_END_ROW, newServerName,
- LOCATOR.getRegionLocation(TABLE_NAME, EMPTY_START_ROW, locateType, false).get());
+ getDefaultRegionLocation(TABLE_NAME, EMPTY_START_ROW, locateType, false).get());
}
}
@@ -351,10 +361,32 @@ public class TestAsyncNonMetaRegionLocator {
public void testLocateBeforeLastRegion()
throws IOException, InterruptedException, ExecutionException {
createMultiRegionTable();
- LOCATOR.getRegionLocation(TABLE_NAME, SPLIT_KEYS[0], RegionLocateType.CURRENT, false).join();
+ getDefaultRegionLocation(TABLE_NAME, SPLIT_KEYS[0], RegionLocateType.CURRENT, false).join();
HRegionLocation loc =
- LOCATOR.getRegionLocation(TABLE_NAME, EMPTY_END_ROW, RegionLocateType.BEFORE, false).get();
+ getDefaultRegionLocation(TABLE_NAME, EMPTY_END_ROW, RegionLocateType.BEFORE, false).get();
// should locate to the last region
assertArrayEquals(loc.getRegion().getEndKey(), EMPTY_END_ROW);
}
+
+ @Test
+ public void testRegionReplicas() throws Exception {
+ TEST_UTIL.getAdmin().createTable(TableDescriptorBuilder.newBuilder(TABLE_NAME)
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILY)).setRegionReplication(3).build());
+ TEST_UTIL.waitUntilAllRegionsAssigned(TABLE_NAME);
+ testLocator(TEST_UTIL, TABLE_NAME, new Locator() {
+
+ @Override
+ public void updateCachedLocationOnError(HRegionLocation loc, Throwable error)
+ throws Exception {
+ LOCATOR.updateCachedLocationOnError(loc, error);
+ }
+
+ @Override
+ public RegionLocations getRegionLocations(TableName tableName, int replicaId, boolean reload)
+ throws Exception {
+ return LOCATOR.getRegionLocations(tableName, EMPTY_START_ROW, replicaId,
+ RegionLocateType.CURRENT, reload).get();
+ }
+ });
+ }
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/db66e6cc/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocatorConcurrenyLimit.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocatorConcurrenyLimit.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocatorConcurrenyLimit.java
index c6624e7..8cdb4a9 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocatorConcurrenyLimit.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocatorConcurrenyLimit.java
@@ -37,6 +37,7 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseClassTestRule;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.RegionLocations;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.coprocessor.ObserverContext;
import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor;
@@ -59,7 +60,7 @@ public class TestAsyncNonMetaRegionLocatorConcurrenyLimit {
@ClassRule
public static final HBaseClassTestRule CLASS_RULE =
- HBaseClassTestRule.forClass(TestAsyncNonMetaRegionLocatorConcurrenyLimit.class);
+ HBaseClassTestRule.forClass(TestAsyncNonMetaRegionLocatorConcurrenyLimit.class);
private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
@@ -124,10 +125,10 @@ public class TestAsyncNonMetaRegionLocatorConcurrenyLimit {
TEST_UTIL.getAdmin().balancerSwitch(false, true);
AsyncRegistry registry = AsyncRegistryFactory.getRegistry(TEST_UTIL.getConfiguration());
CONN = new AsyncConnectionImpl(TEST_UTIL.getConfiguration(), registry,
- registry.getClusterId().get(), User.getCurrent());
+ registry.getClusterId().get(), User.getCurrent());
LOCATOR = new AsyncNonMetaRegionLocator(CONN);
SPLIT_KEYS = IntStream.range(1, 256).mapToObj(i -> Bytes.toBytes(String.format("%02x", i)))
- .toArray(byte[][]::new);
+ .toArray(byte[][]::new);
TEST_UTIL.createTable(TABLE_NAME, FAMILY, SPLIT_KEYS);
TEST_UTIL.waitTableAvailable(TABLE_NAME);
}
@@ -138,11 +139,11 @@ public class TestAsyncNonMetaRegionLocatorConcurrenyLimit {
TEST_UTIL.shutdownMiniCluster();
}
- private void assertLocs(List<CompletableFuture<HRegionLocation>> futures)
+ private void assertLocs(List<CompletableFuture<RegionLocations>> futures)
throws InterruptedException, ExecutionException {
assertEquals(256, futures.size());
for (int i = 0; i < futures.size(); i++) {
- HRegionLocation loc = futures.get(i).get();
+ HRegionLocation loc = futures.get(i).get().getDefaultRegionLocation();
if (i == 0) {
assertTrue(isEmptyStartRow(loc.getRegion().getStartKey()));
} else {
@@ -158,10 +159,11 @@ public class TestAsyncNonMetaRegionLocatorConcurrenyLimit {
@Test
public void test() throws InterruptedException, ExecutionException {
- List<CompletableFuture<HRegionLocation>> futures =
- IntStream.range(0, 256).mapToObj(i -> Bytes.toBytes(String.format("%02x", i)))
- .map(r -> LOCATOR.getRegionLocation(TABLE_NAME, r, RegionLocateType.CURRENT, false))
- .collect(toList());
+ List<CompletableFuture<RegionLocations>> futures =
+ IntStream.range(0, 256).mapToObj(i -> Bytes.toBytes(String.format("%02x", i)))
+ .map(r -> LOCATOR.getRegionLocations(TABLE_NAME, r, RegionReplicaUtil.DEFAULT_REPLICA_ID,
+ RegionLocateType.CURRENT, false))
+ .collect(toList());
assertLocs(futures);
assertTrue("max allowed is " + MAX_ALLOWED + " but actual is " + MAX_CONCURRENCY.get(),
MAX_CONCURRENCY.get() <= MAX_ALLOWED);
http://git-wip-us.apache.org/repos/asf/hbase/blob/db66e6cc/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncSingleRequestRpcRetryingCaller.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncSingleRequestRpcRetryingCaller.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncSingleRequestRpcRetryingCaller.java
index a6c2efb..7d8956b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncSingleRequestRpcRetryingCaller.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncSingleRequestRpcRetryingCaller.java
@@ -49,7 +49,7 @@ public class TestAsyncSingleRequestRpcRetryingCaller {
@ClassRule
public static final HBaseClassTestRule CLASS_RULE =
- HBaseClassTestRule.forClass(TestAsyncSingleRequestRpcRetryingCaller.class);
+ HBaseClassTestRule.forClass(TestAsyncSingleRequestRpcRetryingCaller.class);
private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
@@ -73,7 +73,7 @@ public class TestAsyncSingleRequestRpcRetryingCaller {
TEST_UTIL.waitTableAvailable(TABLE_NAME);
AsyncRegistry registry = AsyncRegistryFactory.getRegistry(TEST_UTIL.getConfiguration());
CONN = new AsyncConnectionImpl(TEST_UTIL.getConfiguration(), registry,
- registry.getClusterId().get(), User.getCurrent());
+ registry.getClusterId().get(), User.getCurrent());
}
@AfterClass
@@ -89,8 +89,8 @@ public class TestAsyncSingleRequestRpcRetryingCaller {
int index = TEST_UTIL.getHBaseCluster().getServerWith(loc.getRegion().getRegionName());
TEST_UTIL.getAdmin().move(loc.getRegion().getEncodedNameAsBytes(), Bytes.toBytes(
TEST_UTIL.getHBaseCluster().getRegionServer(1 - index).getServerName().getServerName()));
- AsyncTable<?> table = CONN.getTableBuilder(TABLE_NAME)
- .setRetryPause(100, TimeUnit.MILLISECONDS).setMaxRetries(30).build();
+ AsyncTable<?> table = CONN.getTableBuilder(TABLE_NAME).setRetryPause(100, TimeUnit.MILLISECONDS)
+ .setMaxRetries(30).build();
table.put(new Put(ROW).addColumn(FAMILY, QUALIFIER, VALUE)).get();
// move back
@@ -110,8 +110,8 @@ public class TestAsyncSingleRequestRpcRetryingCaller {
public void testMaxRetries() throws IOException, InterruptedException {
try {
CONN.callerFactory.single().table(TABLE_NAME).row(ROW).operationTimeout(1, TimeUnit.DAYS)
- .maxAttempts(3).pause(10, TimeUnit.MILLISECONDS)
- .action((controller, loc, stub) -> failedFuture()).call().get();
+ .maxAttempts(3).pause(10, TimeUnit.MILLISECONDS)
+ .action((controller, loc, stub) -> failedFuture()).call().get();
fail();
} catch (ExecutionException e) {
assertThat(e.getCause(), instanceOf(RetriesExhaustedException.class));
@@ -123,8 +123,8 @@ public class TestAsyncSingleRequestRpcRetryingCaller {
long startNs = System.nanoTime();
try {
CONN.callerFactory.single().table(TABLE_NAME).row(ROW).operationTimeout(1, TimeUnit.SECONDS)
- .pause(100, TimeUnit.MILLISECONDS).maxAttempts(Integer.MAX_VALUE)
- .action((controller, loc, stub) -> failedFuture()).call().get();
+ .pause(100, TimeUnit.MILLISECONDS).maxAttempts(Integer.MAX_VALUE)
+ .action((controller, loc, stub) -> failedFuture()).call().get();
fail();
} catch (ExecutionException e) {
e.printStackTrace();
@@ -141,30 +141,30 @@ public class TestAsyncSingleRequestRpcRetryingCaller {
AtomicInteger count = new AtomicInteger(0);
HRegionLocation loc = CONN.getRegionLocator(TABLE_NAME).getRegionLocation(ROW).get();
AsyncRegionLocator mockedLocator =
- new AsyncRegionLocator(CONN, AsyncConnectionImpl.RETRY_TIMER) {
- @Override
- CompletableFuture<HRegionLocation> getRegionLocation(TableName tableName, byte[] row,
- RegionLocateType locateType, long timeoutNs) {
- if (tableName.equals(TABLE_NAME)) {
- CompletableFuture<HRegionLocation> future = new CompletableFuture<>();
- if (count.getAndIncrement() == 0) {
- errorTriggered.set(true);
- future.completeExceptionally(new RuntimeException("Inject error!"));
- } else {
- future.complete(loc);
- }
- return future;
+ new AsyncRegionLocator(CONN, AsyncConnectionImpl.RETRY_TIMER) {
+ @Override
+ CompletableFuture<HRegionLocation> getRegionLocation(TableName tableName, byte[] row,
+ int replicaId, RegionLocateType locateType, long timeoutNs) {
+ if (tableName.equals(TABLE_NAME)) {
+ CompletableFuture<HRegionLocation> future = new CompletableFuture<>();
+ if (count.getAndIncrement() == 0) {
+ errorTriggered.set(true);
+ future.completeExceptionally(new RuntimeException("Inject error!"));
} else {
- return super.getRegionLocation(tableName, row, locateType, timeoutNs);
+ future.complete(loc);
}
+ return future;
+ } else {
+ return super.getRegionLocation(tableName, row, replicaId, locateType, timeoutNs);
}
+ }
- @Override
- void updateCachedLocation(HRegionLocation loc, Throwable exception) {
- }
- };
+ @Override
+ void updateCachedLocationOnError(HRegionLocation loc, Throwable exception) {
+ }
+ };
try (AsyncConnectionImpl mockedConn = new AsyncConnectionImpl(CONN.getConfiguration(),
- CONN.registry, CONN.registry.getClusterId().get(), User.getCurrent()) {
+ CONN.registry, CONN.registry.getClusterId().get(), User.getCurrent()) {
@Override
AsyncRegionLocator getLocator() {
@@ -172,7 +172,7 @@ public class TestAsyncSingleRequestRpcRetryingCaller {
}
}) {
AsyncTable<?> table = mockedConn.getTableBuilder(TABLE_NAME)
- .setRetryPause(100, TimeUnit.MILLISECONDS).setMaxRetries(5).build();
+ .setRetryPause(100, TimeUnit.MILLISECONDS).setMaxRetries(5).build();
table.put(new Put(ROW).addColumn(FAMILY, QUALIFIER, VALUE)).get();
assertTrue(errorTriggered.get());
errorTriggered.set(false);
http://git-wip-us.apache.org/repos/asf/hbase/blob/db66e6cc/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableLocatePrefetch.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableLocatePrefetch.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableLocatePrefetch.java
index 13d8000..6c6bb98 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableLocatePrefetch.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableLocatePrefetch.java
@@ -69,8 +69,8 @@ public class TestAsyncTableLocatePrefetch {
@Test
public void test() throws InterruptedException, ExecutionException {
- assertNotNull(LOCATOR
- .getRegionLocation(TABLE_NAME, Bytes.toBytes("zzz"), RegionLocateType.CURRENT, false).get());
+ assertNotNull(LOCATOR.getRegionLocations(TABLE_NAME, Bytes.toBytes("zzz"),
+ RegionReplicaUtil.DEFAULT_REPLICA_ID, RegionLocateType.CURRENT, false).get());
// we finish the request before we adding the remaining results to cache so sleep a bit here
Thread.sleep(1000);
// confirm that the locations of all the regions have been cached.
http://git-wip-us.apache.org/repos/asf/hbase/blob/db66e6cc/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableRegionReplicasGet.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableRegionReplicasGet.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableRegionReplicasGet.java
new file mode 100644
index 0000000..0445a0e
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableRegionReplicasGet.java
@@ -0,0 +1,204 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.client;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Optional;
+import java.util.concurrent.ForkJoinPool;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Supplier;
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.coprocessor.ObserverContext;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+import org.apache.hadoop.hbase.coprocessor.RegionObserver;
+import org.apache.hadoop.hbase.regionserver.HRegion;
+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.JVMClusterUtil.RegionServerThread;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameter;
+import org.junit.runners.Parameterized.Parameters;
+
+@RunWith(Parameterized.class)
+@Category({ MediumTests.class, ClientTests.class })
+public class TestAsyncTableRegionReplicasGet {
+
+ @ClassRule
+ public static final HBaseClassTestRule CLASS_RULE =
+ HBaseClassTestRule.forClass(TestAsyncTableRegionReplicasGet.class);
+
+ private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+
+ private static TableName TABLE_NAME = TableName.valueOf("async");
+
+ private static byte[] FAMILY = Bytes.toBytes("cf");
+
+ private static byte[] QUALIFIER = Bytes.toBytes("cq");
+
+ private static byte[] ROW = Bytes.toBytes("row");
+
+ private static byte[] VALUE = Bytes.toBytes("value");
+
+ private static AsyncConnection ASYNC_CONN;
+
+ @Rule
+ public TestName testName = new TestName();
+
+ @Parameter
+ public Supplier<AsyncTable<?>> getTable;
+
+ private static AsyncTable<?> getRawTable() {
+ return ASYNC_CONN.getTable(TABLE_NAME);
+ }
+
+ private static AsyncTable<?> getTable() {
+ return ASYNC_CONN.getTable(TABLE_NAME, ForkJoinPool.commonPool());
+ }
+
+ @Parameters
+ public static List<Object[]> params() {
+ return Arrays.asList(new Supplier<?>[] { TestAsyncTableRegionReplicasGet::getRawTable },
+ new Supplier<?>[] { TestAsyncTableRegionReplicasGet::getTable });
+ }
+
+ private static volatile boolean FAIL_PRIMARY_GET = false;
+
+ private static AtomicInteger PRIMARY_GET_COUNT = new AtomicInteger(0);
+
+ private static AtomicInteger SECONDARY_GET_COUNT = new AtomicInteger(0);
+
+ public static final class FailPrimaryGetCP implements RegionObserver, RegionCoprocessor {
+
+ @Override
+ public Optional<RegionObserver> getRegionObserver() {
+ return Optional.of(this);
+ }
+
+ @Override
+ public void preGetOp(ObserverContext<RegionCoprocessorEnvironment> c, Get get,
+ List<Cell> result) throws IOException {
+ RegionInfo region = c.getEnvironment().getRegionInfo();
+ if (!region.getTable().equals(TABLE_NAME)) {
+ return;
+ }
+ if (region.getReplicaId() != RegionReplicaUtil.DEFAULT_REPLICA_ID) {
+ SECONDARY_GET_COUNT.incrementAndGet();
+ } else {
+ PRIMARY_GET_COUNT.incrementAndGet();
+ if (FAIL_PRIMARY_GET) {
+ throw new IOException("Inject error");
+ }
+ }
+ }
+ }
+
+ private static boolean allReplicasHaveRow() throws IOException {
+ for (RegionServerThread t : TEST_UTIL.getMiniHBaseCluster().getRegionServerThreads()) {
+ for (HRegion region : t.getRegionServer().getRegions(TABLE_NAME)) {
+ if (region.get(new Get(ROW), false).isEmpty()) {
+ return false;
+ }
+ }
+ }
+ return true;
+ }
+
+ @BeforeClass
+ public static void setUpBeforeClass() throws Exception {
+ // 10 mins
+ TEST_UTIL.getConfiguration().setLong(HConstants.HBASE_RPC_READ_TIMEOUT_KEY,
+ TimeUnit.MINUTES.toMillis(10));
+ // 1 second
+ TEST_UTIL.getConfiguration().setLong(ConnectionConfiguration.PRIMARY_CALL_TIMEOUT_MICROSECOND,
+ TimeUnit.SECONDS.toMicros(1));
+ // set a small pause so we will retry very quickly
+ TEST_UTIL.getConfiguration().setLong(HConstants.HBASE_CLIENT_PAUSE, 10);
+ // infinite retry
+ TEST_UTIL.getConfiguration().setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, Integer.MAX_VALUE);
+ TEST_UTIL.startMiniCluster(3);
+ TEST_UTIL.getAdmin()
+ .createTable(TableDescriptorBuilder.newBuilder(TABLE_NAME)
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILY)).setRegionReplication(3)
+ .setCoprocessor(FailPrimaryGetCP.class.getName()).build());
+ TEST_UTIL.waitUntilAllRegionsAssigned(TABLE_NAME);
+ ASYNC_CONN = ConnectionFactory.createAsyncConnection(TEST_UTIL.getConfiguration()).get();
+ AsyncTable<?> table = ASYNC_CONN.getTable(TABLE_NAME);
+ table.put(new Put(ROW).addColumn(FAMILY, QUALIFIER, VALUE)).get();
+ // this is the fastest way to let all replicas have the row
+ TEST_UTIL.getAdmin().disableTable(TABLE_NAME);
+ TEST_UTIL.getAdmin().enableTable(TABLE_NAME);
+ TEST_UTIL.waitFor(30000, () -> allReplicasHaveRow());
+ }
+
+ @AfterClass
+ public static void tearDownAfterClass() throws Exception {
+ IOUtils.closeQuietly(ASYNC_CONN);
+ TEST_UTIL.shutdownMiniCluster();
+ }
+
+ @Test
+ public void testNoReplicaRead() throws Exception {
+ FAIL_PRIMARY_GET = false;
+ SECONDARY_GET_COUNT.set(0);
+ AsyncTable<?> table = getTable.get();
+ Get get = new Get(ROW).setConsistency(Consistency.TIMELINE);
+ for (int i = 0; i < 1000; i++) {
+ assertArrayEquals(VALUE, table.get(get).get().getValue(FAMILY, QUALIFIER));
+ }
+ // the primary region is fine and the primary timeout is 1 second which is long enough, so we
+ // should not send any requests to secondary replicas even if the consistency is timeline.
+ Thread.sleep(5000);
+ assertEquals(0, SECONDARY_GET_COUNT.get());
+ }
+
+ @Test
+ public void testReplicaRead() throws Exception {
+ // fail the primary get request
+ FAIL_PRIMARY_GET = true;
+ Get get = new Get(ROW).setConsistency(Consistency.TIMELINE);
+ // make sure that we could still get the value from secondary replicas
+ AsyncTable<?> table = getTable.get();
+ assertArrayEquals(VALUE, table.get(get).get().getValue(FAMILY, QUALIFIER));
+ // make sure that the primary request has been canceled
+ Thread.sleep(5000);
+ int count = PRIMARY_GET_COUNT.get();
+ Thread.sleep(10000);
+ assertEquals(count, PRIMARY_GET_COUNT.get());
+ }
+}
http://git-wip-us.apache.org/repos/asf/hbase/blob/db66e6cc/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestZKAsyncRegistry.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestZKAsyncRegistry.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestZKAsyncRegistry.java
index db7546f..46890d0 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestZKAsyncRegistry.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestZKAsyncRegistry.java
@@ -25,7 +25,6 @@ import static org.junit.Assert.assertNotSame;
import java.io.IOException;
import java.util.concurrent.ExecutionException;
-import java.util.concurrent.atomic.AtomicBoolean;
import java.util.stream.IntStream;
import org.apache.commons.io.IOUtils;
import org.apache.hadoop.conf.Configuration;
@@ -35,7 +34,6 @@ import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.RegionLocations;
import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.Waiter.ExplainingPredicate;
import org.apache.hadoop.hbase.testclassification.ClientTests;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.zookeeper.ReadOnlyZKClient;
@@ -52,43 +50,13 @@ public class TestZKAsyncRegistry {
@ClassRule
public static final HBaseClassTestRule CLASS_RULE =
- HBaseClassTestRule.forClass(TestZKAsyncRegistry.class);
+ HBaseClassTestRule.forClass(TestZKAsyncRegistry.class);
- private static final Logger LOG = LoggerFactory.getLogger(TestZKAsyncRegistry.class);
- private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+ static final Logger LOG = LoggerFactory.getLogger(TestZKAsyncRegistry.class);
+ static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
private static ZKAsyncRegistry REGISTRY;
- // waits for all replicas to have region location
- static void waitUntilAllReplicasHavingRegionLocation(TableName tbl) throws IOException {
- TEST_UTIL.waitFor(
- TEST_UTIL.getConfiguration().getLong("hbase.client.sync.wait.timeout.msec", 60000), 200, true,
- new ExplainingPredicate<IOException>() {
- @Override
- public String explainFailure() throws IOException {
- return TEST_UTIL.explainTableAvailability(tbl);
- }
-
- @Override
- public boolean evaluate() throws IOException {
- AtomicBoolean ready = new AtomicBoolean(true);
- try {
- RegionLocations locs = REGISTRY.getMetaRegionLocation().get();
- assertEquals(3, locs.getRegionLocations().length);
- IntStream.range(0, 3).forEach(i -> {
- HRegionLocation loc = locs.getRegionLocation(i);
- if (loc == null) {
- ready.set(false);
- }
- });
- } catch (Exception e) {
- ready.set(false);
- }
- return ready.get();
- }
- });
- }
-
@BeforeClass
public static void setUp() throws Exception {
TEST_UTIL.getConfiguration().setInt(META_REPLICAS_NUM, 3);
@@ -107,14 +75,14 @@ public class TestZKAsyncRegistry {
LOG.info("STARTED TEST");
String clusterId = REGISTRY.getClusterId().get();
String expectedClusterId = TEST_UTIL.getHBaseCluster().getMaster().getClusterId();
- assertEquals("Expected " + expectedClusterId + ", found=" + clusterId,
- expectedClusterId, clusterId);
+ assertEquals("Expected " + expectedClusterId + ", found=" + clusterId, expectedClusterId,
+ clusterId);
assertEquals(TEST_UTIL.getHBaseCluster().getClusterMetrics().getLiveServerMetrics().size(),
REGISTRY.getCurrentNrHRS().get().intValue());
assertEquals(TEST_UTIL.getHBaseCluster().getMaster().getServerName(),
REGISTRY.getMasterAddress().get());
assertEquals(-1, REGISTRY.getMasterInfoPort().get().intValue());
- waitUntilAllReplicasHavingRegionLocation(TableName.META_TABLE_NAME);
+ RegionReplicaTestHelper.waitUntilAllMetaReplicasHavingRegionLocation(REGISTRY, 3);
RegionLocations locs = REGISTRY.getMetaRegionLocation().get();
assertEquals(3, locs.getRegionLocations().length);
IntStream.range(0, 3).forEach(i -> {
[12/15] hbase git commit: HBASE-21515 Also initialize an
AsyncClusterConnection in HRegionServer
Posted by zh...@apache.org.
HBASE-21515 Also initialize an AsyncClusterConnection in HRegionServer
Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/6fff6c81
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/6fff6c81
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/6fff6c81
Branch: refs/heads/HBASE-21512
Commit: 6fff6c81fe1658827159e85845c9d3e595797375
Parents: 3fbdd5b
Author: zhangduo <zh...@apache.org>
Authored: Fri Nov 30 08:23:47 2018 +0800
Committer: Duo Zhang <zh...@apache.org>
Committed: Fri Jan 4 14:31:41 2019 +0800
----------------------------------------------------------------------
.../hbase/client/AsyncClusterConnection.java | 38 ++++++++++++
.../hbase/client/AsyncConnectionImpl.java | 39 ++++++------
.../hbase/client/ClusterConnectionFactory.java | 63 ++++++++++++++++++++
.../hadoop/hbase/client/ConnectionFactory.java | 5 +-
.../hadoop/hbase/util/ReflectionUtils.java | 22 ++++---
.../java/org/apache/hadoop/hbase/Server.java | 20 +++++++
.../org/apache/hadoop/hbase/master/HMaster.java | 3 +
.../hbase/regionserver/HRegionServer.java | 56 ++++++++++++-----
.../regionserver/ReplicationSyncUp.java | 6 ++
.../hadoop/hbase/MockRegionServerServices.java | 5 ++
.../client/TestAsyncNonMetaRegionLocator.java | 2 +-
...syncNonMetaRegionLocatorConcurrenyLimit.java | 2 +-
.../client/TestAsyncRegionLocatorTimeout.java | 2 +-
...TestAsyncSingleRequestRpcRetryingCaller.java | 4 +-
.../hbase/client/TestAsyncTableNoncedRetry.java | 2 +-
.../hbase/master/MockNoopMasterServices.java | 6 ++
.../hadoop/hbase/master/MockRegionServer.java | 5 ++
.../hbase/master/TestActiveMasterManager.java | 6 ++
.../hbase/master/cleaner/TestHFileCleaner.java | 6 ++
.../master/cleaner/TestHFileLinkCleaner.java | 6 ++
.../hbase/master/cleaner/TestLogsCleaner.java | 6 ++
.../cleaner/TestReplicationHFileCleaner.java | 6 ++
.../regionserver/TestHeapMemoryManager.java | 6 ++
.../hbase/regionserver/TestSplitLogWorker.java | 6 ++
.../hbase/regionserver/TestWALLockup.java | 6 ++
.../TestReplicationTrackerZKImpl.java | 6 ++
.../TestReplicationSourceManager.java | 6 ++
.../security/token/TestTokenAuthentication.java | 6 ++
.../apache/hadoop/hbase/util/MockServer.java | 6 ++
29 files changed, 302 insertions(+), 50 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/hbase/blob/6fff6c81/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncClusterConnection.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncClusterConnection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncClusterConnection.java
new file mode 100644
index 0000000..c7dea25
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncClusterConnection.java
@@ -0,0 +1,38 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.client;
+
+import org.apache.hadoop.hbase.ipc.RpcClient;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * The asynchronous connection for internal usage.
+ */
+@InterfaceAudience.Private
+public interface AsyncClusterConnection extends AsyncConnection {
+
+ /**
+ * Get the nonce generator for this connection.
+ */
+ NonceGenerator getNonceGenerator();
+
+ /**
+ * Get the rpc client we used to communicate with other servers.
+ */
+ RpcClient getRpcClient();
+}
http://git-wip-us.apache.org/repos/asf/hbase/blob/6fff6c81/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java
index 361d5b2..188e830 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java
@@ -21,48 +21,48 @@ import static org.apache.hadoop.hbase.client.ConnectionUtils.NO_NONCE_GENERATOR;
import static org.apache.hadoop.hbase.client.ConnectionUtils.getStubKey;
import static org.apache.hadoop.hbase.client.NonceGenerator.CLIENT_NONCES_ENABLED_KEY;
-import org.apache.hadoop.hbase.AuthUtil;
-import org.apache.hadoop.hbase.ChoreService;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
-
-import org.apache.hbase.thirdparty.io.netty.util.HashedWheelTimer;
-
import java.io.IOException;
+import java.net.SocketAddress;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicReference;
-
import org.apache.commons.io.IOUtils;
import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.AuthUtil;
+import org.apache.hadoop.hbase.ChoreService;
import org.apache.hadoop.hbase.MasterNotRunningException;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
import org.apache.hadoop.hbase.ipc.HBaseRpcController;
import org.apache.hadoop.hbase.ipc.RpcClient;
import org.apache.hadoop.hbase.ipc.RpcClientFactory;
import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.util.CollectionUtils;
+import org.apache.hadoop.hbase.util.Threads;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
+import org.apache.hbase.thirdparty.io.netty.util.HashedWheelTimer;
+
import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientService;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsMasterRunningResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MasterService;
-import org.apache.hadoop.hbase.util.CollectionUtils;
-import org.apache.hadoop.hbase.util.Threads;
/**
* The implementation of AsyncConnection.
*/
@InterfaceAudience.Private
-class AsyncConnectionImpl implements AsyncConnection {
+class AsyncConnectionImpl implements AsyncClusterConnection {
private static final Logger LOG = LoggerFactory.getLogger(AsyncConnectionImpl.class);
@@ -105,7 +105,7 @@ class AsyncConnectionImpl implements AsyncConnection {
private ChoreService authService;
public AsyncConnectionImpl(Configuration conf, AsyncRegistry registry, String clusterId,
- User user) {
+ SocketAddress localAddress, User user) {
this.conf = conf;
this.user = user;
if (user.isLoginFromKeytab()) {
@@ -113,7 +113,7 @@ class AsyncConnectionImpl implements AsyncConnection {
}
this.connConf = new AsyncConnectionConfiguration(conf);
this.registry = registry;
- this.rpcClient = RpcClientFactory.createClient(conf, clusterId);
+ this.rpcClient = RpcClientFactory.createClient(conf, clusterId, localAddress, null);
this.rpcControllerFactory = RpcControllerFactory.instantiate(conf);
this.hostnameCanChange = conf.getBoolean(RESOLVE_HOSTNAME_ON_FAIL_KEY, true);
this.rpcTimeout =
@@ -157,11 +157,16 @@ class AsyncConnectionImpl implements AsyncConnection {
}
// ditto
- @VisibleForTesting
+ @Override
public NonceGenerator getNonceGenerator() {
return nonceGenerator;
}
+ @Override
+ public RpcClient getRpcClient() {
+ return rpcClient;
+ }
+
private ClientService.Interface createRegionServerStub(ServerName serverName) throws IOException {
return ClientService.newStub(rpcClient.createRpcChannel(serverName, user, rpcTimeout));
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/6fff6c81/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterConnectionFactory.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterConnectionFactory.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterConnectionFactory.java
new file mode 100644
index 0000000..68c0630
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterConnectionFactory.java
@@ -0,0 +1,63 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.client;
+
+import java.io.IOException;
+import java.io.InterruptedIOException;
+import java.net.SocketAddress;
+import java.util.concurrent.ExecutionException;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import org.apache.hbase.thirdparty.com.google.common.base.Throwables;
+
+/**
+ * The factory for creating {@link AsyncClusterConnection}.
+ */
+@InterfaceAudience.Private
+public final class ClusterConnectionFactory {
+
+ private ClusterConnectionFactory() {
+ }
+
+ /**
+ * Create a new {@link AsyncClusterConnection} instance.
+ * <p/>
+ * Unlike what we have done in {@link ConnectionFactory}, here we just return an
+ * {@link AsyncClusterConnection} instead of a {@link java.util.concurrent.CompletableFuture},
+ * which means this method could block on fetching the cluster id. This is just used to simplify
+ * the implementation, as when starting new region servers, we do not need to be event-driven. Can
+ * change later if we want a {@link java.util.concurrent.CompletableFuture} here.
+ */
+ public static AsyncClusterConnection createAsyncClusterConnection(Configuration conf,
+ SocketAddress localAddress, User user) throws IOException {
+ AsyncRegistry registry = AsyncRegistryFactory.getRegistry(conf);
+ String clusterId;
+ try {
+ clusterId = registry.getClusterId().get();
+ } catch (InterruptedException e) {
+ throw (IOException) new InterruptedIOException().initCause(e);
+ } catch (ExecutionException e) {
+ Throwable cause = e.getCause();
+ Throwables.propagateIfPossible(cause, IOException.class);
+ throw new IOException(cause);
+ }
+ return new AsyncConnectionImpl(conf, registry, clusterId, localAddress, user);
+ }
+}
http://git-wip-us.apache.org/repos/asf/hbase/blob/6fff6c81/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionFactory.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionFactory.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionFactory.java
index e24af74..2ba732a 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionFactory.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionFactory.java
@@ -295,9 +295,8 @@ public class ConnectionFactory {
AsyncConnectionImpl.class, AsyncConnection.class);
try {
future.complete(
- user.runAs((PrivilegedExceptionAction<? extends AsyncConnection>)() ->
- ReflectionUtils.newInstance(clazz, conf, registry, clusterId, user))
- );
+ user.runAs((PrivilegedExceptionAction<? extends AsyncConnection>) () -> ReflectionUtils
+ .newInstance(clazz, conf, registry, clusterId, null, user)));
} catch (Exception e) {
future.completeExceptionally(e);
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/6fff6c81/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ReflectionUtils.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ReflectionUtils.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ReflectionUtils.java
index a136846..268249d 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ReflectionUtils.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ReflectionUtils.java
@@ -83,15 +83,19 @@ public class ReflectionUtils {
boolean match = true;
for (int i = 0; i < ctorParamTypes.length && match; ++i) {
- Class<?> paramType = paramTypes[i].getClass();
- match = (!ctorParamTypes[i].isPrimitive()) ? ctorParamTypes[i].isAssignableFrom(paramType) :
- ((int.class.equals(ctorParamTypes[i]) && Integer.class.equals(paramType)) ||
- (long.class.equals(ctorParamTypes[i]) && Long.class.equals(paramType)) ||
- (double.class.equals(ctorParamTypes[i]) && Double.class.equals(paramType)) ||
- (char.class.equals(ctorParamTypes[i]) && Character.class.equals(paramType)) ||
- (short.class.equals(ctorParamTypes[i]) && Short.class.equals(paramType)) ||
- (boolean.class.equals(ctorParamTypes[i]) && Boolean.class.equals(paramType)) ||
- (byte.class.equals(ctorParamTypes[i]) && Byte.class.equals(paramType)));
+ if (paramTypes[i] == null) {
+ match = !ctorParamTypes[i].isPrimitive();
+ } else {
+ Class<?> paramType = paramTypes[i].getClass();
+ match = (!ctorParamTypes[i].isPrimitive()) ? ctorParamTypes[i].isAssignableFrom(paramType)
+ : ((int.class.equals(ctorParamTypes[i]) && Integer.class.equals(paramType)) ||
+ (long.class.equals(ctorParamTypes[i]) && Long.class.equals(paramType)) ||
+ (double.class.equals(ctorParamTypes[i]) && Double.class.equals(paramType)) ||
+ (char.class.equals(ctorParamTypes[i]) && Character.class.equals(paramType)) ||
+ (short.class.equals(ctorParamTypes[i]) && Short.class.equals(paramType)) ||
+ (boolean.class.equals(ctorParamTypes[i]) && Boolean.class.equals(paramType)) ||
+ (byte.class.equals(ctorParamTypes[i]) && Byte.class.equals(paramType)));
+ }
}
if (match) {
http://git-wip-us.apache.org/repos/asf/hbase/blob/6fff6c81/hbase-server/src/main/java/org/apache/hadoop/hbase/Server.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/Server.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/Server.java
index fb898ea..c33d5af 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/Server.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/Server.java
@@ -20,6 +20,8 @@ package org.apache.hadoop.hbase;
import java.io.IOException;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.hbase.client.AsyncClusterConnection;
+import org.apache.hadoop.hbase.client.AsyncConnection;
import org.apache.hadoop.hbase.client.ClusterConnection;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
@@ -61,6 +63,24 @@ public interface Server extends Abortable, Stoppable {
ClusterConnection getClusterConnection();
/**
+ * Returns a reference to the servers' async connection.
+ * <p/>
+ * Important note: this method returns a reference to Connection which is managed by Server
+ * itself, so callers must NOT attempt to close connection obtained.
+ */
+ default AsyncConnection getAsyncConnection() {
+ return getAsyncClusterConnection();
+ }
+
+ /**
+ * Returns a reference to the servers' async cluster connection.
+ * <p/>
+ * Important note: this method returns a reference to Connection which is managed by Server
+ * itself, so callers must NOT attempt to close connection obtained.
+ */
+ AsyncClusterConnection getAsyncClusterConnection();
+
+ /**
* @return The unique server name for this server.
*/
ServerName getServerName();
http://git-wip-us.apache.org/repos/asf/hbase/blob/6fff6c81/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 0bcef59..52005d6 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
@@ -3008,6 +3008,9 @@ public class HMaster extends HRegionServer implements MasterServices {
if (this.clusterConnection != null) {
this.clusterConnection.close();
}
+ if (this.asyncClusterConnection != null) {
+ this.asyncClusterConnection.close();
+ }
}
public void stopMaster() throws IOException {
http://git-wip-us.apache.org/repos/asf/hbase/blob/6fff6c81/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 13f277b..375b3f8 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
@@ -77,7 +77,9 @@ import org.apache.hadoop.hbase.TableDescriptors;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.YouAreDeadException;
import org.apache.hadoop.hbase.ZNodeClearer;
+import org.apache.hadoop.hbase.client.AsyncClusterConnection;
import org.apache.hadoop.hbase.client.ClusterConnection;
+import org.apache.hadoop.hbase.client.ClusterConnectionFactory;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionUtils;
import org.apache.hadoop.hbase.client.RegionInfo;
@@ -105,7 +107,6 @@ import org.apache.hadoop.hbase.io.util.MemorySizeUtil;
import org.apache.hadoop.hbase.ipc.CoprocessorRpcUtils;
import org.apache.hadoop.hbase.ipc.NettyRpcClientConfigHelper;
import org.apache.hadoop.hbase.ipc.RpcClient;
-import org.apache.hadoop.hbase.ipc.RpcClientFactory;
import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
import org.apache.hadoop.hbase.ipc.RpcServer;
import org.apache.hadoop.hbase.ipc.RpcServerInterface;
@@ -263,6 +264,11 @@ public class HRegionServer extends HasThread implements
protected ClusterConnection clusterConnection;
/**
+ * The asynchronous cluster connection to be shared by services.
+ */
+ protected AsyncClusterConnection asyncClusterConnection;
+
+ /**
* Go here to get table descriptors.
*/
protected TableDescriptors tableDescriptors;
@@ -776,11 +782,7 @@ public class HRegionServer extends HasThread implements
return true;
}
- /**
- * Create a 'smarter' Connection, one that is capable of by-passing RPC if the request is to the
- * local server; i.e. a short-circuit Connection. Safe to use going to local or remote server.
- */
- private ClusterConnection createClusterConnection() throws IOException {
+ private Configuration unsetClientZookeeperQuorum() {
Configuration conf = this.conf;
if (conf.get(HConstants.CLIENT_ZOOKEEPER_QUORUM) != null) {
// Use server ZK cluster for server-issued connections, so we clone
@@ -788,11 +790,20 @@ public class HRegionServer extends HasThread implements
conf = new Configuration(this.conf);
conf.unset(HConstants.CLIENT_ZOOKEEPER_QUORUM);
}
+ return conf;
+ }
+
+ /**
+ * Create a 'smarter' Connection, one that is capable of by-passing RPC if the request is to the
+ * local server; i.e. a short-circuit Connection. Safe to use going to local or remote server.
+ */
+ private ClusterConnection createClusterConnection() throws IOException {
// Create a cluster connection that when appropriate, can short-circuit and go directly to the
// local server if the request is to the local server bypassing RPC. Can be used for both local
// and remote invocations.
- ClusterConnection conn = ConnectionUtils.createShortCircuitConnection(conf, null,
- userProvider.getCurrent(), serverName, rpcServices, rpcServices);
+ ClusterConnection conn =
+ ConnectionUtils.createShortCircuitConnection(unsetClientZookeeperQuorum(), null,
+ userProvider.getCurrent(), serverName, rpcServices, rpcServices);
// This is used to initialize the batch thread pool inside the connection implementation.
// When deploy a fresh cluster, we may first use the cluster connection in InitMetaProcedure,
// which will be executed inside the PEWorker, and then the batch thread pool will inherit the
@@ -826,9 +837,12 @@ public class HRegionServer extends HasThread implements
/**
* Setup our cluster connection if not already initialized.
*/
- protected synchronized void setupClusterConnection() throws IOException {
+ protected final synchronized void setupClusterConnection() throws IOException {
if (clusterConnection == null) {
clusterConnection = createClusterConnection();
+ asyncClusterConnection =
+ ClusterConnectionFactory.createAsyncClusterConnection(unsetClientZookeeperQuorum(),
+ new InetSocketAddress(this.rpcServices.isa.getAddress(), 0), userProvider.getCurrent());
}
}
@@ -842,8 +856,7 @@ public class HRegionServer extends HasThread implements
initializeZooKeeper();
setupClusterConnection();
// Setup RPC client for master communication
- this.rpcClient = RpcClientFactory.createClient(conf, clusterId, new InetSocketAddress(
- this.rpcServices.isa.getAddress(), 0), clusterConnection.getConnectionMetrics());
+ this.rpcClient = asyncClusterConnection.getRpcClient();
} catch (Throwable t) {
// Call stop if error or process will stick around for ever since server
// puts up non-daemon threads.
@@ -1107,7 +1120,15 @@ public class HRegionServer extends HasThread implements
LOG.warn("Attempt to close server's short circuit ClusterConnection failed.", e);
}
}
-
+ if (this.asyncClusterConnection != null) {
+ try {
+ this.asyncClusterConnection.close();
+ } catch (IOException e) {
+ // Although the {@link Closeable} interface throws an {@link
+ // IOException}, in reality, the implementation would never do that.
+ LOG.warn("Attempt to close server's AsyncClusterConnection failed.", e);
+ }
+ }
// Closing the compactSplit thread before closing meta regions
if (!this.killed && containsMetaTableRegions()) {
if (!abortRequested || this.fsOk) {
@@ -3737,9 +3758,9 @@ public class HRegionServer extends HasThread implements
}
@Override
- public EntityLock regionLock(List<RegionInfo> regionInfos, String description,
- Abortable abort) throws IOException {
- return new LockServiceClient(conf, lockStub, clusterConnection.getNonceGenerator())
+ public EntityLock regionLock(List<RegionInfo> regionInfos, String description, Abortable abort)
+ throws IOException {
+ return new LockServiceClient(conf, lockStub, asyncClusterConnection.getNonceGenerator())
.regionLock(regionInfos, description, abort);
}
@@ -3843,4 +3864,9 @@ public class HRegionServer extends HasThread implements
System.exit(1);
}
}
+
+ @Override
+ public AsyncClusterConnection getAsyncClusterConnection() {
+ return asyncClusterConnection;
+ }
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/6fff6c81/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSyncUp.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSyncUp.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSyncUp.java
index c7bccb3..7d1245c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSyncUp.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSyncUp.java
@@ -29,6 +29,7 @@ import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.client.AsyncClusterConnection;
import org.apache.hadoop.hbase.client.ClusterConnection;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.util.FSUtils;
@@ -180,5 +181,10 @@ public class ReplicationSyncUp extends Configured implements Tool {
public Connection createConnection(Configuration conf) throws IOException {
return null;
}
+
+ @Override
+ public AsyncClusterConnection getAsyncClusterConnection() {
+ return null;
+ }
}
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/6fff6c81/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 0e4f241..5205960 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
@@ -31,6 +31,7 @@ import java.util.concurrent.ConcurrentSkipListMap;
import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.hbase.client.AsyncClusterConnection;
import org.apache.hadoop.hbase.client.ClusterConnection;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.RegionInfo;
@@ -368,4 +369,8 @@ public class MockRegionServerServices implements RegionServerServices {
public Optional<MobFileCache> getMobFileCache() {
return Optional.empty();
}
+
+ public AsyncClusterConnection getAsyncClusterConnection() {
+ return null;
+ }
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/6fff6c81/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocator.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocator.java
index eeaf99f..550a6f3 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocator.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocator.java
@@ -81,7 +81,7 @@ public class TestAsyncNonMetaRegionLocator {
TEST_UTIL.getAdmin().balancerSwitch(false, true);
AsyncRegistry registry = AsyncRegistryFactory.getRegistry(TEST_UTIL.getConfiguration());
CONN = new AsyncConnectionImpl(TEST_UTIL.getConfiguration(), registry,
- registry.getClusterId().get(), User.getCurrent());
+ registry.getClusterId().get(), null, User.getCurrent());
LOCATOR = new AsyncNonMetaRegionLocator(CONN);
SPLIT_KEYS = new byte[8][];
for (int i = 111; i < 999; i += 111) {
http://git-wip-us.apache.org/repos/asf/hbase/blob/6fff6c81/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocatorConcurrenyLimit.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocatorConcurrenyLimit.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocatorConcurrenyLimit.java
index 8cdb4a9..7e06218 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocatorConcurrenyLimit.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocatorConcurrenyLimit.java
@@ -125,7 +125,7 @@ public class TestAsyncNonMetaRegionLocatorConcurrenyLimit {
TEST_UTIL.getAdmin().balancerSwitch(false, true);
AsyncRegistry registry = AsyncRegistryFactory.getRegistry(TEST_UTIL.getConfiguration());
CONN = new AsyncConnectionImpl(TEST_UTIL.getConfiguration(), registry,
- registry.getClusterId().get(), User.getCurrent());
+ registry.getClusterId().get(), null, User.getCurrent());
LOCATOR = new AsyncNonMetaRegionLocator(CONN);
SPLIT_KEYS = IntStream.range(1, 256).mapToObj(i -> Bytes.toBytes(String.format("%02x", i)))
.toArray(byte[][]::new);
http://git-wip-us.apache.org/repos/asf/hbase/blob/6fff6c81/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionLocatorTimeout.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionLocatorTimeout.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionLocatorTimeout.java
index 758aa30..0e28f96 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionLocatorTimeout.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionLocatorTimeout.java
@@ -96,7 +96,7 @@ public class TestAsyncRegionLocatorTimeout {
TEST_UTIL.waitTableAvailable(TABLE_NAME);
AsyncRegistry registry = AsyncRegistryFactory.getRegistry(TEST_UTIL.getConfiguration());
CONN = new AsyncConnectionImpl(TEST_UTIL.getConfiguration(), registry,
- registry.getClusterId().get(), User.getCurrent());
+ registry.getClusterId().get(), null, User.getCurrent());
LOCATOR = CONN.getLocator();
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/6fff6c81/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncSingleRequestRpcRetryingCaller.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncSingleRequestRpcRetryingCaller.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncSingleRequestRpcRetryingCaller.java
index 7d8956b..29dcd56 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncSingleRequestRpcRetryingCaller.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncSingleRequestRpcRetryingCaller.java
@@ -73,7 +73,7 @@ public class TestAsyncSingleRequestRpcRetryingCaller {
TEST_UTIL.waitTableAvailable(TABLE_NAME);
AsyncRegistry registry = AsyncRegistryFactory.getRegistry(TEST_UTIL.getConfiguration());
CONN = new AsyncConnectionImpl(TEST_UTIL.getConfiguration(), registry,
- registry.getClusterId().get(), User.getCurrent());
+ registry.getClusterId().get(), null, User.getCurrent());
}
@AfterClass
@@ -164,7 +164,7 @@ public class TestAsyncSingleRequestRpcRetryingCaller {
}
};
try (AsyncConnectionImpl mockedConn = new AsyncConnectionImpl(CONN.getConfiguration(),
- CONN.registry, CONN.registry.getClusterId().get(), User.getCurrent()) {
+ CONN.registry, CONN.registry.getClusterId().get(), null, User.getCurrent()) {
@Override
AsyncRegionLocator getLocator() {
http://git-wip-us.apache.org/repos/asf/hbase/blob/6fff6c81/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableNoncedRetry.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableNoncedRetry.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableNoncedRetry.java
index 3008561..e1e55f5 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableNoncedRetry.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableNoncedRetry.java
@@ -85,7 +85,7 @@ public class TestAsyncTableNoncedRetry {
TEST_UTIL.waitTableAvailable(TABLE_NAME);
AsyncRegistry registry = AsyncRegistryFactory.getRegistry(TEST_UTIL.getConfiguration());
ASYNC_CONN = new AsyncConnectionImpl(TEST_UTIL.getConfiguration(), registry,
- registry.getClusterId().get(), User.getCurrent()) {
+ registry.getClusterId().get(), null, User.getCurrent()) {
@Override
public NonceGenerator getNonceGenerator() {
http://git-wip-us.apache.org/repos/asf/hbase/blob/6fff6c81/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 9c55f57..3ebad66 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
@@ -29,6 +29,7 @@ import org.apache.hadoop.hbase.CoordinatedStateManager;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableDescriptors;
import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.AsyncClusterConnection;
import org.apache.hadoop.hbase.client.ClusterConnection;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
import org.apache.hadoop.hbase.client.Connection;
@@ -473,4 +474,9 @@ public class MockNoopMasterServices implements MasterServices {
public SyncReplicationReplayWALManager getSyncReplicationReplayWALManager() {
return null;
}
+
+ @Override
+ public AsyncClusterConnection getAsyncClusterConnection() {
+ return null;
+ }
}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/hbase/blob/6fff6c81/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 a930d7f..73d53c7 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
@@ -41,6 +41,7 @@ import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableDescriptors;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.ZooKeeperConnectionException;
+import org.apache.hadoop.hbase.client.AsyncClusterConnection;
import org.apache.hadoop.hbase.client.ClusterConnection;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.RegionInfo;
@@ -721,4 +722,8 @@ class MockRegionServer implements AdminProtos.AdminService.BlockingInterface,
public Optional<MobFileCache> getMobFileCache() {
return Optional.empty();
}
+
+ public AsyncClusterConnection getAsyncClusterConnection() {
+ return null;
+ }
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/6fff6c81/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestActiveMasterManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestActiveMasterManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestActiveMasterManager.java
index 2300f54..77667a7 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestActiveMasterManager.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestActiveMasterManager.java
@@ -31,6 +31,7 @@ import org.apache.hadoop.hbase.HBaseClassTestRule;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.client.AsyncClusterConnection;
import org.apache.hadoop.hbase.client.ClusterConnection;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
@@ -349,5 +350,10 @@ public class TestActiveMasterManager {
public Connection createConnection(Configuration conf) throws IOException {
return null;
}
+
+ @Override
+ public AsyncClusterConnection getAsyncClusterConnection() {
+ return null;
+ }
}
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/6fff6c81/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileCleaner.java
index 5c8db3e..c5fad32 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileCleaner.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileCleaner.java
@@ -36,6 +36,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.client.AsyncClusterConnection;
import org.apache.hadoop.hbase.client.ClusterConnection;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.testclassification.MasterTests;
@@ -279,6 +280,11 @@ public class TestHFileCleaner {
public Connection createConnection(Configuration conf) throws IOException {
return null;
}
+
+ @Override
+ public AsyncClusterConnection getAsyncClusterConnection() {
+ return null;
+ }
}
@Test
http://git-wip-us.apache.org/repos/asf/hbase/blob/6fff6c81/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileLinkCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileLinkCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileLinkCleaner.java
index 119194b..fd11ff8 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileLinkCleaner.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileLinkCleaner.java
@@ -34,6 +34,7 @@ import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.AsyncClusterConnection;
import org.apache.hadoop.hbase.client.ClusterConnection;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.io.HFileLink;
@@ -213,5 +214,10 @@ public class TestHFileLinkCleaner {
public Connection createConnection(Configuration conf) throws IOException {
return null;
}
+
+ @Override
+ public AsyncClusterConnection getAsyncClusterConnection() {
+ return null;
+ }
}
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/6fff6c81/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java
index 247ed01..3286032 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java
@@ -45,6 +45,7 @@ import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.Waiter;
import org.apache.hadoop.hbase.ZooKeeperConnectionException;
+import org.apache.hadoop.hbase.client.AsyncClusterConnection;
import org.apache.hadoop.hbase.client.ClusterConnection;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.master.HMaster;
@@ -409,6 +410,11 @@ public class TestLogsCleaner {
public Connection createConnection(Configuration conf) throws IOException {
return null;
}
+
+ @Override
+ public AsyncClusterConnection getAsyncClusterConnection() {
+ return null;
+ }
}
static class FaultyZooKeeperWatcher extends ZKWatcher {
http://git-wip-us.apache.org/repos/asf/hbase/blob/6fff6c81/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationHFileCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationHFileCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationHFileCleaner.java
index d162bf3..9791643 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationHFileCleaner.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationHFileCleaner.java
@@ -41,6 +41,7 @@ import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.ZooKeeperConnectionException;
+import org.apache.hadoop.hbase.client.AsyncClusterConnection;
import org.apache.hadoop.hbase.client.ClusterConnection;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.master.HMaster;
@@ -303,6 +304,11 @@ public class TestReplicationHFileCleaner {
public Connection createConnection(Configuration conf) throws IOException {
return null;
}
+
+ @Override
+ public AsyncClusterConnection getAsyncClusterConnection() {
+ return null;
+ }
}
static class FaultyZooKeeperWatcher extends ZKWatcher {
http://git-wip-us.apache.org/repos/asf/hbase/blob/6fff6c81/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHeapMemoryManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHeapMemoryManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHeapMemoryManager.java
index 8c9ce75..4a359e4 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHeapMemoryManager.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHeapMemoryManager.java
@@ -36,6 +36,7 @@ import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.Waiter;
+import org.apache.hadoop.hbase.client.AsyncClusterConnection;
import org.apache.hadoop.hbase.client.ClusterConnection;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.io.hfile.BlockCache;
@@ -862,6 +863,11 @@ public class TestHeapMemoryManager {
public Connection createConnection(Configuration conf) throws IOException {
return null;
}
+
+ @Override
+ public AsyncClusterConnection getAsyncClusterConnection() {
+ return null;
+ }
}
static class CustomHeapMemoryTuner implements HeapMemoryTuner {
http://git-wip-us.apache.org/repos/asf/hbase/blob/6fff6c81/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitLogWorker.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitLogWorker.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitLogWorker.java
index cbf932c..5481ff8 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitLogWorker.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitLogWorker.java
@@ -41,6 +41,7 @@ import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.SplitLogCounters;
import org.apache.hadoop.hbase.SplitLogTask;
import org.apache.hadoop.hbase.Waiter;
+import org.apache.hadoop.hbase.client.AsyncClusterConnection;
import org.apache.hadoop.hbase.client.ClusterConnection;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.coordination.ZkCoordinatedStateManager;
@@ -159,6 +160,11 @@ public class TestSplitLogWorker {
public Connection createConnection(Configuration conf) throws IOException {
return null;
}
+
+ @Override
+ public AsyncClusterConnection getAsyncClusterConnection() {
+ return null;
+ }
}
private void waitForCounter(LongAdder ctr, long oldval, long newval, long timems)
http://git-wip-us.apache.org/repos/asf/hbase/blob/6fff6c81/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java
index 0e20252..9e9d1d6 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java
@@ -36,6 +36,7 @@ import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.AsyncClusterConnection;
import org.apache.hadoop.hbase.client.ClusterConnection;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.Durability;
@@ -523,6 +524,11 @@ public class TestWALLockup {
public Connection createConnection(Configuration conf) throws IOException {
return null;
}
+
+ @Override
+ public AsyncClusterConnection getAsyncClusterConnection() {
+ return null;
+ }
}
static class DummyWALActionsListener implements WALActionsListener {
http://git-wip-us.apache.org/repos/asf/hbase/blob/6fff6c81/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationTrackerZKImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationTrackerZKImpl.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationTrackerZKImpl.java
index 863d558..62ab265 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationTrackerZKImpl.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationTrackerZKImpl.java
@@ -32,6 +32,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.client.AsyncClusterConnection;
import org.apache.hadoop.hbase.client.ClusterConnection;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.testclassification.MediumTests;
@@ -263,5 +264,10 @@ public class TestReplicationTrackerZKImpl {
public Connection createConnection(Configuration conf) throws IOException {
return null;
}
+
+ @Override
+ public AsyncClusterConnection getAsyncClusterConnection() {
+ return null;
+ }
}
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/6fff6c81/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java
index 86bbb09..427f319 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java
@@ -57,6 +57,7 @@ import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.Waiter;
+import org.apache.hadoop.hbase.client.AsyncClusterConnection;
import org.apache.hadoop.hbase.client.ClusterConnection;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
import org.apache.hadoop.hbase.client.Connection;
@@ -906,5 +907,10 @@ public abstract class TestReplicationSourceManager {
public Connection createConnection(Configuration conf) throws IOException {
return null;
}
+
+ @Override
+ public AsyncClusterConnection getAsyncClusterConnection() {
+ return null;
+ }
}
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/6fff6c81/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenAuthentication.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenAuthentication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenAuthentication.java
index e4780f1..92c8e54 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenAuthentication.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenAuthentication.java
@@ -44,6 +44,7 @@ import org.apache.hadoop.hbase.HBaseClassTestRule;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.client.AsyncClusterConnection;
import org.apache.hadoop.hbase.client.ClusterConnection;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory;
@@ -363,6 +364,11 @@ public class TestTokenAuthentication {
public Connection createConnection(Configuration conf) throws IOException {
return null;
}
+
+ @Override
+ public AsyncClusterConnection getAsyncClusterConnection() {
+ return null;
+ }
}
@Parameters(name = "{index}: rpcServerImpl={0}")
http://git-wip-us.apache.org/repos/asf/hbase/blob/6fff6c81/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MockServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MockServer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MockServer.java
index c25db01..13212d2 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MockServer.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MockServer.java
@@ -26,6 +26,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.ZooKeeperConnectionException;
+import org.apache.hadoop.hbase.client.AsyncClusterConnection;
import org.apache.hadoop.hbase.client.ClusterConnection;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.log.HBaseMarkers;
@@ -143,4 +144,9 @@ public class MockServer implements Server {
public Connection createConnection(Configuration conf) throws IOException {
return null;
}
+
+ @Override
+ public AsyncClusterConnection getAsyncClusterConnection() {
+ return null;
+ }
}
[13/15] hbase git commit: HBASE-21516 Use AsyncConnection instead of
Connection in SecureBulkLoadManager
Posted by zh...@apache.org.
HBASE-21516 Use AsyncConnection instead of Connection in SecureBulkLoadManager
Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/721af37c
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/721af37c
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/721af37c
Branch: refs/heads/HBASE-21512
Commit: 721af37c0ce2777620334cfd6fbb14a430533704
Parents: 6fff6c8
Author: zhangduo <zh...@apache.org>
Authored: Sat Dec 1 21:15:48 2018 +0800
Committer: Duo Zhang <zh...@apache.org>
Committed: Fri Jan 4 14:31:41 2019 +0800
----------------------------------------------------------------------
.../hadoop/hbase/protobuf/ProtobufUtil.java | 5 +-
.../hbase/shaded/protobuf/ProtobufUtil.java | 7 ++-
.../hbase/regionserver/HRegionServer.java | 2 +-
.../regionserver/SecureBulkLoadManager.java | 24 +++++----
.../hadoop/hbase/security/token/TokenUtil.java | 57 +++++++++++++++-----
.../hbase/security/token/TestTokenUtil.java | 42 +++++++++++----
6 files changed, 96 insertions(+), 41 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/hbase/blob/721af37c/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
index a3d49b5..d9e620b 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
@@ -261,13 +261,12 @@ public final class ProtobufUtil {
* just {@link ServiceException}. Prefer this method to
* {@link #getRemoteException(ServiceException)} because trying to
* contain direct protobuf references.
- * @param e
*/
- public static IOException handleRemoteException(Exception e) {
+ public static IOException handleRemoteException(Throwable e) {
return makeIOExceptionOfException(e);
}
- private static IOException makeIOExceptionOfException(Exception e) {
+ private static IOException makeIOExceptionOfException(Throwable e) {
Throwable t = e;
if (e instanceof ServiceException ||
e instanceof org.apache.hbase.thirdparty.com.google.protobuf.ServiceException) {
http://git-wip-us.apache.org/repos/asf/hbase/blob/721af37c/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 fea81f1..de2fb7d 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
@@ -40,7 +40,6 @@ import java.util.concurrent.TimeUnit;
import java.util.function.Function;
import java.util.regex.Pattern;
import java.util.stream.Collectors;
-
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.ByteBufferExtendedCell;
@@ -123,6 +122,7 @@ import org.apache.hbase.thirdparty.com.google.protobuf.Service;
import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;
import org.apache.hbase.thirdparty.com.google.protobuf.TextFormat;
import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations;
+
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetOnlineRegionRequest;
@@ -343,13 +343,12 @@ public final class ProtobufUtil {
* just {@link ServiceException}. Prefer this method to
* {@link #getRemoteException(ServiceException)} because trying to
* contain direct protobuf references.
- * @param e
*/
- public static IOException handleRemoteException(Exception e) {
+ public static IOException handleRemoteException(Throwable e) {
return makeIOExceptionOfException(e);
}
- private static IOException makeIOExceptionOfException(Exception e) {
+ private static IOException makeIOExceptionOfException(Throwable e) {
Throwable t = e;
if (e instanceof ServiceException) {
t = e.getCause();
http://git-wip-us.apache.org/repos/asf/hbase/blob/721af37c/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 375b3f8..6e5fc9f 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
@@ -1930,7 +1930,7 @@ public class HRegionServer extends HasThread implements
if (!isStopped() && !isAborted()) {
initializeThreads();
}
- this.secureBulkLoadManager = new SecureBulkLoadManager(this.conf, clusterConnection);
+ this.secureBulkLoadManager = new SecureBulkLoadManager(this.conf, asyncClusterConnection);
this.secureBulkLoadManager.start();
// Health checker thread.
http://git-wip-us.apache.org/repos/asf/hbase/blob/721af37c/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SecureBulkLoadManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SecureBulkLoadManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SecureBulkLoadManager.java
index 566a6b6..add6519 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SecureBulkLoadManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SecureBulkLoadManager.java
@@ -1,4 +1,4 @@
-/*
+/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
@@ -28,7 +28,6 @@ import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
import java.util.function.BiFunction;
import java.util.function.Consumer;
-
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
@@ -38,11 +37,12 @@ import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.AsyncConnection;
import org.apache.hadoop.hbase.ipc.RpcServer;
import org.apache.hadoop.hbase.regionserver.HRegion.BulkLoadListener;
import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.security.UserProvider;
+import org.apache.hadoop.hbase.security.token.AuthenticationTokenIdentifier;
import org.apache.hadoop.hbase.security.token.FsDelegationToken;
import org.apache.hadoop.hbase.security.token.TokenUtil;
import org.apache.hadoop.hbase.util.Bytes;
@@ -56,7 +56,9 @@ import org.apache.hadoop.security.token.Token;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
+
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CleanupBulkLoadRequest;
@@ -111,9 +113,9 @@ public class SecureBulkLoadManager {
private UserProvider userProvider;
private ConcurrentHashMap<UserGroupInformation, Integer> ugiReferenceCounter;
- private Connection conn;
+ private AsyncConnection conn;
- SecureBulkLoadManager(Configuration conf, Connection conn) {
+ SecureBulkLoadManager(Configuration conf, AsyncConnection conn) {
this.conf = conf;
this.conn = conn;
}
@@ -212,23 +214,23 @@ public class SecureBulkLoadManager {
familyPaths.add(new Pair<>(el.getFamily().toByteArray(), el.getPath()));
}
- Token userToken = null;
+ Token<AuthenticationTokenIdentifier> userToken = null;
if (userProvider.isHadoopSecurityEnabled()) {
- userToken = new Token(request.getFsToken().getIdentifier().toByteArray(), request.getFsToken()
- .getPassword().toByteArray(), new Text(request.getFsToken().getKind()), new Text(
- request.getFsToken().getService()));
+ userToken = new Token<>(request.getFsToken().getIdentifier().toByteArray(),
+ request.getFsToken().getPassword().toByteArray(), new Text(request.getFsToken().getKind()),
+ new Text(request.getFsToken().getService()));
}
final String bulkToken = request.getBulkToken();
User user = getActiveUser();
final UserGroupInformation ugi = user.getUGI();
if (userProvider.isHadoopSecurityEnabled()) {
try {
- Token tok = TokenUtil.obtainToken(conn);
+ Token<AuthenticationTokenIdentifier> tok = TokenUtil.obtainToken(conn).get();
if (tok != null) {
boolean b = ugi.addToken(tok);
LOG.debug("token added " + tok + " for user " + ugi + " return=" + b);
}
- } catch (IOException ioe) {
+ } catch (Exception ioe) {
LOG.warn("unable to add token", ioe);
}
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/721af37c/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/TokenUtil.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/TokenUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/TokenUtil.java
index c54d905..28efb84 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/TokenUtil.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/TokenUtil.java
@@ -1,4 +1,4 @@
-/*
+/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
@@ -15,27 +15,29 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-
package org.apache.hadoop.hbase.security.token;
+import com.google.protobuf.ByteString;
+import com.google.protobuf.ServiceException;
import java.io.IOException;
import java.lang.reflect.UndeclaredThrowableException;
import java.security.PrivilegedExceptionAction;
-
-import com.google.protobuf.ByteString;
-import com.google.protobuf.ServiceException;
-
-import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
+import java.util.concurrent.CompletableFuture;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.AsyncConnection;
+import org.apache.hadoop.hbase.client.AsyncTable;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
import org.apache.hadoop.hbase.protobuf.generated.AuthenticationProtos;
+import org.apache.hadoop.hbase.protobuf.generated.AuthenticationProtos.AuthenticationService;
+import org.apache.hadoop.hbase.protobuf.generated.AuthenticationProtos.GetAuthenticationTokenRequest;
+import org.apache.hadoop.hbase.protobuf.generated.AuthenticationProtos.GetAuthenticationTokenResponse;
import org.apache.hadoop.hbase.security.User;
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.mapred.JobConf;
import org.apache.hadoop.mapreduce.Job;
@@ -45,6 +47,8 @@ import org.apache.zookeeper.KeeperException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+
/**
* Utility methods for obtaining authentication tokens.
*/
@@ -64,12 +68,39 @@ public class TokenUtil {
/**
* Obtain and return an authentication token for the current user.
+ * @param conn The async HBase cluster connection
+ * @return the authentication token instance, wrapped by a {@link CompletableFuture}.
+ */
+ public static CompletableFuture<Token<AuthenticationTokenIdentifier>> obtainToken(
+ AsyncConnection conn) {
+ CompletableFuture<Token<AuthenticationTokenIdentifier>> future = new CompletableFuture<>();
+ if (injectedException != null) {
+ future.completeExceptionally(injectedException);
+ return future;
+ }
+ AsyncTable<?> table = conn.getTable(TableName.META_TABLE_NAME);
+ table.<AuthenticationService.Interface, GetAuthenticationTokenResponse> coprocessorService(
+ AuthenticationProtos.AuthenticationService::newStub,
+ (s, c, r) -> s.getAuthenticationToken(c,
+ AuthenticationProtos.GetAuthenticationTokenRequest.getDefaultInstance(), r),
+ HConstants.EMPTY_START_ROW).whenComplete((resp, error) -> {
+ if (error != null) {
+ future.completeExceptionally(ProtobufUtil.handleRemoteException(error));
+ } else {
+ future.complete(toToken(resp.getToken()));
+ }
+ });
+ return future;
+ }
+
+ /**
+ * Obtain and return an authentication token for the current user.
* @param conn The HBase cluster connection
* @throws IOException if a remote error or serialization problem occurs.
* @return the authentication token instance
*/
- public static Token<AuthenticationTokenIdentifier> obtainToken(
- Connection conn) throws IOException {
+ public static Token<AuthenticationTokenIdentifier> obtainToken(Connection conn)
+ throws IOException {
Table meta = null;
try {
injectFault();
@@ -77,9 +108,9 @@ public class TokenUtil {
meta = conn.getTable(TableName.META_TABLE_NAME);
CoprocessorRpcChannel rpcChannel = meta.coprocessorService(HConstants.EMPTY_START_ROW);
AuthenticationProtos.AuthenticationService.BlockingInterface service =
- AuthenticationProtos.AuthenticationService.newBlockingStub(rpcChannel);
- AuthenticationProtos.GetAuthenticationTokenResponse response = service.getAuthenticationToken(null,
- AuthenticationProtos.GetAuthenticationTokenRequest.getDefaultInstance());
+ AuthenticationService.newBlockingStub(rpcChannel);
+ GetAuthenticationTokenResponse response =
+ service.getAuthenticationToken(null, GetAuthenticationTokenRequest.getDefaultInstance());
return toToken(response.getToken());
} catch (ServiceException se) {
http://git-wip-us.apache.org/repos/asf/hbase/blob/721af37c/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenUtil.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenUtil.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenUtil.java
index 32fcddb..585a3ec 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenUtil.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenUtil.java
@@ -18,35 +18,53 @@
package org.apache.hadoop.hbase.security.token;
import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertSame;
import static org.junit.Assert.fail;
+import java.io.IOException;
import java.lang.reflect.Field;
import java.lang.reflect.InvocationTargetException;
import java.net.URL;
import java.net.URLClassLoader;
-
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.client.AsyncConnection;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.junit.After;
+import org.junit.Before;
import org.junit.ClassRule;
import org.junit.Test;
import org.junit.experimental.categories.Category;
+import org.apache.hbase.thirdparty.com.google.common.io.Closeables;
+
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
@Category(SmallTests.class)
public class TestTokenUtil {
+
@ClassRule
public static final HBaseClassTestRule CLASS_RULE =
- HBaseClassTestRule.forClass(TestTokenUtil.class);
+ HBaseClassTestRule.forClass(TestTokenUtil.class);
- @Test
- public void testObtainToken() throws Exception {
+ private URLClassLoader cl;
+
+ @Before
+ public void setUp() {
URL urlPU = ProtobufUtil.class.getProtectionDomain().getCodeSource().getLocation();
URL urlTU = TokenUtil.class.getProtectionDomain().getCodeSource().getLocation();
+ cl = new URLClassLoader(new URL[] { urlPU, urlTU }, getClass().getClassLoader());
+ }
- ClassLoader cl = new URLClassLoader(new URL[] { urlPU, urlTU }, getClass().getClassLoader());
+ @After
+ public void tearDown() throws IOException {
+ Closeables.close(cl, true);
+ }
+ @Test
+ public void testObtainToken() throws Exception {
Throwable injected = new com.google.protobuf.ServiceException("injected");
Class<?> tokenUtil = cl.loadClass(TokenUtil.class.getCanonicalName());
@@ -55,8 +73,7 @@ public class TestTokenUtil {
shouldInjectFault.set(null, injected);
try {
- tokenUtil.getMethod("obtainToken", Connection.class)
- .invoke(null, new Object[] { null });
+ tokenUtil.getMethod("obtainToken", Connection.class).invoke(null, new Object[] { null });
fail("Should have injected exception.");
} catch (InvocationTargetException e) {
Throwable t = e;
@@ -72,9 +89,16 @@ public class TestTokenUtil {
}
}
+ CompletableFuture<?> future = (CompletableFuture<?>) tokenUtil
+ .getMethod("obtainToken", AsyncConnection.class).invoke(null, new Object[] { null });
+ try {
+ future.get();
+ fail("Should have injected exception.");
+ } catch (ExecutionException e) {
+ assertSame(injected, e.getCause());
+ }
Boolean loaded = (Boolean) cl.loadClass(ProtobufUtil.class.getCanonicalName())
- .getDeclaredMethod("isClassLoaderLoaded")
- .invoke(null);
+ .getDeclaredMethod("isClassLoaderLoaded").invoke(null);
assertFalse("Should not have loaded DynamicClassLoader", loaded);
}
}
[02/15] hbase git commit: HBASE-21652 Refactor ThriftServer making
thrift2 server inherited from thrift1 server
Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/e4b6b4af/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftServer.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftServer.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftServer.java
index 5681569..fa3d39d 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftServer.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftServer.java
@@ -18,355 +18,86 @@
*/
package org.apache.hadoop.hbase.thrift2;
-import java.io.IOException;
-import java.net.InetAddress;
-import java.net.InetSocketAddress;
-import java.net.UnknownHostException;
-import java.security.PrivilegedAction;
-import java.util.Map;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.SynchronousQueue;
-import java.util.concurrent.ThreadPoolExecutor;
-import java.util.concurrent.TimeUnit;
+import static org.apache.hadoop.hbase.thrift.Constants.READONLY_OPTION;
+import static org.apache.hadoop.hbase.thrift.Constants.THRIFT_READONLY_ENABLED;
+import static org.apache.hadoop.hbase.thrift.Constants.THRIFT_READONLY_ENABLED_DEFAULT;
-import javax.security.auth.callback.Callback;
-import javax.security.auth.callback.UnsupportedCallbackException;
-import javax.security.sasl.AuthorizeCallback;
-import javax.security.sasl.SaslServer;
+import java.io.IOException;
import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.conf.Configured;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
-import org.apache.hadoop.hbase.filter.ParseFilter;
-import org.apache.hadoop.hbase.http.InfoServer;
-import org.apache.hadoop.hbase.security.SaslUtil;
-import org.apache.hadoop.hbase.security.SecurityUtil;
import org.apache.hadoop.hbase.security.UserProvider;
-import org.apache.hadoop.hbase.thrift.CallQueue;
-import org.apache.hadoop.hbase.thrift.THBaseThreadPoolExecutor;
-import org.apache.hadoop.hbase.thrift.ThriftMetrics;
+import org.apache.hadoop.hbase.thrift.HBaseServiceHandler;
+import org.apache.hadoop.hbase.thrift.HbaseHandlerMetricsProxy;
import org.apache.hadoop.hbase.thrift2.generated.THBaseService;
-import org.apache.hadoop.hbase.util.DNS;
-import org.apache.hadoop.hbase.util.JvmPauseMonitor;
-import org.apache.hadoop.hbase.util.Strings;
-import org.apache.hadoop.security.SaslRpcServer.SaslGssCallbackHandler;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.Shell;
import org.apache.hadoop.util.ToolRunner;
-import org.apache.thrift.TException;
import org.apache.thrift.TProcessor;
-import org.apache.thrift.protocol.TBinaryProtocol;
-import org.apache.thrift.protocol.TCompactProtocol;
-import org.apache.thrift.protocol.TProtocol;
-import org.apache.thrift.protocol.TProtocolFactory;
-import org.apache.thrift.server.THsHaServer;
-import org.apache.thrift.server.TNonblockingServer;
-import org.apache.thrift.server.TServer;
-import org.apache.thrift.server.TThreadPoolServer;
-import org.apache.thrift.server.TThreadedSelectorServer;
-import org.apache.thrift.transport.TFramedTransport;
-import org.apache.thrift.transport.TNonblockingServerSocket;
-import org.apache.thrift.transport.TNonblockingServerTransport;
-import org.apache.thrift.transport.TSaslServerTransport;
-import org.apache.thrift.transport.TServerSocket;
-import org.apache.thrift.transport.TServerTransport;
-import org.apache.thrift.transport.TTransportException;
-import org.apache.thrift.transport.TTransportFactory;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
+
import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLine;
-import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLineParser;
-import org.apache.hbase.thirdparty.org.apache.commons.cli.DefaultParser;
import org.apache.hbase.thirdparty.org.apache.commons.cli.HelpFormatter;
-import org.apache.hbase.thirdparty.org.apache.commons.cli.Option;
-import org.apache.hbase.thirdparty.org.apache.commons.cli.OptionGroup;
import org.apache.hbase.thirdparty.org.apache.commons.cli.Options;
-import org.apache.hbase.thirdparty.org.apache.commons.cli.ParseException;
/**
* ThriftServer - this class starts up a Thrift server which implements the HBase API specified in
* the HbaseClient.thrift IDL file.
*/
+@edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "NM_SAME_SIMPLE_NAME_AS_SUPERCLASS",
+ justification = "Change the name will be an incompatible change, will do it later")
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.TOOLS)
@SuppressWarnings({ "rawtypes", "unchecked" })
-public class ThriftServer extends Configured implements Tool {
+public class ThriftServer extends org.apache.hadoop.hbase.thrift.ThriftServer {
private static final Logger log = LoggerFactory.getLogger(ThriftServer.class);
- /**
- * Thrift quality of protection configuration key. Valid values can be:
- * privacy: authentication, integrity and confidentiality checking
- * integrity: authentication and integrity checking
- * authentication: authentication only
- *
- * This is used to authenticate the callers and support impersonation.
- * The thrift server and the HBase cluster must run in secure mode.
- */
- static final String THRIFT_QOP_KEY = "hbase.thrift.security.qop";
-
- static final String BACKLOG_CONF_KEY = "hbase.regionserver.thrift.backlog";
- public static final int DEFAULT_LISTEN_PORT = 9090;
-
- private static final String READ_TIMEOUT_OPTION = "readTimeout";
-
- /**
- * Amount of time in milliseconds before a server thread will timeout
- * waiting for client to send data on a connected socket. Currently,
- * applies only to TBoundedThreadPoolServer
- */
- public static final String THRIFT_SERVER_SOCKET_READ_TIMEOUT_KEY =
- "hbase.thrift.server.socket.read.timeout";
- public static final int THRIFT_SERVER_SOCKET_READ_TIMEOUT_DEFAULT = 60000;
-
- public ThriftServer() {
+ public ThriftServer(Configuration conf) {
+ super(conf);
}
- private static void printUsage() {
+ @Override
+ protected void printUsageAndExit(Options options, int exitCode)
+ throws Shell.ExitCodeException {
HelpFormatter formatter = new HelpFormatter();
- formatter.printHelp("Thrift", null, getOptions(),
+ formatter.printHelp("Thrift", null, options,
"To start the Thrift server run 'hbase-daemon.sh start thrift2' or " +
- "'hbase thrift2'\n" +
- "To shutdown the thrift server run 'hbase-daemon.sh stop thrift2' or" +
- " send a kill signal to the thrift server pid",
+ "'hbase thrift2'\n" +
+ "To shutdown the thrift server run 'hbase-daemon.sh stop thrift2' or" +
+ " send a kill signal to the thrift server pid",
true);
+ throw new Shell.ExitCodeException(exitCode, "");
}
- private static Options getOptions() {
- Options options = new Options();
- options.addOption("b", "bind", true,
- "Address to bind the Thrift server to. [default: 0.0.0.0]");
- options.addOption("p", "port", true, "Port to bind to [default: " + DEFAULT_LISTEN_PORT + "]");
- options.addOption("f", "framed", false, "Use framed transport");
- options.addOption("c", "compact", false, "Use the compact protocol");
- options.addOption("w", "workers", true, "How many worker threads to use.");
- options.addOption("s", "selectors", true, "How many selector threads to use.");
- options.addOption("q", "callQueueSize", true,
- "Max size of request queue (unbounded by default)");
- options.addOption("h", "help", false, "Print help information");
- options.addOption(null, "infoport", true, "Port for web UI");
- options.addOption("t", READ_TIMEOUT_OPTION, true,
- "Amount of time in milliseconds before a server thread will timeout " +
- "waiting for client to send data on a connected socket. Currently, " +
- "only applies to TBoundedThreadPoolServer");
- options.addOption("ro", "readonly", false,
- "Respond only to read method requests [default: false]");
- OptionGroup servers = new OptionGroup();
- servers.addOption(new Option("nonblocking", false,
- "Use the TNonblockingServer. This implies the framed transport."));
- servers.addOption(new Option("hsha", false,
- "Use the THsHaServer. This implies the framed transport."));
- servers.addOption(new Option("selector", false,
- "Use the TThreadedSelectorServer. This implies the framed transport."));
- servers.addOption(new Option("threadpool", false,
- "Use the TThreadPoolServer. This is the default."));
- options.addOptionGroup(servers);
- return options;
- }
-
- private static CommandLine parseArguments(Configuration conf, Options options, String[] args)
- throws ParseException, IOException {
- CommandLineParser parser = new DefaultParser();
- return parser.parse(options, args);
- }
-
- private static TProtocolFactory getTProtocolFactory(boolean isCompact) {
- if (isCompact) {
- log.debug("Using compact protocol");
- return new TCompactProtocol.Factory();
- } else {
- log.debug("Using binary protocol");
- return new TBinaryProtocol.Factory();
- }
- }
-
- private static TTransportFactory getTTransportFactory(
- SaslUtil.QualityOfProtection qop, String name, String host,
- boolean framed, int frameSize) {
- if (framed) {
- if (qop != null) {
- throw new RuntimeException("Thrift server authentication"
- + " doesn't work with framed transport yet");
- }
- log.debug("Using framed transport");
- return new TFramedTransport.Factory(frameSize);
- } else if (qop == null) {
- return new TTransportFactory();
- } else {
- Map<String, String> saslProperties = SaslUtil.initSaslProperties(qop.name());
- TSaslServerTransport.Factory saslFactory = new TSaslServerTransport.Factory();
- saslFactory.addServerDefinition("GSSAPI", name, host, saslProperties,
- new SaslGssCallbackHandler() {
- @Override
- public void handle(Callback[] callbacks)
- throws UnsupportedCallbackException {
- AuthorizeCallback ac = null;
- for (Callback callback : callbacks) {
- if (callback instanceof AuthorizeCallback) {
- ac = (AuthorizeCallback) callback;
- } else {
- throw new UnsupportedCallbackException(callback,
- "Unrecognized SASL GSSAPI Callback");
- }
- }
- if (ac != null) {
- String authid = ac.getAuthenticationID();
- String authzid = ac.getAuthorizationID();
- if (!authid.equals(authzid)) {
- ac.setAuthorized(false);
- } else {
- ac.setAuthorized(true);
- String userName = SecurityUtil.getUserFromPrincipal(authzid);
- log.info("Effective user: " + userName);
- ac.setAuthorizedID(userName);
- }
- }
- }
- });
- return saslFactory;
- }
- }
-
- /*
- * If bindValue is null, we don't bind.
- */
- private static InetSocketAddress bindToPort(String bindValue, int listenPort)
- throws UnknownHostException {
- try {
- if (bindValue == null) {
- return new InetSocketAddress(listenPort);
- } else {
- return new InetSocketAddress(InetAddress.getByName(bindValue), listenPort);
- }
- } catch (UnknownHostException e) {
- throw new RuntimeException("Could not bind to provided ip address", e);
- }
- }
-
- private static TServer getTNonBlockingServer(TProtocolFactory protocolFactory,
- TProcessor processor, TTransportFactory transportFactory, InetSocketAddress inetSocketAddress)
- throws TTransportException {
- TNonblockingServerTransport serverTransport = new TNonblockingServerSocket(inetSocketAddress);
- log.info("starting HBase Nonblocking Thrift server on " + inetSocketAddress.toString());
- TNonblockingServer.Args serverArgs = new TNonblockingServer.Args(serverTransport);
- serverArgs.processor(processor);
- serverArgs.transportFactory(transportFactory);
- serverArgs.protocolFactory(protocolFactory);
- return new TNonblockingServer(serverArgs);
- }
-
- private static TServer getTHsHaServer(TProtocolFactory protocolFactory,
- TProcessor processor, TTransportFactory transportFactory,
- int workerThreads, int maxCallQueueSize,
- InetSocketAddress inetSocketAddress, ThriftMetrics metrics)
- throws TTransportException {
- TNonblockingServerTransport serverTransport = new TNonblockingServerSocket(inetSocketAddress);
- log.info("starting HBase HsHA Thrift server on " + inetSocketAddress.toString());
- THsHaServer.Args serverArgs = new THsHaServer.Args(serverTransport);
- if (workerThreads > 0) {
- // Could support the min & max threads, avoiding to preserve existing functionality.
- serverArgs.minWorkerThreads(workerThreads).maxWorkerThreads(workerThreads);
- }
- ExecutorService executorService = createExecutor(
- workerThreads, maxCallQueueSize, metrics);
- serverArgs.executorService(executorService);
- serverArgs.processor(processor);
- serverArgs.transportFactory(transportFactory);
- serverArgs.protocolFactory(protocolFactory);
- return new THsHaServer(serverArgs);
- }
-
- private static TServer getTThreadedSelectorServer(TProtocolFactory protocolFactory,
- TProcessor processor, TTransportFactory transportFactory,
- int workerThreads, int selectorThreads, int maxCallQueueSize,
- InetSocketAddress inetSocketAddress, ThriftMetrics metrics)
- throws TTransportException {
- TNonblockingServerTransport serverTransport = new TNonblockingServerSocket(inetSocketAddress);
- log.info("starting HBase ThreadedSelector Thrift server on " + inetSocketAddress.toString());
- TThreadedSelectorServer.Args serverArgs = new TThreadedSelectorServer.Args(serverTransport);
- if (workerThreads > 0) {
- serverArgs.workerThreads(workerThreads);
- }
- if (selectorThreads > 0) {
- serverArgs.selectorThreads(selectorThreads);
- }
-
- ExecutorService executorService = createExecutor(
- workerThreads, maxCallQueueSize, metrics);
- serverArgs.executorService(executorService);
- serverArgs.processor(processor);
- serverArgs.transportFactory(transportFactory);
- serverArgs.protocolFactory(protocolFactory);
- return new TThreadedSelectorServer(serverArgs);
+ @Override
+ protected HBaseServiceHandler createHandler(Configuration conf, UserProvider userProvider)
+ throws IOException {
+ return new ThriftHBaseServiceHandler(conf, userProvider);
}
- private static ExecutorService createExecutor(
- int workerThreads, int maxCallQueueSize, ThriftMetrics metrics) {
- CallQueue callQueue;
- if (maxCallQueueSize > 0) {
- callQueue = new CallQueue(new LinkedBlockingQueue<>(maxCallQueueSize), metrics);
- } else {
- callQueue = new CallQueue(new LinkedBlockingQueue<>(), metrics);
- }
-
- ThreadFactoryBuilder tfb = new ThreadFactoryBuilder();
- tfb.setDaemon(true);
- tfb.setNameFormat("thrift2-worker-%d");
- ThreadPoolExecutor pool = new THBaseThreadPoolExecutor(workerThreads, workerThreads,
- Long.MAX_VALUE, TimeUnit.SECONDS, callQueue, tfb.build(), metrics);
- pool.prestartAllCoreThreads();
- return pool;
+ @Override
+ protected TProcessor createProcessor() {
+ return new THBaseService.Processor<>(HbaseHandlerMetricsProxy
+ .newInstance((THBaseService.Iface) hbaseServiceHandler, metrics, conf));
}
- private static TServer getTThreadPoolServer(TProtocolFactory protocolFactory,
- TProcessor processor,
- TTransportFactory transportFactory,
- int workerThreads,
- InetSocketAddress inetSocketAddress,
- int backlog,
- int clientTimeout,
- ThriftMetrics metrics)
- throws TTransportException {
- TServerTransport serverTransport = new TServerSocket(
- new TServerSocket.ServerSocketTransportArgs().
- bindAddr(inetSocketAddress).backlog(backlog).
- clientTimeout(clientTimeout));
- log.info("starting HBase ThreadPool Thrift server on " + inetSocketAddress.toString());
- TThreadPoolServer.Args serverArgs = new TThreadPoolServer.Args(serverTransport);
- serverArgs.processor(processor);
- serverArgs.transportFactory(transportFactory);
- serverArgs.protocolFactory(protocolFactory);
- if (workerThreads > 0) {
- serverArgs.maxWorkerThreads(workerThreads);
- }
- ThreadPoolExecutor executor = new THBaseThreadPoolExecutor(serverArgs.minWorkerThreads,
- serverArgs.maxWorkerThreads, serverArgs.stopTimeoutVal, TimeUnit.SECONDS,
- new SynchronousQueue<>(), metrics);
- serverArgs.executorService(executor);
-
- return new TThreadPoolServer(serverArgs);
+ @Override
+ protected void addOptions(Options options) {
+ super.addOptions(options);
+ options.addOption("ro", READONLY_OPTION, false,
+ "Respond only to read method requests [default: false]");
}
- /**
- * Adds the option to pre-load filters at startup.
- *
- * @param conf The current configuration instance.
- */
- protected static void registerFilters(Configuration conf) {
- String[] filters = conf.getStrings("hbase.thrift.filters");
- if(filters != null) {
- for(String filterClass: filters) {
- String[] filterPart = filterClass.split(":");
- if(filterPart.length != 2) {
- log.warn("Invalid filter specification " + filterClass + " - skipping");
- } else {
- ParseFilter.registerFilter(filterPart[0], filterPart[1]);
- }
- }
+ @Override
+ protected void parseCommandLine(CommandLine cmd, Options options) throws Shell.ExitCodeException {
+ super.parseCommandLine(cmd, options);
+ boolean readOnly = THRIFT_READONLY_ENABLED_DEFAULT;
+ if (cmd.hasOption(READONLY_OPTION)) {
+ readOnly = true;
}
+ conf.setBoolean(THRIFT_READONLY_ENABLED, readOnly);
}
/**
@@ -375,249 +106,8 @@ public class ThriftServer extends Configured implements Tool {
public static void main(String[] args) throws Exception {
final Configuration conf = HBaseConfiguration.create();
// for now, only time we return is on an argument error.
- final int status = ToolRunner.run(conf, new ThriftServer(), args);
+ final int status = ToolRunner.run(conf, new ThriftServer(conf), args);
System.exit(status);
}
- @Override
- public int run(String[] args) throws Exception {
- final Configuration conf = getConf();
- Options options = getOptions();
- CommandLine cmd = parseArguments(conf, options, args);
- int workerThreads = 0;
- int selectorThreads = 0;
- int maxCallQueueSize = -1; // use unbounded queue by default
-
- if (cmd.hasOption("help")) {
- printUsage();
- return 1;
- }
-
- // Get address to bind
- String bindAddress = getBindAddress(conf, cmd);
-
- // check if server should only process read requests, if so override the conf
- if (cmd.hasOption("readonly")) {
- conf.setBoolean("hbase.thrift.readonly", true);
- if (log.isDebugEnabled()) {
- log.debug("readonly set to true");
- }
- }
-
- // Get read timeout
- int readTimeout = getReadTimeout(conf, cmd);
- // Get port to bind to
- int listenPort = getListenPort(conf, cmd);
- // Thrift's implementation uses '0' as a placeholder for 'use the default.'
- int backlog = conf.getInt(BACKLOG_CONF_KEY, 0);
-
- // Local hostname and user name, used only if QOP is configured.
- String host = null;
- String name = null;
-
- UserProvider userProvider = UserProvider.instantiate(conf);
- // login the server principal (if using secure Hadoop)
- boolean securityEnabled = userProvider.isHadoopSecurityEnabled()
- && userProvider.isHBaseSecurityEnabled();
- if (securityEnabled) {
- host = Strings.domainNamePointerToHostName(DNS.getDefaultHost(
- conf.get("hbase.thrift.dns.interface", "default"),
- conf.get("hbase.thrift.dns.nameserver", "default")));
- userProvider.login("hbase.thrift.keytab.file", "hbase.thrift.kerberos.principal", host);
- }
-
- UserGroupInformation realUser = userProvider.getCurrent().getUGI();
- String stringQop = conf.get(THRIFT_QOP_KEY);
- SaslUtil.QualityOfProtection qop = null;
- if (stringQop != null) {
- qop = SaslUtil.getQop(stringQop);
- if (!securityEnabled) {
- throw new IOException("Thrift server must run in secure mode to support authentication");
- }
- // Extract the name from the principal
- name = SecurityUtil.getUserFromPrincipal(conf.get("hbase.thrift.kerberos.principal"));
- }
-
- boolean nonblocking = cmd.hasOption("nonblocking");
- boolean hsha = cmd.hasOption("hsha");
- boolean selector = cmd.hasOption("selector");
-
- ThriftMetrics metrics = new ThriftMetrics(conf, ThriftMetrics.ThriftServerType.TWO);
- final JvmPauseMonitor pauseMonitor = new JvmPauseMonitor(conf, metrics.getSource());
-
- String implType = getImplType(nonblocking, hsha, selector);
-
- conf.set("hbase.regionserver.thrift.server.type", implType);
- conf.setInt("hbase.regionserver.thrift.port", listenPort);
- registerFilters(conf);
-
- // Construct correct ProtocolFactory
- boolean compact = cmd.hasOption("compact") ||
- conf.getBoolean("hbase.regionserver.thrift.compact", false);
- TProtocolFactory protocolFactory = getTProtocolFactory(compact);
- final ThriftHBaseServiceHandler hbaseHandler =
- new ThriftHBaseServiceHandler(conf, userProvider);
- THBaseService.Iface handler =
- ThriftHBaseServiceHandler.newInstance(hbaseHandler, metrics);
- final THBaseService.Processor p = new THBaseService.Processor(handler);
- conf.setBoolean("hbase.regionserver.thrift.compact", compact);
- TProcessor processor = p;
-
- boolean framed = cmd.hasOption("framed") ||
- conf.getBoolean("hbase.regionserver.thrift.framed", false) || nonblocking || hsha;
- TTransportFactory transportFactory = getTTransportFactory(qop, name, host, framed,
- conf.getInt("hbase.regionserver.thrift.framed.max_frame_size_in_mb", 2) * 1024 * 1024);
- InetSocketAddress inetSocketAddress = bindToPort(bindAddress, listenPort);
- conf.setBoolean("hbase.regionserver.thrift.framed", framed);
- if (qop != null) {
- // Create a processor wrapper, to get the caller
- processor = new TProcessor() {
- @Override
- public boolean process(TProtocol inProt,
- TProtocol outProt) throws TException {
- TSaslServerTransport saslServerTransport =
- (TSaslServerTransport)inProt.getTransport();
- SaslServer saslServer = saslServerTransport.getSaslServer();
- String principal = saslServer.getAuthorizationID();
- hbaseHandler.setEffectiveUser(principal);
- return p.process(inProt, outProt);
- }
- };
- }
-
- if (cmd.hasOption("w")) {
- workerThreads = Integer.parseInt(cmd.getOptionValue("w"));
- }
- if (cmd.hasOption("s")) {
- selectorThreads = Integer.parseInt(cmd.getOptionValue("s"));
- }
- if (cmd.hasOption("q")) {
- maxCallQueueSize = Integer.parseInt(cmd.getOptionValue("q"));
- }
-
- // check for user-defined info server port setting, if so override the conf
- try {
- if (cmd.hasOption("infoport")) {
- String val = cmd.getOptionValue("infoport");
- conf.setInt("hbase.thrift.info.port", Integer.parseInt(val));
- log.debug("Web UI port set to " + val);
- }
- } catch (NumberFormatException e) {
- log.error("Could not parse the value provided for the infoport option", e);
- printUsage();
- System.exit(1);
- }
-
- // Put up info server.
- startInfoServer(conf);
-
- final TServer tserver = getServer(workerThreads, selectorThreads, maxCallQueueSize, readTimeout,
- backlog, nonblocking, hsha, selector, metrics, protocolFactory, processor,
- transportFactory, inetSocketAddress);
-
- realUser.doAs(
- new PrivilegedAction<Object>() {
- @Override
- public Object run() {
- pauseMonitor.start();
- try {
- tserver.serve();
- return null;
- } finally {
- pauseMonitor.stop();
- }
- }
- });
- // when tserver.stop eventually happens we'll get here.
- return 0;
- }
-
- private String getImplType(boolean nonblocking, boolean hsha, boolean selector) {
- String implType = "threadpool";
-
- if (nonblocking) {
- implType = "nonblocking";
- } else if (hsha) {
- implType = "hsha";
- } else if (selector) {
- implType = "selector";
- }
-
- return implType;
- }
-
- private String getBindAddress(Configuration conf, CommandLine cmd) {
- String bindAddress;
- if (cmd.hasOption("bind")) {
- bindAddress = cmd.getOptionValue("bind");
- conf.set("hbase.thrift.info.bindAddress", bindAddress);
- } else {
- bindAddress = conf.get("hbase.thrift.info.bindAddress");
- }
- return bindAddress;
- }
-
- private int getListenPort(Configuration conf, CommandLine cmd) {
- int listenPort;
- try {
- if (cmd.hasOption("port")) {
- listenPort = Integer.parseInt(cmd.getOptionValue("port"));
- } else {
- listenPort = conf.getInt("hbase.regionserver.thrift.port", DEFAULT_LISTEN_PORT);
- }
- } catch (NumberFormatException e) {
- throw new RuntimeException("Could not parse the value provided for the port option", e);
- }
- return listenPort;
- }
-
- private int getReadTimeout(Configuration conf, CommandLine cmd) {
- int readTimeout;
- if (cmd.hasOption(READ_TIMEOUT_OPTION)) {
- try {
- readTimeout = Integer.parseInt(cmd.getOptionValue(READ_TIMEOUT_OPTION));
- } catch (NumberFormatException e) {
- throw new RuntimeException("Could not parse the value provided for the timeout option", e);
- }
- } else {
- readTimeout = conf.getInt(THRIFT_SERVER_SOCKET_READ_TIMEOUT_KEY,
- THRIFT_SERVER_SOCKET_READ_TIMEOUT_DEFAULT);
- }
- return readTimeout;
- }
-
- private void startInfoServer(Configuration conf) throws IOException {
- int port = conf.getInt("hbase.thrift.info.port", 9095);
-
- if (port >= 0) {
- conf.setLong("startcode", System.currentTimeMillis());
- String a = conf.get("hbase.thrift.info.bindAddress", "0.0.0.0");
- InfoServer infoServer = new InfoServer("thrift", a, port, false, conf);
- infoServer.setAttribute("hbase.conf", conf);
- infoServer.start();
- }
- }
-
- private TServer getServer(int workerThreads, int selectorThreads, int maxCallQueueSize,
- int readTimeout, int backlog, boolean nonblocking, boolean hsha, boolean selector,
- ThriftMetrics metrics, TProtocolFactory protocolFactory, TProcessor processor,
- TTransportFactory transportFactory, InetSocketAddress inetSocketAddress)
- throws TTransportException {
- TServer server;
-
- if (nonblocking) {
- server = getTNonBlockingServer(protocolFactory, processor, transportFactory,
- inetSocketAddress);
- } else if (hsha) {
- server = getTHsHaServer(protocolFactory, processor, transportFactory, workerThreads,
- maxCallQueueSize, inetSocketAddress, metrics);
- } else if (selector) {
- server = getTThreadedSelectorServer(protocolFactory, processor, transportFactory,
- workerThreads, selectorThreads, maxCallQueueSize, inetSocketAddress, metrics);
- } else {
- server = getTThreadPoolServer(protocolFactory, processor, transportFactory, workerThreads,
- inetSocketAddress, backlog, readTimeout, metrics);
- }
- return server;
- }
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/e4b6b4af/hbase-thrift/src/main/resources/hbase-webapps/thrift/thrift.jsp
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/resources/hbase-webapps/thrift/thrift.jsp b/hbase-thrift/src/main/resources/hbase-webapps/thrift/thrift.jsp
index eb329fe..b0064b4 100644
--- a/hbase-thrift/src/main/resources/hbase-webapps/thrift/thrift.jsp
+++ b/hbase-thrift/src/main/resources/hbase-webapps/thrift/thrift.jsp
@@ -20,10 +20,10 @@
<%@ page contentType="text/html;charset=UTF-8"
import="org.apache.hadoop.conf.Configuration"
import="org.apache.hadoop.hbase.HBaseConfiguration"
- import="org.apache.hadoop.hbase.thrift.ThriftServerRunner.ImplType"
import="org.apache.hadoop.hbase.util.VersionInfo"
import="java.util.Date"
%>
+<%@ page import="org.apache.hadoop.hbase.thrift.ImplType" %>
<%
Configuration conf = (Configuration)getServletContext().getAttribute("hbase.conf");
http://git-wip-us.apache.org/repos/asf/hbase/blob/e4b6b4af/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftHttpServer.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftHttpServer.java b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftHttpServer.java
index 0eb7641..fe6655e 100644
--- a/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftHttpServer.java
+++ b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftHttpServer.java
@@ -17,8 +17,9 @@
*/
package org.apache.hadoop.hbase.thrift;
+import static org.apache.hadoop.hbase.thrift.Constants.INFOPORT_OPTION;
+import static org.apache.hadoop.hbase.thrift.Constants.PORT_OPTION;
import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNull;
import static org.junit.Assert.fail;
import java.net.HttpURLConnection;
@@ -65,7 +66,7 @@ public class TestThriftHttpServer {
private static final Logger LOG =
LoggerFactory.getLogger(TestThriftHttpServer.class);
- static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+ protected static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
private Thread httpServerThread;
private volatile Exception httpServerException;
@@ -97,18 +98,17 @@ public class TestThriftHttpServer {
conf.set("hbase.thrift.security.qop", "privacy");
conf.setBoolean("hbase.thrift.ssl.enabled", false);
- ThriftServerRunner runner = null;
+ ThriftServer server = null;
ExpectedException thrown = ExpectedException.none();
try {
thrown.expect(IllegalArgumentException.class);
thrown.expectMessage("Thrift HTTP Server's QoP is privacy, " +
"but hbase.thrift.ssl.enabled is false");
- runner = new ThriftServerRunner(conf);
+ server = new ThriftServer(conf);
+ server.run();
fail("Thrift HTTP Server starts up even with wrong security configurations.");
} catch (Exception e) {
}
-
- assertNull(runner);
}
private void startHttpServerThread(final String[] args) {
@@ -117,7 +117,7 @@ public class TestThriftHttpServer {
httpServerException = null;
httpServerThread = new Thread(() -> {
try {
- thriftServer.doMain(args);
+ thriftServer.run(args);
} catch (Exception e) {
httpServerException = e;
}
@@ -145,6 +145,10 @@ public class TestThriftHttpServer {
runThriftServer(1024 * 64);
}
+ protected ThriftServer createThriftServer() {
+ return new ThriftServer(TEST_UTIL.getConfiguration());
+ }
+
@Test
public void testRunThriftServer() throws Exception {
runThriftServer(0);
@@ -153,14 +157,14 @@ public class TestThriftHttpServer {
void runThriftServer(int customHeaderSize) throws Exception {
List<String> args = new ArrayList<>(3);
port = HBaseTestingUtility.randomFreePort();
- args.add("-" + ThriftServer.PORT_OPTION);
+ args.add("-" + PORT_OPTION);
args.add(String.valueOf(port));
- args.add("-" + ThriftServer.INFOPORT_OPTION);
+ args.add("-" + INFOPORT_OPTION);
int infoPort = HBaseTestingUtility.randomFreePort();
args.add(String.valueOf(infoPort));
args.add("start");
- thriftServer = new ThriftServer(TEST_UTIL.getConfiguration());
+ thriftServer = createThriftServer();
startHttpServerThread(args.toArray(new String[args.size()]));
// wait up to 10s for the server to start
@@ -195,9 +199,9 @@ public class TestThriftHttpServer {
Assert.assertEquals(HttpURLConnection.HTTP_FORBIDDEN, conn.getResponseCode());
}
- static volatile boolean tableCreated = false;
+ protected static volatile boolean tableCreated = false;
- void talkToThriftServer(String url, int customHeaderSize) throws Exception {
+ protected void talkToThriftServer(String url, int customHeaderSize) throws Exception {
THttpClient httpClient = new THttpClient(url);
httpClient.open();
http://git-wip-us.apache.org/repos/asf/hbase/blob/e4b6b4af/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftServer.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftServer.java b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftServer.java
index 05dc2ae..09d0605 100644
--- a/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftServer.java
+++ b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftServer.java
@@ -17,6 +17,7 @@
*/
package org.apache.hadoop.hbase.thrift;
+import static org.apache.hadoop.hbase.thrift.Constants.COALESCE_INC_KEY;
import static org.junit.Assert.assertArrayEquals;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
@@ -46,7 +47,6 @@ import org.apache.hadoop.hbase.security.UserProvider;
import org.apache.hadoop.hbase.test.MetricsAssertHelper;
import org.apache.hadoop.hbase.testclassification.ClientTests;
import org.apache.hadoop.hbase.testclassification.LargeTests;
-import org.apache.hadoop.hbase.thrift.ThriftServerRunner.HBaseHandler;
import org.apache.hadoop.hbase.thrift.generated.BatchMutation;
import org.apache.hadoop.hbase.thrift.generated.ColumnDescriptor;
import org.apache.hadoop.hbase.thrift.generated.Hbase;
@@ -71,7 +71,7 @@ import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
- * Unit testing for ThriftServerRunner.HBaseHandler, a part of the
+ * Unit testing for ThriftServerRunner.HBaseServiceHandler, a part of the
* org.apache.hadoop.hbase.thrift package.
*/
@Category({ClientTests.class, LargeTests.class})
@@ -113,7 +113,7 @@ public class TestThriftServer {
@BeforeClass
public static void beforeClass() throws Exception {
- UTIL.getConfiguration().setBoolean(ThriftServerRunner.COALESCE_INC_KEY, true);
+ UTIL.getConfiguration().setBoolean(COALESCE_INC_KEY, true);
UTIL.getConfiguration().setBoolean("hbase.table.sanity.checks", false);
UTIL.getConfiguration().setInt("hbase.client.retries.number", 3);
UTIL.startMiniCluster();
@@ -152,8 +152,8 @@ public class TestThriftServer {
* IllegalArgument exception.
*/
public void doTestTableCreateDrop() throws Exception {
- ThriftServerRunner.HBaseHandler handler =
- new ThriftServerRunner.HBaseHandler(UTIL.getConfiguration(),
+ ThriftHBaseServiceHandler handler =
+ new ThriftHBaseServiceHandler(UTIL.getConfiguration(),
UserProvider.instantiate(UTIL.getConfiguration()));
doTestTableCreateDrop(handler);
}
@@ -163,7 +163,7 @@ public class TestThriftServer {
dropTestTables(handler);
}
- public static final class MySlowHBaseHandler extends ThriftServerRunner.HBaseHandler
+ public static final class MySlowHBaseHandler extends ThriftHBaseServiceHandler
implements Hbase.Iface {
protected MySlowHBaseHandler(Configuration c)
@@ -230,7 +230,7 @@ public class TestThriftServer {
private static Hbase.Iface getHandlerForMetricsTest(ThriftMetrics metrics, Configuration conf)
throws Exception {
Hbase.Iface handler = new MySlowHBaseHandler(conf);
- return HbaseHandlerMetricsProxy.newInstance(handler, metrics, conf);
+ return HbaseHandlerMetricsProxy.newInstance((ThriftHBaseServiceHandler)handler, metrics, conf);
}
private static ThriftMetrics getMetrics(Configuration conf) throws Exception {
@@ -270,15 +270,15 @@ public class TestThriftServer {
}
public void doTestIncrements() throws Exception {
- ThriftServerRunner.HBaseHandler handler =
- new ThriftServerRunner.HBaseHandler(UTIL.getConfiguration(),
+ ThriftHBaseServiceHandler handler =
+ new ThriftHBaseServiceHandler(UTIL.getConfiguration(),
UserProvider.instantiate(UTIL.getConfiguration()));
createTestTables(handler);
doTestIncrements(handler);
dropTestTables(handler);
}
- public static void doTestIncrements(HBaseHandler handler) throws Exception {
+ public static void doTestIncrements(ThriftHBaseServiceHandler handler) throws Exception {
List<Mutation> mutations = new ArrayList<>(1);
mutations.add(new Mutation(false, columnAAname, valueEname, true));
mutations.add(new Mutation(false, columnAname, valueEname, true));
@@ -318,8 +318,8 @@ public class TestThriftServer {
* versions.
*/
public void doTestTableMutations() throws Exception {
- ThriftServerRunner.HBaseHandler handler =
- new ThriftServerRunner.HBaseHandler(UTIL.getConfiguration(),
+ ThriftHBaseServiceHandler handler =
+ new ThriftHBaseServiceHandler(UTIL.getConfiguration(),
UserProvider.instantiate(UTIL.getConfiguration()));
doTestTableMutations(handler);
}
@@ -395,8 +395,8 @@ public class TestThriftServer {
*/
public void doTestTableTimestampsAndColumns() throws Exception {
// Setup
- ThriftServerRunner.HBaseHandler handler =
- new ThriftServerRunner.HBaseHandler(UTIL.getConfiguration(),
+ ThriftHBaseServiceHandler handler =
+ new ThriftHBaseServiceHandler(UTIL.getConfiguration(),
UserProvider.instantiate(UTIL.getConfiguration()));
handler.createTable(tableAname, getColumnDescriptors());
@@ -473,8 +473,8 @@ public class TestThriftServer {
*/
public void doTestTableScanners() throws Exception {
// Setup
- ThriftServerRunner.HBaseHandler handler =
- new ThriftServerRunner.HBaseHandler(UTIL.getConfiguration(),
+ ThriftHBaseServiceHandler handler =
+ new ThriftHBaseServiceHandler(UTIL.getConfiguration(),
UserProvider.instantiate(UTIL.getConfiguration()));
handler.createTable(tableAname, getColumnDescriptors());
@@ -592,8 +592,8 @@ public class TestThriftServer {
* Tests for GetTableRegions
*/
public void doTestGetTableRegions() throws Exception {
- ThriftServerRunner.HBaseHandler handler =
- new ThriftServerRunner.HBaseHandler(UTIL.getConfiguration(),
+ ThriftHBaseServiceHandler handler =
+ new ThriftHBaseServiceHandler(UTIL.getConfiguration(),
UserProvider.instantiate(UTIL.getConfiguration()));
doTestGetTableRegions(handler);
}
@@ -620,7 +620,7 @@ public class TestThriftServer {
conf.set("hbase.thrift.filters", "MyFilter:filterclass");
- ThriftServerRunner.registerFilters(conf);
+ ThriftServer.registerFilters(conf);
Map<String, String> registeredFilters = ParseFilter.getAllFilters();
@@ -628,8 +628,8 @@ public class TestThriftServer {
}
public void doTestGetRegionInfo() throws Exception {
- ThriftServerRunner.HBaseHandler handler =
- new ThriftServerRunner.HBaseHandler(UTIL.getConfiguration(),
+ ThriftHBaseServiceHandler handler =
+ new ThriftHBaseServiceHandler(UTIL.getConfiguration(),
UserProvider.instantiate(UTIL.getConfiguration()));
doTestGetRegionInfo(handler);
}
@@ -655,8 +655,8 @@ public class TestThriftServer {
* Appends the value to a cell and checks that the cell value is updated properly.
*/
public static void doTestAppend() throws Exception {
- ThriftServerRunner.HBaseHandler handler =
- new ThriftServerRunner.HBaseHandler(UTIL.getConfiguration(),
+ ThriftHBaseServiceHandler handler =
+ new ThriftHBaseServiceHandler(UTIL.getConfiguration(),
UserProvider.instantiate(UTIL.getConfiguration()));
handler.createTable(tableAname, getColumnDescriptors());
try {
@@ -687,8 +687,8 @@ public class TestThriftServer {
* the checkAndPut succeeds.
*/
public static void doTestCheckAndPut() throws Exception {
- ThriftServerRunner.HBaseHandler handler =
- new ThriftServerRunner.HBaseHandler(UTIL.getConfiguration(),
+ ThriftHBaseServiceHandler handler =
+ new ThriftHBaseServiceHandler(UTIL.getConfiguration(),
UserProvider.instantiate(UTIL.getConfiguration()));
handler.createTable(tableAname, getColumnDescriptors());
try {
@@ -729,8 +729,8 @@ public class TestThriftServer {
Configuration conf = UTIL.getConfiguration();
ThriftMetrics metrics = getMetrics(conf);
- ThriftServerRunner.HBaseHandler hbaseHandler =
- new ThriftServerRunner.HBaseHandler(UTIL.getConfiguration(),
+ ThriftHBaseServiceHandler hbaseHandler =
+ new ThriftHBaseServiceHandler(UTIL.getConfiguration(),
UserProvider.instantiate(UTIL.getConfiguration()));
Hbase.Iface handler = HbaseHandlerMetricsProxy.newInstance(hbaseHandler, metrics, conf);
@@ -863,10 +863,10 @@ public class TestThriftServer {
* the scanner.
*
* @param scannerId the scanner to close
- * @param handler the HBaseHandler interfacing to HBase
+ * @param handler the HBaseServiceHandler interfacing to HBase
*/
private void closeScanner(
- int scannerId, ThriftServerRunner.HBaseHandler handler) throws Exception {
+ int scannerId, ThriftHBaseServiceHandler handler) throws Exception {
handler.scannerGet(scannerId);
handler.scannerClose(scannerId);
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/e4b6b4af/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftServerCmdLine.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftServerCmdLine.java b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftServerCmdLine.java
index 9e5a103..08dae61 100644
--- a/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftServerCmdLine.java
+++ b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftServerCmdLine.java
@@ -17,6 +17,11 @@
*/
package org.apache.hadoop.hbase.thrift;
+import static org.apache.hadoop.hbase.thrift.Constants.BIND_OPTION;
+import static org.apache.hadoop.hbase.thrift.Constants.COMPACT_OPTION;
+import static org.apache.hadoop.hbase.thrift.Constants.FRAMED_OPTION;
+import static org.apache.hadoop.hbase.thrift.Constants.INFOPORT_OPTION;
+import static org.apache.hadoop.hbase.thrift.Constants.PORT_OPTION;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
@@ -28,7 +33,6 @@ import org.apache.hadoop.hbase.HBaseClassTestRule;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.testclassification.ClientTests;
import org.apache.hadoop.hbase.testclassification.LargeTests;
-import org.apache.hadoop.hbase.thrift.ThriftServerRunner.ImplType;
import org.apache.hadoop.hbase.thrift.generated.Hbase;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManagerTestHelper;
@@ -68,12 +72,12 @@ public class TestThriftServerCmdLine {
private static final Logger LOG =
LoggerFactory.getLogger(TestThriftServerCmdLine.class);
- private final ImplType implType;
- private boolean specifyFramed;
- private boolean specifyBindIP;
- private boolean specifyCompact;
+ protected final ImplType implType;
+ protected boolean specifyFramed;
+ protected boolean specifyBindIP;
+ protected boolean specifyCompact;
- private static final HBaseTestingUtility TEST_UTIL =
+ protected static final HBaseTestingUtility TEST_UTIL =
new HBaseTestingUtility();
private Thread cmdLineThread;
@@ -81,8 +85,8 @@ public class TestThriftServerCmdLine {
private Exception clientSideException;
- private ThriftServer thriftServer;
- private int port;
+ private volatile ThriftServer thriftServer;
+ protected int port;
@Parameters
public static Collection<Object[]> getParameters() {
@@ -143,8 +147,9 @@ public class TestThriftServerCmdLine {
@Override
public void run() {
try {
- thriftServer.doMain(args);
+ thriftServer.run(args);
} catch (Exception e) {
+ LOG.error("Error when start thrift server", e);
cmdLineException = e;
}
}
@@ -154,6 +159,10 @@ public class TestThriftServerCmdLine {
cmdLineThread.start();
}
+ protected ThriftServer createThriftServer() {
+ return new ThriftServer(TEST_UTIL.getConfiguration());
+ }
+
@Test
public void testRunThriftServer() throws Exception {
List<String> args = new ArrayList<>();
@@ -163,37 +172,37 @@ public class TestThriftServerCmdLine {
args.add(serverTypeOption);
}
port = HBaseTestingUtility.randomFreePort();
- args.add("-" + ThriftServer.PORT_OPTION);
+ args.add("-" + PORT_OPTION);
args.add(String.valueOf(port));
- args.add("-" + ThriftServer.INFOPORT_OPTION);
+ args.add("-" + INFOPORT_OPTION);
int infoPort = HBaseTestingUtility.randomFreePort();
args.add(String.valueOf(infoPort));
if (specifyFramed) {
- args.add("-" + ThriftServer.FRAMED_OPTION);
+ args.add("-" + FRAMED_OPTION);
}
if (specifyBindIP) {
- args.add("-" + ThriftServer.BIND_OPTION);
+ args.add("-" + BIND_OPTION);
args.add(InetAddress.getLocalHost().getHostName());
}
if (specifyCompact) {
- args.add("-" + ThriftServer.COMPACT_OPTION);
+ args.add("-" + COMPACT_OPTION);
}
args.add("start");
- thriftServer = new ThriftServer(TEST_UTIL.getConfiguration());
+ thriftServer = createThriftServer();
startCmdLineThread(args.toArray(new String[args.size()]));
// wait up to 10s for the server to start
for (int i = 0; i < 100
- && (thriftServer.serverRunner == null || thriftServer.serverRunner.tserver == null); i++) {
+ && (thriftServer.tserver == null); i++) {
Thread.sleep(100);
}
Class<? extends TServer> expectedClass = implType != null ?
implType.serverClass : TBoundedThreadPoolServer.class;
assertEquals(expectedClass,
- thriftServer.serverRunner.tserver.getClass());
+ thriftServer.tserver.getClass());
try {
talkToThriftServer();
@@ -210,9 +219,9 @@ public class TestThriftServerCmdLine {
}
}
- private static volatile boolean tableCreated = false;
+ protected static volatile boolean tableCreated = false;
- private void talkToThriftServer() throws Exception {
+ protected void talkToThriftServer() throws Exception {
TSocket sock = new TSocket(InetAddress.getLocalHost().getHostName(),
port);
TTransport transport = sock;
@@ -228,6 +237,7 @@ public class TestThriftServerCmdLine {
} else {
prot = new TBinaryProtocol(transport);
}
+
Hbase.Client client = new Hbase.Client(prot);
if (!tableCreated){
TestThriftServer.createTestTables(client);
http://git-wip-us.apache.org/repos/asf/hbase/blob/e4b6b4af/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftSpnegoHttpServer.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftSpnegoHttpServer.java b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftSpnegoHttpServer.java
index 49eafec..45abe79 100644
--- a/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftSpnegoHttpServer.java
+++ b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftSpnegoHttpServer.java
@@ -18,12 +18,7 @@
*/
package org.apache.hadoop.hbase.thrift;
-import static org.apache.hadoop.hbase.thrift.ThriftServerRunner.THRIFT_KERBEROS_PRINCIPAL_KEY;
-import static org.apache.hadoop.hbase.thrift.ThriftServerRunner.THRIFT_KEYTAB_FILE_KEY;
-import static org.apache.hadoop.hbase.thrift.ThriftServerRunner.THRIFT_SPNEGO_KEYTAB_FILE_KEY;
-import static org.apache.hadoop.hbase.thrift.ThriftServerRunner.THRIFT_SPNEGO_PRINCIPAL_KEY;
-import static org.apache.hadoop.hbase.thrift.ThriftServerRunner.THRIFT_SUPPORT_PROXYUSER_KEY;
-import static org.apache.hadoop.hbase.thrift.ThriftServerRunner.USE_HTTP_CONF_KEY;
+import static org.apache.hadoop.hbase.thrift.Constants.THRIFT_SUPPORT_PROXYUSER_KEY;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertTrue;
@@ -133,14 +128,14 @@ public class TestThriftSpnegoHttpServer extends TestThriftHttpServer {
HBaseKerberosUtils.setSecuredConfiguration(conf, serverPrincipal, spnegoServerPrincipal);
conf.setBoolean(THRIFT_SUPPORT_PROXYUSER_KEY, true);
- conf.setBoolean(USE_HTTP_CONF_KEY, true);
+ conf.setBoolean(Constants.USE_HTTP_CONF_KEY, true);
conf.set("hadoop.proxyuser.hbase.hosts", "*");
conf.set("hadoop.proxyuser.hbase.groups", "*");
- conf.set(THRIFT_KERBEROS_PRINCIPAL_KEY, serverPrincipal);
- conf.set(THRIFT_KEYTAB_FILE_KEY, serverKeytab.getAbsolutePath());
- conf.set(THRIFT_SPNEGO_PRINCIPAL_KEY, spnegoServerPrincipal);
- conf.set(THRIFT_SPNEGO_KEYTAB_FILE_KEY, spnegoServerKeytab.getAbsolutePath());
+ conf.set(Constants.THRIFT_KERBEROS_PRINCIPAL_KEY, serverPrincipal);
+ conf.set(Constants.THRIFT_KEYTAB_FILE_KEY, serverKeytab.getAbsolutePath());
+ conf.set(Constants.THRIFT_SPNEGO_PRINCIPAL_KEY, spnegoServerPrincipal);
+ conf.set(Constants.THRIFT_SPNEGO_KEYTAB_FILE_KEY, spnegoServerKeytab.getAbsolutePath());
}
@BeforeClass
@@ -170,7 +165,7 @@ public class TestThriftSpnegoHttpServer extends TestThriftHttpServer {
spnegoServerKeytab = new File(keytabDir, spnegoServerPrincipal.replace('/', '_') + ".keytab");
setupUser(kdc, spnegoServerKeytab, spnegoServerPrincipal);
- TEST_UTIL.getConfiguration().setBoolean(USE_HTTP_CONF_KEY, true);
+ TEST_UTIL.getConfiguration().setBoolean(Constants.USE_HTTP_CONF_KEY, true);
TEST_UTIL.getConfiguration().setBoolean("hbase.table.sanity.checks", false);
addSecurityConfigurations(TEST_UTIL.getConfiguration());
@@ -191,7 +186,7 @@ public class TestThriftSpnegoHttpServer extends TestThriftHttpServer {
}
@Override
- void talkToThriftServer(String url, int customHeaderSize) throws Exception {
+ protected void talkToThriftServer(String url, int customHeaderSize) throws Exception {
// Close httpClient and THttpClient automatically on any failures
try (
CloseableHttpClient httpClient = createHttpClient();
http://git-wip-us.apache.org/repos/asf/hbase/blob/e4b6b4af/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThrift2HttpServer.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThrift2HttpServer.java b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThrift2HttpServer.java
new file mode 100644
index 0000000..cf084c9
--- /dev/null
+++ b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThrift2HttpServer.java
@@ -0,0 +1,90 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.thrift2;
+
+import java.util.ArrayList;
+
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.testclassification.ClientTests;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.thrift.TestThriftHttpServer;
+import org.apache.hadoop.hbase.thrift2.generated.TColumnFamilyDescriptor;
+import org.apache.hadoop.hbase.thrift2.generated.THBaseService;
+import org.apache.hadoop.hbase.thrift2.generated.TTableDescriptor;
+import org.apache.hadoop.hbase.thrift2.generated.TTableName;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.thrift.protocol.TBinaryProtocol;
+import org.apache.thrift.protocol.TProtocol;
+import org.apache.thrift.transport.THttpClient;
+import org.junit.Assert;
+import org.junit.ClassRule;
+import org.junit.experimental.categories.Category;
+
+@Category({ ClientTests.class, MediumTests.class})
+public class TestThrift2HttpServer extends TestThriftHttpServer {
+ private static final String TABLENAME = "TestThrift2HttpServerTable";
+
+ @ClassRule
+ public static final HBaseClassTestRule CLASS_RULE =
+ HBaseClassTestRule.forClass(TestThrift2HttpServer.class);
+
+
+
+ @Override
+ protected ThriftServer createThriftServer() {
+ return new ThriftServer(TEST_UTIL.getConfiguration());
+ }
+
+ @Override
+ protected void talkToThriftServer(String url, int customHeaderSize) throws Exception {
+ THttpClient httpClient = new THttpClient(url);
+ httpClient.open();
+
+ if (customHeaderSize > 0) {
+ StringBuilder sb = new StringBuilder();
+ for (int i = 0; i < customHeaderSize; i++) {
+ sb.append("a");
+ }
+ httpClient.setCustomHeader("User-Agent", sb.toString());
+ }
+
+ try {
+ TProtocol prot;
+ prot = new TBinaryProtocol(httpClient);
+ THBaseService.Client client = new THBaseService.Client(prot);
+ TTableName tTableName = new TTableName();
+ tTableName.setNs(Bytes.toBytes(""));
+ tTableName.setQualifier(Bytes.toBytes(TABLENAME));
+ if (!tableCreated){
+ Assert.assertTrue(!client.tableExists(tTableName));
+ TTableDescriptor tTableDescriptor = new TTableDescriptor();
+ tTableDescriptor.setTableName(tTableName);
+ TColumnFamilyDescriptor columnFamilyDescriptor = new TColumnFamilyDescriptor();
+ columnFamilyDescriptor.setName(Bytes.toBytes(TABLENAME));
+ tTableDescriptor.addToColumns(columnFamilyDescriptor);
+ client.createTable(tTableDescriptor, new ArrayList<>());
+ tableCreated = true;
+ }
+ Assert.assertTrue(client.tableExists(tTableName));
+ } finally {
+ httpClient.close();
+ }
+ }
+
+
+}
http://git-wip-us.apache.org/repos/asf/hbase/blob/e4b6b4af/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThrift2ServerCmdLine.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThrift2ServerCmdLine.java b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThrift2ServerCmdLine.java
new file mode 100644
index 0000000..7489bd7
--- /dev/null
+++ b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThrift2ServerCmdLine.java
@@ -0,0 +1,99 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.thrift2;
+
+import java.net.InetAddress;
+import java.util.ArrayList;
+
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.testclassification.ClientTests;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.thrift.ImplType;
+import org.apache.hadoop.hbase.thrift.TestThriftServerCmdLine;
+import org.apache.hadoop.hbase.thrift2.generated.TColumnFamilyDescriptor;
+import org.apache.hadoop.hbase.thrift2.generated.THBaseService;
+import org.apache.hadoop.hbase.thrift2.generated.TTableDescriptor;
+import org.apache.hadoop.hbase.thrift2.generated.TTableName;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.thrift.protocol.TBinaryProtocol;
+import org.apache.thrift.protocol.TCompactProtocol;
+import org.apache.thrift.protocol.TProtocol;
+import org.apache.thrift.transport.TFramedTransport;
+import org.apache.thrift.transport.TSocket;
+import org.apache.thrift.transport.TTransport;
+import org.junit.Assert;
+import org.junit.ClassRule;
+import org.junit.experimental.categories.Category;
+
+@Category({ ClientTests.class, MediumTests.class})
+public class TestThrift2ServerCmdLine extends TestThriftServerCmdLine {
+ @ClassRule
+ public static final HBaseClassTestRule CLASS_RULE =
+ HBaseClassTestRule.forClass(TestThrift2ServerCmdLine.class);
+
+ private static final String TABLENAME = "TestThrift2ServerCmdLineTable";
+
+
+ @Override
+ protected ThriftServer createThriftServer() {
+ return new ThriftServer(TEST_UTIL.getConfiguration());
+ }
+
+ public TestThrift2ServerCmdLine(ImplType implType, boolean specifyFramed,
+ boolean specifyBindIP, boolean specifyCompact) {
+ super(implType, specifyFramed, specifyBindIP, specifyCompact);
+ }
+
+ @Override
+ protected void talkToThriftServer() throws Exception {
+ TSocket sock = new TSocket(InetAddress.getLocalHost().getHostName(),
+ port);
+ TTransport transport = sock;
+ if (specifyFramed || implType.isAlwaysFramed()) {
+ transport = new TFramedTransport(transport);
+ }
+
+ sock.open();
+ try {
+ TProtocol tProtocol;
+ if (specifyCompact) {
+ tProtocol = new TCompactProtocol(transport);
+ } else {
+ tProtocol = new TBinaryProtocol(transport);
+ }
+ THBaseService.Client client = new THBaseService.Client(tProtocol);
+ TTableName tTableName = new TTableName();
+ tTableName.setNs(Bytes.toBytes(""));
+ tTableName.setQualifier(Bytes.toBytes(TABLENAME));
+ if (!tableCreated){
+ Assert.assertTrue(!client.tableExists(tTableName));
+ TTableDescriptor tTableDescriptor = new TTableDescriptor();
+ tTableDescriptor.setTableName(tTableName);
+ TColumnFamilyDescriptor columnFamilyDescriptor = new TColumnFamilyDescriptor();
+ columnFamilyDescriptor.setName(Bytes.toBytes(TABLENAME));
+ tTableDescriptor.addToColumns(columnFamilyDescriptor);
+ client.createTable(tTableDescriptor, new ArrayList<>());
+ tableCreated = true;
+ }
+ Assert.assertTrue(client.tableExists(tTableName));
+ } finally {
+ sock.close();
+ }
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/hbase/blob/e4b6b4af/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThriftHBaseServiceHandler.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThriftHBaseServiceHandler.java b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThriftHBaseServiceHandler.java
index 0734f85..844c1c0 100644
--- a/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThriftHBaseServiceHandler.java
+++ b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThriftHBaseServiceHandler.java
@@ -18,6 +18,8 @@
package org.apache.hadoop.hbase.thrift2;
import static java.nio.ByteBuffer.wrap;
+import static org.apache.hadoop.hbase.thrift.HBaseServiceHandler.CLEANUP_INTERVAL;
+import static org.apache.hadoop.hbase.thrift.HBaseServiceHandler.MAX_IDLETIME;
import static org.apache.hadoop.hbase.thrift2.ThriftUtilities.deleteFromThrift;
import static org.apache.hadoop.hbase.thrift2.ThriftUtilities.getFromThrift;
import static org.apache.hadoop.hbase.thrift2.ThriftUtilities.incrementFromThrift;
@@ -71,6 +73,7 @@ import org.apache.hadoop.hbase.test.MetricsAssertHelper;
import org.apache.hadoop.hbase.testclassification.ClientTests;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.thrift.ErrorThrowingGetObserver;
+import org.apache.hadoop.hbase.thrift.HbaseHandlerMetricsProxy;
import org.apache.hadoop.hbase.thrift.ThriftMetrics;
import org.apache.hadoop.hbase.thrift2.generated.TAppend;
import org.apache.hadoop.hbase.thrift2.generated.TColumn;
@@ -115,7 +118,7 @@ import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
import org.apache.hbase.thirdparty.org.apache.commons.collections4.CollectionUtils;
/**
- * Unit testing for ThriftServer.HBaseHandler, a part of the org.apache.hadoop.hbase.thrift2
+ * Unit testing for ThriftServer.HBaseServiceHandler, a part of the org.apache.hadoop.hbase.thrift2
* package.
*/
@Category({ClientTests.class, MediumTests.class})
@@ -749,8 +752,8 @@ public class TestThriftHBaseServiceHandler {
int cleanUpInterval = 100;
Configuration conf = new Configuration(UTIL.getConfiguration());
// Set the ConnectionCache timeout to trigger halfway through the trials
- conf.setInt(ThriftHBaseServiceHandler.MAX_IDLETIME, (numTrials / 2) * trialPause);
- conf.setInt(ThriftHBaseServiceHandler.CLEANUP_INTERVAL, cleanUpInterval);
+ conf.setInt(MAX_IDLETIME, (numTrials / 2) * trialPause);
+ conf.setInt(CLEANUP_INTERVAL, cleanUpInterval);
ThriftHBaseServiceHandler handler = new ThriftHBaseServiceHandler(conf,
UserProvider.instantiate(conf));
@@ -1206,7 +1209,7 @@ public class TestThriftHBaseServiceHandler {
ThriftMetrics metrics = getMetrics(conf);
ThriftHBaseServiceHandler hbaseHandler = createHandler();
THBaseService.Iface handler =
- ThriftHBaseServiceHandler.newInstance(hbaseHandler, metrics);
+ HbaseHandlerMetricsProxy.newInstance(hbaseHandler, metrics, conf);
byte[] rowName = Bytes.toBytes("testMetrics");
ByteBuffer table = wrap(tableAname);
@@ -1249,7 +1252,7 @@ public class TestThriftHBaseServiceHandler {
ThriftHBaseServiceHandler hbaseHandler = createHandler();
ThriftMetrics metrics = getMetrics(UTIL.getConfiguration());
THBaseService.Iface handler =
- ThriftHBaseServiceHandler.newInstance(hbaseHandler, metrics);
+ HbaseHandlerMetricsProxy.newInstance(hbaseHandler, metrics, null);
ByteBuffer tTableName = wrap(tableName.getName());
// check metrics increment with a successful get
@@ -1323,7 +1326,7 @@ public class TestThriftHBaseServiceHandler {
ThriftHBaseServiceHandler hbaseHandler = createHandler();
ThriftMetrics metrics = getMetrics(UTIL.getConfiguration());
THBaseService.Iface handler =
- ThriftHBaseServiceHandler.newInstance(hbaseHandler, metrics);
+ HbaseHandlerMetricsProxy.newInstance(hbaseHandler, metrics, null);
ByteBuffer tTableName = wrap(tableName.getName());
// check metrics latency with a successful get
[03/15] hbase git commit: HBASE-21652 Refactor ThriftServer making
thrift2 server inherited from thrift1 server
Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/e4b6b4af/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
deleted file mode 100644
index 5e248f1..0000000
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServerRunner.java
+++ /dev/null
@@ -1,2031 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.thrift;
-
-import static org.apache.hadoop.hbase.util.Bytes.getBytes;
-
-import java.io.IOException;
-import java.net.InetAddress;
-import java.net.InetSocketAddress;
-import java.net.UnknownHostException;
-import java.nio.ByteBuffer;
-import java.security.PrivilegedAction;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.TreeMap;
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.ExecutorService;
-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.SaslServer;
-
-import org.apache.commons.lang3.ArrayUtils;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.Cell.Type;
-import org.apache.hadoop.hbase.CellBuilder;
-import org.apache.hadoop.hbase.CellBuilderFactory;
-import org.apache.hadoop.hbase.CellBuilderType;
-import org.apache.hadoop.hbase.CellUtil;
-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.HTableDescriptor;
-import org.apache.hadoop.hbase.KeyValue;
-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.hadoop.hbase.client.Admin;
-import org.apache.hadoop.hbase.client.Append;
-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.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;
-import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.client.Table;
-import org.apache.hadoop.hbase.filter.Filter;
-import org.apache.hadoop.hbase.filter.ParseFilter;
-import org.apache.hadoop.hbase.filter.PrefixFilter;
-import org.apache.hadoop.hbase.filter.WhileMatchFilter;
-import org.apache.hadoop.hbase.http.HttpServerUtil;
-import org.apache.hadoop.hbase.log.HBaseMarkers;
-import org.apache.hadoop.hbase.security.SaslUtil;
-import org.apache.hadoop.hbase.security.SaslUtil.QualityOfProtection;
-import org.apache.hadoop.hbase.security.SecurityUtil;
-import org.apache.hadoop.hbase.security.UserProvider;
-import org.apache.hadoop.hbase.thrift.generated.AlreadyExists;
-import org.apache.hadoop.hbase.thrift.generated.BatchMutation;
-import org.apache.hadoop.hbase.thrift.generated.ColumnDescriptor;
-import org.apache.hadoop.hbase.thrift.generated.Hbase;
-import org.apache.hadoop.hbase.thrift.generated.IOError;
-import org.apache.hadoop.hbase.thrift.generated.IllegalArgument;
-import org.apache.hadoop.hbase.thrift.generated.Mutation;
-import org.apache.hadoop.hbase.thrift.generated.TAppend;
-import org.apache.hadoop.hbase.thrift.generated.TCell;
-import org.apache.hadoop.hbase.thrift.generated.TIncrement;
-import org.apache.hadoop.hbase.thrift.generated.TRegionInfo;
-import org.apache.hadoop.hbase.thrift.generated.TRowResult;
-import org.apache.hadoop.hbase.thrift.generated.TScan;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.ConnectionCache;
-import org.apache.hadoop.hbase.util.DNS;
-import org.apache.hadoop.hbase.util.JvmPauseMonitor;
-import org.apache.hadoop.hbase.util.Strings;
-import org.apache.hadoop.security.SaslRpcServer.SaslGssCallbackHandler;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.security.authorize.ProxyUsers;
-import org.apache.thrift.TException;
-import org.apache.thrift.TProcessor;
-import org.apache.thrift.protocol.TBinaryProtocol;
-import org.apache.thrift.protocol.TCompactProtocol;
-import org.apache.thrift.protocol.TProtocolFactory;
-import org.apache.thrift.server.THsHaServer;
-import org.apache.thrift.server.TNonblockingServer;
-import org.apache.thrift.server.TServer;
-import org.apache.thrift.server.TServlet;
-import org.apache.thrift.server.TThreadedSelectorServer;
-import org.apache.thrift.transport.TFramedTransport;
-import org.apache.thrift.transport.TNonblockingServerSocket;
-import org.apache.thrift.transport.TNonblockingServerTransport;
-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.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.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.apache.hbase.thirdparty.com.google.common.base.Joiner;
-import org.apache.hbase.thirdparty.com.google.common.base.Splitter;
-import org.apache.hbase.thirdparty.com.google.common.base.Throwables;
-import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
-import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLine;
-import org.apache.hbase.thirdparty.org.apache.commons.cli.Option;
-import org.apache.hbase.thirdparty.org.apache.commons.cli.OptionGroup;
-
-/**
- * ThriftServerRunner - this class starts up a Thrift server which implements
- * the Hbase API specified in the Hbase.thrift IDL file.
- */
-@InterfaceAudience.Private
-public class ThriftServerRunner implements Runnable {
-
- private static final Logger LOG = LoggerFactory.getLogger(ThriftServerRunner.class);
-
- private static final int DEFAULT_HTTP_MAX_HEADER_SIZE = 64 * 1024; // 64k
-
- static final String SERVER_TYPE_CONF_KEY =
- "hbase.regionserver.thrift.server.type";
-
- static final String BIND_CONF_KEY = "hbase.regionserver.thrift.ipaddress";
- static final String COMPACT_CONF_KEY = "hbase.regionserver.thrift.compact";
- static final String FRAMED_CONF_KEY = "hbase.regionserver.thrift.framed";
- static final String MAX_FRAME_SIZE_CONF_KEY =
- "hbase.regionserver.thrift.framed.max_frame_size_in_mb";
- static final String PORT_CONF_KEY = "hbase.regionserver.thrift.port";
- static final String COALESCE_INC_KEY = "hbase.regionserver.thrift.coalesceIncrement";
- static final String USE_HTTP_CONF_KEY = "hbase.regionserver.thrift.http";
- static final String HTTP_MIN_THREADS_KEY = "hbase.thrift.http_threads.min";
- static final String HTTP_MAX_THREADS_KEY = "hbase.thrift.http_threads.max";
-
- static final String THRIFT_SSL_ENABLED_KEY = "hbase.thrift.ssl.enabled";
- static final String THRIFT_SSL_KEYSTORE_STORE_KEY = "hbase.thrift.ssl.keystore.store";
- static final String THRIFT_SSL_KEYSTORE_PASSWORD_KEY = "hbase.thrift.ssl.keystore.password";
- static final String THRIFT_SSL_KEYSTORE_KEYPASSWORD_KEY = "hbase.thrift.ssl.keystore.keypassword";
- static final String THRIFT_SSL_EXCLUDE_CIPHER_SUITES_KEY =
- "hbase.thrift.ssl.exclude.cipher.suites";
- static final String THRIFT_SSL_INCLUDE_CIPHER_SUITES_KEY =
- "hbase.thrift.ssl.include.cipher.suites";
- static final String THRIFT_SSL_EXCLUDE_PROTOCOLS_KEY = "hbase.thrift.ssl.exclude.protocols";
- static final String THRIFT_SSL_INCLUDE_PROTOCOLS_KEY = "hbase.thrift.ssl.include.protocols";
-
- static final String THRIFT_SUPPORT_PROXYUSER_KEY = "hbase.thrift.support.proxyuser";
-
- static final String THRIFT_DNS_INTERFACE_KEY = "hbase.thrift.dns.interface";
- static final String THRIFT_DNS_NAMESERVER_KEY = "hbase.thrift.dns.nameserver";
- static final String THRIFT_KERBEROS_PRINCIPAL_KEY = "hbase.thrift.kerberos.principal";
- static final String THRIFT_KEYTAB_FILE_KEY = "hbase.thrift.keytab.file";
- static final String THRIFT_SPNEGO_PRINCIPAL_KEY = "hbase.thrift.spnego.principal";
- static final String THRIFT_SPNEGO_KEYTAB_FILE_KEY = "hbase.thrift.spnego.keytab.file";
-
- /**
- * Amount of time in milliseconds before a server thread will timeout
- * waiting for client to send data on a connected socket. Currently,
- * applies only to TBoundedThreadPoolServer
- */
- public static final String THRIFT_SERVER_SOCKET_READ_TIMEOUT_KEY =
- "hbase.thrift.server.socket.read.timeout";
- public static final int THRIFT_SERVER_SOCKET_READ_TIMEOUT_DEFAULT = 60000;
-
-
- /**
- * Thrift quality of protection configuration key. Valid values can be:
- * auth-conf: authentication, integrity and confidentiality checking
- * auth-int: authentication and integrity checking
- * auth: authentication only
- *
- * This is used to authenticate the callers and support impersonation.
- * The thrift server and the HBase cluster must run in secure mode.
- */
- static final String THRIFT_QOP_KEY = "hbase.thrift.security.qop";
- static final String BACKLOG_CONF_KEY = "hbase.regionserver.thrift.backlog";
-
- private static final String DEFAULT_BIND_ADDR = "0.0.0.0";
- public static final int DEFAULT_LISTEN_PORT = 9090;
- public static final int HREGION_VERSION = 1;
-
- private final int listenPort;
-
- private Configuration conf;
- volatile TServer tserver;
- volatile Server httpServer;
- private final Hbase.Iface handler;
- private final ThriftMetrics metrics;
- private final HBaseHandler hbaseHandler;
- private final UserGroupInformation serviceUGI;
-
- private SaslUtil.QualityOfProtection qop;
- private String host;
-
- private final boolean securityEnabled;
- private final boolean doAsEnabled;
-
- private final JvmPauseMonitor pauseMonitor;
-
- static String THRIFT_HTTP_ALLOW_OPTIONS_METHOD = "hbase.thrift.http.allow.options.method";
- private static boolean THRIFT_HTTP_ALLOW_OPTIONS_METHOD_DEFAULT = false;
-
- /** An enum of server implementation selections */
- public enum ImplType {
- HS_HA("hsha", true, THsHaServer.class, true),
- NONBLOCKING("nonblocking", true, TNonblockingServer.class, true),
- THREAD_POOL("threadpool", false, TBoundedThreadPoolServer.class, true),
- THREADED_SELECTOR("threadedselector", true, TThreadedSelectorServer.class, true);
-
- public static final ImplType DEFAULT = THREAD_POOL;
-
- final String option;
- final boolean isAlwaysFramed;
- final Class<? extends TServer> serverClass;
- final boolean canSpecifyBindIP;
-
- private ImplType(String option, boolean isAlwaysFramed,
- Class<? extends TServer> serverClass, boolean canSpecifyBindIP) {
- this.option = option;
- this.isAlwaysFramed = isAlwaysFramed;
- this.serverClass = serverClass;
- this.canSpecifyBindIP = canSpecifyBindIP;
- }
-
- /**
- * @return <code>-option</code>
- */
- @Override
- public String toString() {
- return "-" + option;
- }
-
- public String getOption() {
- return option;
- }
-
- public boolean isAlwaysFramed() {
- return isAlwaysFramed;
- }
-
- public String getDescription() {
- StringBuilder sb = new StringBuilder("Use the " +
- serverClass.getSimpleName());
- if (isAlwaysFramed) {
- sb.append(" This implies the framed transport.");
- }
- if (this == DEFAULT) {
- sb.append("This is the default.");
- }
- return sb.toString();
- }
-
- static OptionGroup createOptionGroup() {
- OptionGroup group = new OptionGroup();
- for (ImplType t : values()) {
- group.addOption(new Option(t.option, t.getDescription()));
- }
- return group;
- }
-
- public static ImplType getServerImpl(Configuration conf) {
- String confType = conf.get(SERVER_TYPE_CONF_KEY, THREAD_POOL.option);
- for (ImplType t : values()) {
- if (confType.equals(t.option)) {
- return t;
- }
- }
- throw new AssertionError("Unknown server ImplType.option:" + confType);
- }
-
- static void setServerImpl(CommandLine cmd, Configuration conf) {
- ImplType chosenType = null;
- int numChosen = 0;
- for (ImplType t : values()) {
- if (cmd.hasOption(t.option)) {
- chosenType = t;
- ++numChosen;
- }
- }
- if (numChosen < 1) {
- LOG.info("Using default thrift server type");
- chosenType = DEFAULT;
- } else if (numChosen > 1) {
- throw new AssertionError("Exactly one option out of " +
- Arrays.toString(values()) + " has to be specified");
- }
- LOG.info("Using thrift server type " + chosenType.option);
- conf.set(SERVER_TYPE_CONF_KEY, chosenType.option);
- }
-
- public String simpleClassName() {
- return serverClass.getSimpleName();
- }
-
- public static List<String> serversThatCannotSpecifyBindIP() {
- List<String> l = new ArrayList<>();
- for (ImplType t : values()) {
- if (!t.canSpecifyBindIP) {
- l.add(t.simpleClassName());
- }
- }
- return l;
- }
- }
-
- public ThriftServerRunner(Configuration conf) throws IOException {
- // login the server principal (if using secure Hadoop)
- UserProvider userProvider = UserProvider.instantiate(conf);
- securityEnabled = userProvider.isHadoopSecurityEnabled()
- && userProvider.isHBaseSecurityEnabled();
- if (securityEnabled) {
- host = Strings.domainNamePointerToHostName(DNS.getDefaultHost(
- conf.get(THRIFT_DNS_INTERFACE_KEY, "default"),
- conf.get(THRIFT_DNS_NAMESERVER_KEY, "default")));
- userProvider.login(THRIFT_KEYTAB_FILE_KEY, THRIFT_KERBEROS_PRINCIPAL_KEY, host);
- }
- this.serviceUGI = userProvider.getCurrent().getUGI();
-
- this.conf = HBaseConfiguration.create(conf);
- this.listenPort = conf.getInt(PORT_CONF_KEY, DEFAULT_LISTEN_PORT);
- this.metrics = new ThriftMetrics(conf, ThriftMetrics.ThriftServerType.ONE);
- this.pauseMonitor = new JvmPauseMonitor(conf, this.metrics.getSource());
- this.hbaseHandler = new HBaseHandler(conf, userProvider);
- this.hbaseHandler.initMetrics(metrics);
- this.handler = HbaseHandlerMetricsProxy.newInstance(hbaseHandler, metrics, conf);
-
- boolean httpEnabled = conf.getBoolean(USE_HTTP_CONF_KEY, false);
- doAsEnabled = conf.getBoolean(THRIFT_SUPPORT_PROXYUSER_KEY, false);
- if (doAsEnabled && !httpEnabled) {
- LOG.warn("Fail to enable the doAs feature. " + USE_HTTP_CONF_KEY + " is not configured");
- }
-
- String strQop = conf.get(THRIFT_QOP_KEY);
- if (strQop != null) {
- this.qop = SaslUtil.getQop(strQop);
- }
- if (qop != null) {
- if (qop != QualityOfProtection.AUTHENTICATION &&
- qop != QualityOfProtection.INTEGRITY &&
- qop != QualityOfProtection.PRIVACY) {
- throw new IOException(String.format("Invalid %s: It must be one of %s, %s, or %s.",
- THRIFT_QOP_KEY,
- QualityOfProtection.AUTHENTICATION.name(),
- QualityOfProtection.INTEGRITY.name(),
- QualityOfProtection.PRIVACY.name()));
- }
- checkHttpSecurity(qop, conf);
- if (!securityEnabled) {
- throw new IOException("Thrift server must run in secure mode to support authentication");
- }
- }
- }
-
- private void checkHttpSecurity(QualityOfProtection qop, Configuration conf) {
- if (qop == QualityOfProtection.PRIVACY &&
- conf.getBoolean(USE_HTTP_CONF_KEY, false) &&
- !conf.getBoolean(THRIFT_SSL_ENABLED_KEY, false)) {
- throw new IllegalArgumentException("Thrift HTTP Server's QoP is privacy, but " +
- THRIFT_SSL_ENABLED_KEY + " is false");
- }
- }
-
- /*
- * Runs the Thrift server
- */
- @Override
- public void run() {
- serviceUGI.doAs(new PrivilegedAction<Object>() {
- @Override
- public Object run() {
- try {
- pauseMonitor.start();
- if (conf.getBoolean(USE_HTTP_CONF_KEY, false)) {
- setupHTTPServer();
- httpServer.start();
- httpServer.join();
- } else {
- setupServer();
- tserver.serve();
- }
- } catch (Exception e) {
- LOG.error(HBaseMarkers.FATAL, "Cannot run ThriftServer", e);
- // Crash the process if the ThriftServer is not running
- System.exit(-1);
- }
- return null;
- }
- });
-
- }
-
- public void shutdown() {
- if (pauseMonitor != null) {
- pauseMonitor.stop();
- }
- if (tserver != null) {
- tserver.stop();
- tserver = null;
- }
- if (httpServer != null) {
- try {
- httpServer.stop();
- httpServer = null;
- } catch (Exception e) {
- LOG.error("Problem encountered in shutting down HTTP server", e);
- }
- httpServer = null;
- }
- }
-
- private void setupHTTPServer() throws IOException {
- TProtocolFactory protocolFactory = new TBinaryProtocol.Factory();
- TProcessor processor = new Hbase.Processor<>(handler);
- TServlet thriftHttpServlet = new ThriftHttpServlet(processor, protocolFactory, serviceUGI,
- conf, hbaseHandler, securityEnabled, doAsEnabled);
-
- // Set the default max thread number to 100 to limit
- // the number of concurrent requests so that Thrfit HTTP server doesn't OOM easily.
- // Jetty set the default max thread number to 250, if we don't set it.
- //
- // Our default min thread number 2 is the same as that used by Jetty.
- int minThreads = conf.getInt(HTTP_MIN_THREADS_KEY, 2);
- int maxThreads = conf.getInt(HTTP_MAX_THREADS_KEY, 100);
- QueuedThreadPool threadPool = new QueuedThreadPool(maxThreads);
- threadPool.setMinThreads(minThreads);
- httpServer = new Server(threadPool);
-
- // Context handler
- ServletContextHandler ctxHandler = new ServletContextHandler(httpServer, "/",
- ServletContextHandler.SESSIONS);
- ctxHandler.addServlet(new ServletHolder(thriftHttpServlet), "/*");
- HttpServerUtil.constrainHttpMethods(ctxHandler,
- conf.getBoolean(THRIFT_HTTP_ALLOW_OPTIONS_METHOD, THRIFT_HTTP_ALLOW_OPTIONS_METHOD_DEFAULT));
-
- // set up Jetty and run the embedded server
- HttpConfiguration httpConfig = new HttpConfiguration();
- httpConfig.setSecureScheme("https");
- httpConfig.setSecurePort(listenPort);
- httpConfig.setHeaderCacheSize(DEFAULT_HTTP_MAX_HEADER_SIZE);
- httpConfig.setRequestHeaderSize(DEFAULT_HTTP_MAX_HEADER_SIZE);
- httpConfig.setResponseHeaderSize(DEFAULT_HTTP_MAX_HEADER_SIZE);
- httpConfig.setSendServerVersion(false);
- httpConfig.setSendDateHeader(false);
-
- ServerConnector serverConnector;
- if(conf.getBoolean(THRIFT_SSL_ENABLED_KEY, false)) {
- HttpConfiguration httpsConfig = new HttpConfiguration(httpConfig);
- httpsConfig.addCustomizer(new SecureRequestCustomizer());
-
- SslContextFactory sslCtxFactory = new SslContextFactory();
- String keystore = conf.get(THRIFT_SSL_KEYSTORE_STORE_KEY);
- String password = HBaseConfiguration.getPassword(conf,
- THRIFT_SSL_KEYSTORE_PASSWORD_KEY, null);
- String keyPassword = HBaseConfiguration.getPassword(conf,
- THRIFT_SSL_KEYSTORE_KEYPASSWORD_KEY, password);
- sslCtxFactory.setKeyStorePath(keystore);
- sslCtxFactory.setKeyStorePassword(password);
- sslCtxFactory.setKeyManagerPassword(keyPassword);
-
- String[] excludeCiphers = conf.getStrings(
- THRIFT_SSL_EXCLUDE_CIPHER_SUITES_KEY, ArrayUtils.EMPTY_STRING_ARRAY);
- if (excludeCiphers.length != 0) {
- sslCtxFactory.setExcludeCipherSuites(excludeCiphers);
- }
- String[] includeCiphers = conf.getStrings(
- THRIFT_SSL_INCLUDE_CIPHER_SUITES_KEY, ArrayUtils.EMPTY_STRING_ARRAY);
- if (includeCiphers.length != 0) {
- sslCtxFactory.setIncludeCipherSuites(includeCiphers);
- }
-
- // Disable SSLv3 by default due to "Poodle" Vulnerability - CVE-2014-3566
- String[] excludeProtocols = conf.getStrings(
- THRIFT_SSL_EXCLUDE_PROTOCOLS_KEY, "SSLv3");
- if (excludeProtocols.length != 0) {
- sslCtxFactory.setExcludeProtocols(excludeProtocols);
- }
- String[] includeProtocols = conf.getStrings(
- THRIFT_SSL_INCLUDE_PROTOCOLS_KEY, ArrayUtils.EMPTY_STRING_ARRAY);
- if (includeProtocols.length != 0) {
- sslCtxFactory.setIncludeProtocols(includeProtocols);
- }
-
- serverConnector = new ServerConnector(httpServer,
- new SslConnectionFactory(sslCtxFactory, HttpVersion.HTTP_1_1.toString()),
- new HttpConnectionFactory(httpsConfig));
- } else {
- serverConnector = new ServerConnector(httpServer, new HttpConnectionFactory(httpConfig));
- }
- serverConnector.setPort(listenPort);
- serverConnector.setHost(getBindAddress(conf).getHostAddress());
- httpServer.addConnector(serverConnector);
- httpServer.setStopAtShutdown(true);
-
- if (doAsEnabled) {
- ProxyUsers.refreshSuperUserGroupsConfiguration(conf);
- }
-
- LOG.info("Starting Thrift HTTP Server on {}", Integer.toString(listenPort));
- }
-
- /**
- * Setting up the thrift TServer
- */
- private void setupServer() throws Exception {
- // Construct correct ProtocolFactory
- TProtocolFactory protocolFactory = getProtocolFactory();
-
- final TProcessor p = new Hbase.Processor<>(handler);
- ImplType implType = ImplType.getServerImpl(conf);
- TProcessor processor = p;
-
- // Construct correct TransportFactory
- TTransportFactory transportFactory;
- if (conf.getBoolean(FRAMED_CONF_KEY, false) || implType.isAlwaysFramed) {
- if (qop != null) {
- throw new RuntimeException("Thrift server authentication"
- + " doesn't work with framed transport yet");
- }
- transportFactory = new TFramedTransport.Factory(
- conf.getInt(MAX_FRAME_SIZE_CONF_KEY, 2) * 1024 * 1024);
- LOG.debug("Using framed transport");
- } else if (qop == null) {
- transportFactory = new TTransportFactory();
- } else {
- // Extract the name from the principal
- String thriftKerberosPrincipal = conf.get(THRIFT_KERBEROS_PRINCIPAL_KEY);
- if (thriftKerberosPrincipal == null) {
- throw new IllegalArgumentException(THRIFT_KERBEROS_PRINCIPAL_KEY + " cannot be null");
- }
- String name = SecurityUtil.getUserFromPrincipal(thriftKerberosPrincipal);
- Map<String, String> saslProperties = SaslUtil.initSaslProperties(qop.name());
- TSaslServerTransport.Factory saslFactory = new TSaslServerTransport.Factory();
- saslFactory.addServerDefinition("GSSAPI", name, host, saslProperties,
- new SaslGssCallbackHandler() {
- @Override
- public void handle(Callback[] callbacks)
- throws UnsupportedCallbackException {
- AuthorizeCallback ac = null;
- for (Callback callback : callbacks) {
- if (callback instanceof AuthorizeCallback) {
- ac = (AuthorizeCallback) callback;
- } else {
- throw new UnsupportedCallbackException(callback,
- "Unrecognized SASL GSSAPI Callback");
- }
- }
- if (ac != null) {
- String authid = ac.getAuthenticationID();
- String authzid = ac.getAuthorizationID();
- if (!authid.equals(authzid)) {
- ac.setAuthorized(false);
- } else {
- ac.setAuthorized(true);
- String userName = SecurityUtil.getUserFromPrincipal(authzid);
- LOG.info("Effective user: {}", userName);
- ac.setAuthorizedID(userName);
- }
- }
- }
- });
- transportFactory = saslFactory;
-
- // Create a processor wrapper, to get the caller
- processor = (inProt, outProt) -> {
- TSaslServerTransport saslServerTransport =
- (TSaslServerTransport)inProt.getTransport();
- SaslServer saslServer = saslServerTransport.getSaslServer();
- String principal = saslServer.getAuthorizationID();
- hbaseHandler.setEffectiveUser(principal);
- return p.process(inProt, outProt);
- };
- }
-
- if (conf.get(BIND_CONF_KEY) != null && !implType.canSpecifyBindIP) {
- LOG.error("Server types {} don't support IP address binding at the moment. See " +
- "https://issues.apache.org/jira/browse/HBASE-2155 for details.",
- Joiner.on(", ").join(ImplType.serversThatCannotSpecifyBindIP()));
- throw new RuntimeException("-" + BIND_CONF_KEY + " not supported with " + implType);
- }
-
- // Thrift's implementation uses '0' as a placeholder for 'use the default.'
- int backlog = conf.getInt(BACKLOG_CONF_KEY, 0);
-
- if (implType == ImplType.HS_HA || implType == ImplType.NONBLOCKING ||
- implType == ImplType.THREADED_SELECTOR) {
- InetAddress listenAddress = getBindAddress(conf);
- TNonblockingServerTransport serverTransport = new TNonblockingServerSocket(
- new InetSocketAddress(listenAddress, listenPort));
-
- if (implType == ImplType.NONBLOCKING) {
- TNonblockingServer.Args serverArgs =
- new TNonblockingServer.Args(serverTransport);
- serverArgs.processor(processor)
- .transportFactory(transportFactory)
- .protocolFactory(protocolFactory);
- tserver = new TNonblockingServer(serverArgs);
- } else if (implType == ImplType.HS_HA) {
- THsHaServer.Args serverArgs = new THsHaServer.Args(serverTransport);
- CallQueue callQueue = new CallQueue(new LinkedBlockingQueue<>(), metrics);
- ExecutorService executorService = createExecutor(
- callQueue, serverArgs.getMaxWorkerThreads(), serverArgs.getMaxWorkerThreads());
- serverArgs.executorService(executorService).processor(processor)
- .transportFactory(transportFactory).protocolFactory(protocolFactory);
- tserver = new THsHaServer(serverArgs);
- } else { // THREADED_SELECTOR
- TThreadedSelectorServer.Args serverArgs =
- new HThreadedSelectorServerArgs(serverTransport, conf);
- CallQueue callQueue = new CallQueue(new LinkedBlockingQueue<>(), metrics);
- ExecutorService executorService = createExecutor(
- callQueue, serverArgs.getWorkerThreads(), serverArgs.getWorkerThreads());
- serverArgs.executorService(executorService).processor(processor)
- .transportFactory(transportFactory).protocolFactory(protocolFactory);
- tserver = new TThreadedSelectorServer(serverArgs);
- }
- LOG.info("starting HBase {} server on {}", implType.simpleClassName(),
- Integer.toString(listenPort));
- } else if (implType == ImplType.THREAD_POOL) {
- // Thread pool server. Get the IP address to bind to.
- InetAddress listenAddress = getBindAddress(conf);
- int readTimeout = conf.getInt(THRIFT_SERVER_SOCKET_READ_TIMEOUT_KEY,
- THRIFT_SERVER_SOCKET_READ_TIMEOUT_DEFAULT);
- TServerTransport serverTransport = new TServerSocket(
- new TServerSocket.ServerSocketTransportArgs().
- bindAddr(new InetSocketAddress(listenAddress, listenPort)).backlog(backlog).
- clientTimeout(readTimeout));
-
- TBoundedThreadPoolServer.Args serverArgs =
- new TBoundedThreadPoolServer.Args(serverTransport, conf);
- serverArgs.processor(processor).transportFactory(transportFactory)
- .protocolFactory(protocolFactory);
- LOG.info("starting " + ImplType.THREAD_POOL.simpleClassName() + " on "
- + listenAddress + ":" + Integer.toString(listenPort)
- + " with readTimeout " + readTimeout + "ms; " + serverArgs);
- this.tserver = new TBoundedThreadPoolServer(serverArgs, metrics);
- } else {
- throw new AssertionError("Unsupported Thrift server implementation: " +
- implType.simpleClassName());
- }
-
- // A sanity check that we instantiated the right type of server.
- if (tserver.getClass() != implType.serverClass) {
- throw new AssertionError("Expected to create Thrift server class " +
- implType.serverClass.getName() + " but got " +
- tserver.getClass().getName());
- }
-
-
-
- registerFilters(conf);
- }
-
- private TProtocolFactory getProtocolFactory() {
- TProtocolFactory protocolFactory;
-
- if (conf.getBoolean(COMPACT_CONF_KEY, false)) {
- LOG.debug("Using compact protocol");
- protocolFactory = new TCompactProtocol.Factory();
- } else {
- LOG.debug("Using binary protocol");
- protocolFactory = new TBinaryProtocol.Factory();
- }
-
- return protocolFactory;
- }
-
- ExecutorService createExecutor(BlockingQueue<Runnable> callQueue,
- int minWorkers, int maxWorkers) {
- ThreadFactoryBuilder tfb = new ThreadFactoryBuilder();
- tfb.setDaemon(true);
- tfb.setNameFormat("thrift-worker-%d");
- ThreadPoolExecutor threadPool = new THBaseThreadPoolExecutor(minWorkers, maxWorkers,
- Long.MAX_VALUE, TimeUnit.SECONDS, callQueue, tfb.build(), metrics);
- threadPool.allowCoreThreadTimeOut(true);
- return threadPool;
- }
-
- private InetAddress getBindAddress(Configuration conf)
- throws UnknownHostException {
- String bindAddressStr = conf.get(BIND_CONF_KEY, DEFAULT_BIND_ADDR);
- return InetAddress.getByName(bindAddressStr);
- }
-
- protected static class ResultScannerWrapper {
-
- private final ResultScanner scanner;
- private final boolean sortColumns;
- public ResultScannerWrapper(ResultScanner resultScanner,
- boolean sortResultColumns) {
- scanner = resultScanner;
- sortColumns = sortResultColumns;
- }
-
- public ResultScanner getScanner() {
- return scanner;
- }
-
- public boolean isColumnSorted() {
- return sortColumns;
- }
- }
-
- /**
- * The HBaseHandler is a glue object that connects Thrift RPC calls to the
- * HBase client API primarily defined in the Admin and Table objects.
- */
- public static class HBaseHandler implements Hbase.Iface {
- protected Configuration conf;
- protected static final Logger LOG = LoggerFactory.getLogger(HBaseHandler.class);
-
- // nextScannerId and scannerMap are used to manage scanner state
- protected int nextScannerId = 0;
- protected HashMap<Integer, ResultScannerWrapper> scannerMap;
- private ThriftMetrics metrics = null;
-
- private final ConnectionCache connectionCache;
- IncrementCoalescer coalescer;
-
- static final String CLEANUP_INTERVAL = "hbase.thrift.connection.cleanup-interval";
- static final String MAX_IDLETIME = "hbase.thrift.connection.max-idletime";
-
- /**
- * Returns a list of all the column families for a given Table.
- *
- * @param table
- * @throws IOException
- */
- byte[][] getAllColumns(Table table) throws IOException {
- HColumnDescriptor[] cds = table.getTableDescriptor().getColumnFamilies();
- byte[][] columns = new byte[cds.length][];
- for (int i = 0; i < cds.length; i++) {
- columns[i] = Bytes.add(cds[i].getName(),
- KeyValue.COLUMN_FAMILY_DELIM_ARRAY);
- }
- return columns;
- }
-
- /**
- * Creates and returns a Table instance from a given table name.
- *
- * @param tableName
- * name of table
- * @return Table object
- * @throws IOException if getting the table fails
- */
- public Table getTable(final byte[] tableName) throws IOException {
- String table = Bytes.toString(tableName);
- return connectionCache.getTable(table);
- }
-
- public Table getTable(final ByteBuffer tableName) throws IOException {
- return getTable(getBytes(tableName));
- }
-
- /**
- * Assigns a unique ID to the scanner and adds the mapping to an internal
- * hash-map.
- *
- * @param scanner the {@link ResultScanner} to add
- * @return integer scanner id
- */
- protected synchronized int addScanner(ResultScanner scanner, boolean sortColumns) {
- int id = nextScannerId++;
- ResultScannerWrapper resultScannerWrapper = new ResultScannerWrapper(scanner, sortColumns);
- scannerMap.put(id, resultScannerWrapper);
- return id;
- }
-
- /**
- * Returns the scanner associated with the specified ID.
- *
- * @param id the ID of the scanner to get
- * @return a Scanner, or null if ID was invalid.
- */
- protected synchronized ResultScannerWrapper getScanner(int id) {
- return scannerMap.get(id);
- }
-
- /**
- * Removes the scanner associated with the specified ID from the internal
- * id->scanner hash-map.
- *
- * @param id the ID of the scanner to remove
- * @return a Scanner, or null if ID was invalid.
- */
- protected synchronized ResultScannerWrapper removeScanner(int id) {
- return scannerMap.remove(id);
- }
-
- protected HBaseHandler(final Configuration c,
- final UserProvider userProvider) throws IOException {
- this.conf = c;
- scannerMap = new HashMap<>();
- this.coalescer = new IncrementCoalescer(this);
-
- int cleanInterval = conf.getInt(CLEANUP_INTERVAL, 10 * 1000);
- int maxIdleTime = conf.getInt(MAX_IDLETIME, 10 * 60 * 1000);
- connectionCache = new ConnectionCache(
- conf, userProvider, cleanInterval, maxIdleTime);
- }
-
- /**
- * Obtain HBaseAdmin. Creates the instance if it is not already created.
- */
- private Admin getAdmin() throws IOException {
- return connectionCache.getAdmin();
- }
-
- void setEffectiveUser(String effectiveUser) {
- connectionCache.setEffectiveUser(effectiveUser);
- }
-
- @Override
- public void enableTable(ByteBuffer tableName) throws IOError {
- try{
- getAdmin().enableTable(getTableName(tableName));
- } catch (IOException e) {
- LOG.warn(e.getMessage(), e);
- throw getIOError(e);
- }
- }
-
- @Override
- public void disableTable(ByteBuffer tableName) throws IOError{
- try{
- getAdmin().disableTable(getTableName(tableName));
- } catch (IOException e) {
- LOG.warn(e.getMessage(), e);
- throw getIOError(e);
- }
- }
-
- @Override
- public boolean isTableEnabled(ByteBuffer tableName) throws IOError {
- try {
- return this.connectionCache.getAdmin().isTableEnabled(getTableName(tableName));
- } catch (IOException e) {
- LOG.warn(e.getMessage(), e);
- throw getIOError(e);
- }
- }
-
- // ThriftServerRunner.compact should be deprecated and replaced with methods specific to
- // table and region.
- @Override
- public void compact(ByteBuffer tableNameOrRegionName) throws IOError {
- try {
- try {
- getAdmin().compactRegion(getBytes(tableNameOrRegionName));
- } catch (IllegalArgumentException e) {
- // Invalid region, try table
- getAdmin().compact(TableName.valueOf(getBytes(tableNameOrRegionName)));
- }
- } catch (IOException e) {
- LOG.warn(e.getMessage(), e);
- throw getIOError(e);
- }
- }
-
- // ThriftServerRunner.majorCompact should be deprecated and replaced with methods specific
- // to table and region.
- @Override
- public void majorCompact(ByteBuffer tableNameOrRegionName) throws IOError {
- try {
- try {
- getAdmin().compactRegion(getBytes(tableNameOrRegionName));
- } catch (IllegalArgumentException e) {
- // Invalid region, try table
- getAdmin().compact(TableName.valueOf(getBytes(tableNameOrRegionName)));
- }
- } catch (IOException e) {
- LOG.warn(e.getMessage(), e);
- throw getIOError(e);
- }
- }
-
- @Override
- public List<ByteBuffer> getTableNames() throws IOError {
- try {
- TableName[] tableNames = this.getAdmin().listTableNames();
- ArrayList<ByteBuffer> list = new ArrayList<>(tableNames.length);
- for (TableName tableName : tableNames) {
- list.add(ByteBuffer.wrap(tableName.getName()));
- }
- return list;
- } catch (IOException e) {
- LOG.warn(e.getMessage(), e);
- throw getIOError(e);
- }
- }
-
- /**
- * @return the list of regions in the given table, or an empty list if the table does not exist
- */
- @Override
- public List<TRegionInfo> getTableRegions(ByteBuffer tableName) throws IOError {
- try (RegionLocator locator = connectionCache.getRegionLocator(getBytes(tableName))) {
- List<HRegionLocation> regionLocations = locator.getAllRegionLocations();
- List<TRegionInfo> results = new ArrayList<>(regionLocations.size());
- for (HRegionLocation regionLocation : regionLocations) {
- RegionInfo info = regionLocation.getRegionInfo();
- ServerName serverName = regionLocation.getServerName();
- TRegionInfo region = new TRegionInfo();
- region.serverName = ByteBuffer.wrap(
- Bytes.toBytes(serverName.getHostname()));
- region.port = serverName.getPort();
- region.startKey = ByteBuffer.wrap(info.getStartKey());
- region.endKey = ByteBuffer.wrap(info.getEndKey());
- region.id = info.getRegionId();
- region.name = ByteBuffer.wrap(info.getRegionName());
- region.version = HREGION_VERSION; // HRegion now not versioned, PB encoding used
- results.add(region);
- }
- return results;
- } catch (TableNotFoundException e) {
- // Return empty list for non-existing table
- return Collections.emptyList();
- } catch (IOException e){
- LOG.warn(e.getMessage(), e);
- throw getIOError(e);
- }
- }
-
- @Override
- public List<TCell> get(
- ByteBuffer tableName, ByteBuffer row, ByteBuffer column,
- Map<ByteBuffer, ByteBuffer> attributes)
- throws IOError {
- byte [][] famAndQf = CellUtil.parseColumn(getBytes(column));
- if (famAndQf.length == 1) {
- return get(tableName, row, famAndQf[0], null, attributes);
- }
- if (famAndQf.length == 2) {
- return get(tableName, row, famAndQf[0], famAndQf[1], attributes);
- }
- throw new IllegalArgumentException("Invalid familyAndQualifier provided.");
- }
-
- /**
- * Note: this internal interface is slightly different from public APIs in regard to handling
- * of the qualifier. Here we differ from the public Java API in that null != byte[0]. Rather,
- * we respect qual == null as a request for the entire column family. The caller (
- * {@link #get(ByteBuffer, ByteBuffer, ByteBuffer, Map)}) interface IS consistent in that the
- * column is parse like normal.
- */
- protected List<TCell> get(ByteBuffer tableName,
- ByteBuffer row,
- byte[] family,
- byte[] qualifier,
- Map<ByteBuffer, ByteBuffer> attributes) throws IOError {
- Table table = null;
- try {
- table = getTable(tableName);
- Get get = new Get(getBytes(row));
- addAttributes(get, attributes);
- if (qualifier == null) {
- get.addFamily(family);
- } else {
- get.addColumn(family, qualifier);
- }
- Result result = table.get(get);
- return ThriftUtilities.cellFromHBase(result.rawCells());
- } catch (IOException e) {
- LOG.warn(e.getMessage(), e);
- throw getIOError(e);
- } finally {
- closeTable(table);
- }
- }
-
- @Override
- public List<TCell> getVer(ByteBuffer tableName, ByteBuffer row, ByteBuffer column,
- int numVersions, Map<ByteBuffer, ByteBuffer> attributes) throws IOError {
- byte [][] famAndQf = CellUtil.parseColumn(getBytes(column));
- if(famAndQf.length == 1) {
- return getVer(tableName, row, famAndQf[0], null, numVersions, attributes);
- }
- if (famAndQf.length == 2) {
- return getVer(tableName, row, famAndQf[0], famAndQf[1], numVersions, attributes);
- }
- throw new IllegalArgumentException("Invalid familyAndQualifier provided.");
-
- }
-
- /**
- * Note: this public interface is slightly different from public Java APIs in regard to
- * handling of the qualifier. Here we differ from the public Java API in that null != byte[0].
- * Rather, we respect qual == null as a request for the entire column family. If you want to
- * access the entire column family, use
- * {@link #getVer(ByteBuffer, ByteBuffer, ByteBuffer, int, Map)} with a {@code column} value
- * that lacks a {@code ':'}.
- */
- 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);
- Get get = new Get(getBytes(row));
- addAttributes(get, attributes);
- if (null == qualifier) {
- get.addFamily(family);
- } else {
- get.addColumn(family, qualifier);
- }
- get.setMaxVersions(numVersions);
- Result result = table.get(get);
- return ThriftUtilities.cellFromHBase(result.rawCells());
- } catch (IOException e) {
- LOG.warn(e.getMessage(), e);
- throw getIOError(e);
- } finally{
- closeTable(table);
- }
- }
-
- @Override
- public List<TCell> getVerTs(ByteBuffer tableName, ByteBuffer row, ByteBuffer column,
- long timestamp, int numVersions, Map<ByteBuffer, ByteBuffer> attributes) throws IOError {
- byte [][] famAndQf = CellUtil.parseColumn(getBytes(column));
- if (famAndQf.length == 1) {
- return getVerTs(tableName, row, famAndQf[0], null, timestamp, numVersions, attributes);
- }
- if (famAndQf.length == 2) {
- return getVerTs(tableName, row, famAndQf[0], famAndQf[1], timestamp, numVersions,
- attributes);
- }
- throw new IllegalArgumentException("Invalid familyAndQualifier provided.");
- }
-
- /**
- * Note: this internal interface is slightly different from public APIs in regard to handling
- * of the qualifier. Here we differ from the public Java API in that null != byte[0]. Rather,
- * we respect qual == null as a request for the entire column family. The caller (
- * {@link #getVerTs(ByteBuffer, ByteBuffer, ByteBuffer, long, int, Map)}) interface IS
- * consistent in that the column is parse like normal.
- */
- 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);
- Get get = new Get(getBytes(row));
- addAttributes(get, attributes);
- if (null == qualifier) {
- get.addFamily(family);
- } else {
- get.addColumn(family, qualifier);
- }
- get.setTimeRange(0, timestamp);
- get.setMaxVersions(numVersions);
- Result result = table.get(get);
- return ThriftUtilities.cellFromHBase(result.rawCells());
- } catch (IOException e) {
- LOG.warn(e.getMessage(), e);
- throw getIOError(e);
- } finally{
- closeTable(table);
- }
- }
-
- @Override
- public List<TRowResult> getRow(ByteBuffer tableName, ByteBuffer row,
- Map<ByteBuffer, ByteBuffer> attributes) throws IOError {
- return getRowWithColumnsTs(tableName, row, null,
- HConstants.LATEST_TIMESTAMP,
- attributes);
- }
-
- @Override
- public List<TRowResult> getRowWithColumns(ByteBuffer tableName,
- ByteBuffer row,
- List<ByteBuffer> columns,
- Map<ByteBuffer, ByteBuffer> attributes) throws IOError {
- return getRowWithColumnsTs(tableName, row, columns,
- HConstants.LATEST_TIMESTAMP,
- attributes);
- }
-
- @Override
- public List<TRowResult> getRowTs(ByteBuffer tableName, ByteBuffer row,
- long timestamp, Map<ByteBuffer, ByteBuffer> attributes) throws IOError {
- return getRowWithColumnsTs(tableName, row, null,
- timestamp, attributes);
- }
-
- @Override
- 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);
- if (columns == null) {
- Get get = new Get(getBytes(row));
- addAttributes(get, attributes);
- get.setTimeRange(0, timestamp);
- Result result = table.get(get);
- return ThriftUtilities.rowResultFromHBase(result);
- }
- Get get = new Get(getBytes(row));
- addAttributes(get, attributes);
- for(ByteBuffer column : columns) {
- byte [][] famAndQf = CellUtil.parseColumn(getBytes(column));
- if (famAndQf.length == 1) {
- get.addFamily(famAndQf[0]);
- } else {
- get.addColumn(famAndQf[0], famAndQf[1]);
- }
- }
- get.setTimeRange(0, timestamp);
- Result result = table.get(get);
- return ThriftUtilities.rowResultFromHBase(result);
- } catch (IOException e) {
- LOG.warn(e.getMessage(), e);
- throw getIOError(e);
- } finally{
- closeTable(table);
- }
- }
-
- @Override
- public List<TRowResult> getRows(ByteBuffer tableName,
- List<ByteBuffer> rows,
- Map<ByteBuffer, ByteBuffer> attributes)
- throws IOError {
- return getRowsWithColumnsTs(tableName, rows, null,
- HConstants.LATEST_TIMESTAMP,
- attributes);
- }
-
- @Override
- public List<TRowResult> getRowsWithColumns(ByteBuffer tableName,
- List<ByteBuffer> rows,
- List<ByteBuffer> columns,
- Map<ByteBuffer, ByteBuffer> attributes) throws IOError {
- return getRowsWithColumnsTs(tableName, rows, columns,
- HConstants.LATEST_TIMESTAMP,
- attributes);
- }
-
- @Override
- public List<TRowResult> getRowsTs(ByteBuffer tableName,
- List<ByteBuffer> rows,
- long timestamp,
- Map<ByteBuffer, ByteBuffer> attributes) throws IOError {
- return getRowsWithColumnsTs(tableName, rows, null,
- timestamp, attributes);
- }
-
- @Override
- public List<TRowResult> getRowsWithColumnsTs(ByteBuffer tableName,
- 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());
- table = getTable(tableName);
- if (metrics != null) {
- metrics.incNumRowKeysInBatchGet(rows.size());
- }
- for (ByteBuffer row : rows) {
- Get get = new Get(getBytes(row));
- addAttributes(get, attributes);
- if (columns != null) {
-
- for(ByteBuffer column : columns) {
- byte [][] famAndQf = CellUtil.parseColumn(getBytes(column));
- if (famAndQf.length == 1) {
- get.addFamily(famAndQf[0]);
- } else {
- get.addColumn(famAndQf[0], famAndQf[1]);
- }
- }
- }
- get.setTimeRange(0, timestamp);
- gets.add(get);
- }
- Result[] result = table.get(gets);
- return ThriftUtilities.rowResultFromHBase(result);
- } catch (IOException e) {
- LOG.warn(e.getMessage(), e);
- throw getIOError(e);
- } finally{
- closeTable(table);
- }
- }
-
- @Override
- public void deleteAll(
- ByteBuffer tableName, ByteBuffer row, ByteBuffer column,
- Map<ByteBuffer, ByteBuffer> attributes)
- throws IOError {
- deleteAllTs(tableName, row, column, HConstants.LATEST_TIMESTAMP,
- attributes);
- }
-
- @Override
- public void deleteAllTs(ByteBuffer tableName,
- ByteBuffer row,
- ByteBuffer column,
- long timestamp, Map<ByteBuffer, ByteBuffer> attributes) throws IOError {
- Table table = null;
- try {
- table = getTable(tableName);
- Delete delete = new Delete(getBytes(row));
- addAttributes(delete, attributes);
- byte [][] famAndQf = CellUtil.parseColumn(getBytes(column));
- if (famAndQf.length == 1) {
- delete.addFamily(famAndQf[0], timestamp);
- } else {
- delete.addColumns(famAndQf[0], famAndQf[1], timestamp);
- }
- table.delete(delete);
-
- } catch (IOException e) {
- LOG.warn(e.getMessage(), e);
- throw getIOError(e);
- } finally {
- closeTable(table);
- }
- }
-
- @Override
- public void deleteAllRow(
- ByteBuffer tableName, ByteBuffer row,
- Map<ByteBuffer, ByteBuffer> attributes) throws IOError {
- deleteAllRowTs(tableName, row, HConstants.LATEST_TIMESTAMP, attributes);
- }
-
- @Override
- public void deleteAllRowTs(
- ByteBuffer tableName, ByteBuffer row, long timestamp,
- Map<ByteBuffer, ByteBuffer> attributes) throws IOError {
- Table table = null;
- try {
- table = getTable(tableName);
- Delete delete = new Delete(getBytes(row), timestamp);
- addAttributes(delete, attributes);
- table.delete(delete);
- } catch (IOException e) {
- LOG.warn(e.getMessage(), e);
- throw getIOError(e);
- } finally {
- closeTable(table);
- }
- }
-
- @Override
- public void createTable(ByteBuffer in_tableName,
- List<ColumnDescriptor> columnFamilies) throws IOError,
- IllegalArgument, AlreadyExists {
- TableName tableName = getTableName(in_tableName);
- try {
- if (getAdmin().tableExists(tableName)) {
- throw new AlreadyExists("table name already in use");
- }
- HTableDescriptor desc = new HTableDescriptor(tableName);
- for (ColumnDescriptor col : columnFamilies) {
- HColumnDescriptor colDesc = ThriftUtilities.colDescFromThrift(col);
- desc.addFamily(colDesc);
- }
- getAdmin().createTable(desc);
- } catch (IOException e) {
- LOG.warn(e.getMessage(), e);
- throw getIOError(e);
- } catch (IllegalArgumentException e) {
- LOG.warn(e.getMessage(), e);
- throw new IllegalArgument(Throwables.getStackTraceAsString(e));
- }
- }
-
- private static TableName getTableName(ByteBuffer buffer) {
- return TableName.valueOf(getBytes(buffer));
- }
-
- @Override
- public void deleteTable(ByteBuffer in_tableName) throws IOError {
- TableName tableName = getTableName(in_tableName);
- if (LOG.isDebugEnabled()) {
- LOG.debug("deleteTable: table={}", tableName);
- }
- try {
- if (!getAdmin().tableExists(tableName)) {
- throw new IOException("table does not exist");
- }
- getAdmin().deleteTable(tableName);
- } catch (IOException e) {
- LOG.warn(e.getMessage(), e);
- throw getIOError(e);
- }
- }
-
- @Override
- public void mutateRow(ByteBuffer tableName, ByteBuffer row,
- List<Mutation> mutations, Map<ByteBuffer, ByteBuffer> attributes)
- throws IOError, IllegalArgument {
- mutateRowTs(tableName, row, mutations, HConstants.LATEST_TIMESTAMP, attributes);
- }
-
- @Override
- public void mutateRowTs(ByteBuffer tableName, ByteBuffer row,
- List<Mutation> mutations, long timestamp,
- Map<ByteBuffer, ByteBuffer> attributes)
- throws IOError, IllegalArgument {
- Table table = null;
- try {
- table = getTable(tableName);
- Put put = new Put(getBytes(row), timestamp);
- addAttributes(put, attributes);
-
- Delete delete = new Delete(getBytes(row));
- addAttributes(delete, attributes);
- if (metrics != null) {
- metrics.incNumRowKeysInBatchMutate(mutations.size());
- }
-
- // I apologize for all this mess :)
- CellBuilder builder = CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY);
- for (Mutation m : mutations) {
- byte[][] famAndQf = CellUtil.parseColumn(getBytes(m.column));
- if (m.isDelete) {
- if (famAndQf.length == 1) {
- delete.addFamily(famAndQf[0], timestamp);
- } else {
- delete.addColumns(famAndQf[0], famAndQf[1], timestamp);
- }
- delete.setDurability(m.writeToWAL ? Durability.SYNC_WAL : Durability.SKIP_WAL);
- } else {
- if(famAndQf.length == 1) {
- LOG.warn("No column qualifier specified. Delete is the only mutation supported "
- + "over the whole column family.");
- } else {
- put.add(builder.clear()
- .setRow(put.getRow())
- .setFamily(famAndQf[0])
- .setQualifier(famAndQf[1])
- .setTimestamp(put.getTimestamp())
- .setType(Type.Put)
- .setValue(m.value != null ? getBytes(m.value)
- : HConstants.EMPTY_BYTE_ARRAY)
- .build());
- }
- put.setDurability(m.writeToWAL ? Durability.SYNC_WAL : Durability.SKIP_WAL);
- }
- }
- if (!delete.isEmpty()) {
- table.delete(delete);
- }
- if (!put.isEmpty()) {
- table.put(put);
- }
- } catch (IOException e) {
- LOG.warn(e.getMessage(), e);
- throw getIOError(e);
- } catch (IllegalArgumentException e) {
- LOG.warn(e.getMessage(), e);
- throw new IllegalArgument(Throwables.getStackTraceAsString(e));
- } finally{
- closeTable(table);
- }
- }
-
- @Override
- public void mutateRows(ByteBuffer tableName, List<BatchMutation> rowBatches,
- Map<ByteBuffer, ByteBuffer> attributes)
- throws IOError, IllegalArgument, TException {
- mutateRowsTs(tableName, rowBatches, HConstants.LATEST_TIMESTAMP, attributes);
- }
-
- @Override
- public void mutateRowsTs(
- ByteBuffer tableName, List<BatchMutation> rowBatches, long timestamp,
- Map<ByteBuffer, ByteBuffer> attributes)
- throws IOError, IllegalArgument, TException {
- List<Put> puts = new ArrayList<>();
- List<Delete> deletes = new ArrayList<>();
- CellBuilder builder = CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY);
- for (BatchMutation batch : rowBatches) {
- byte[] row = getBytes(batch.row);
- List<Mutation> mutations = batch.mutations;
- Delete delete = new Delete(row);
- addAttributes(delete, attributes);
- Put put = new Put(row, timestamp);
- addAttributes(put, attributes);
- for (Mutation m : mutations) {
- byte[][] famAndQf = CellUtil.parseColumn(getBytes(m.column));
- if (m.isDelete) {
- // no qualifier, family only.
- if (famAndQf.length == 1) {
- delete.addFamily(famAndQf[0], timestamp);
- } else {
- delete.addColumns(famAndQf[0], famAndQf[1], timestamp);
- }
- delete.setDurability(m.writeToWAL ? Durability.SYNC_WAL
- : Durability.SKIP_WAL);
- } else {
- if (famAndQf.length == 1) {
- LOG.warn("No column qualifier specified. Delete is the only mutation supported "
- + "over the whole column family.");
- }
- if (famAndQf.length == 2) {
- try {
- put.add(builder.clear()
- .setRow(put.getRow())
- .setFamily(famAndQf[0])
- .setQualifier(famAndQf[1])
- .setTimestamp(put.getTimestamp())
- .setType(Type.Put)
- .setValue(m.value != null ? getBytes(m.value)
- : HConstants.EMPTY_BYTE_ARRAY)
- .build());
- } catch (IOException e) {
- throw new IllegalArgumentException(e);
- }
- } else {
- throw new IllegalArgumentException("Invalid famAndQf provided.");
- }
- put.setDurability(m.writeToWAL ? Durability.SYNC_WAL : Durability.SKIP_WAL);
- }
- }
- if (!delete.isEmpty()) {
- deletes.add(delete);
- }
- if (!put.isEmpty()) {
- puts.add(put);
- }
- }
-
- Table table = null;
- try {
- table = getTable(tableName);
- if (!puts.isEmpty()) {
- table.put(puts);
- }
- if (!deletes.isEmpty()) {
- table.delete(deletes);
- }
- } catch (IOException e) {
- LOG.warn(e.getMessage(), e);
- throw getIOError(e);
- } catch (IllegalArgumentException e) {
- LOG.warn(e.getMessage(), e);
- throw new IllegalArgument(Throwables.getStackTraceAsString(e));
- } finally{
- closeTable(table);
- }
- }
-
- @Override
- public long atomicIncrement(
- ByteBuffer tableName, ByteBuffer row, ByteBuffer column, long amount)
- throws IOError, IllegalArgument, TException {
- byte [][] famAndQf = CellUtil.parseColumn(getBytes(column));
- if(famAndQf.length == 1) {
- return atomicIncrement(tableName, row, famAndQf[0], HConstants.EMPTY_BYTE_ARRAY, amount);
- }
- return atomicIncrement(tableName, row, famAndQf[0], famAndQf[1], amount);
- }
-
- protected long atomicIncrement(ByteBuffer tableName, ByteBuffer row,
- byte [] family, byte [] qualifier, long amount)
- throws IOError, IllegalArgument, TException {
- Table table = null;
- try {
- table = getTable(tableName);
- return table.incrementColumnValue(
- getBytes(row), family, qualifier, amount);
- } catch (IOException e) {
- LOG.warn(e.getMessage(), e);
- throw getIOError(e);
- } finally {
- closeTable(table);
- }
- }
-
- @Override
- public void scannerClose(int id) throws IOError, IllegalArgument {
- LOG.debug("scannerClose: id={}", id);
- ResultScannerWrapper resultScannerWrapper = getScanner(id);
- if (resultScannerWrapper == null) {
- LOG.warn("scanner ID is invalid");
- throw new IllegalArgument("scanner ID is invalid");
- }
- resultScannerWrapper.getScanner().close();
- removeScanner(id);
- }
-
- @Override
- public List<TRowResult> scannerGetList(int id,int nbRows)
- throws IllegalArgument, IOError {
- LOG.debug("scannerGetList: id={}", id);
- ResultScannerWrapper resultScannerWrapper = getScanner(id);
- if (null == resultScannerWrapper) {
- String message = "scanner ID is invalid";
- LOG.warn(message);
- throw new IllegalArgument("scanner ID is invalid");
- }
-
- Result [] results;
- try {
- results = resultScannerWrapper.getScanner().next(nbRows);
- if (null == results) {
- return new ArrayList<>();
- }
- } catch (IOException e) {
- LOG.warn(e.getMessage(), e);
- throw getIOError(e);
- }
- return ThriftUtilities.rowResultFromHBase(results, resultScannerWrapper.isColumnSorted());
- }
-
- @Override
- public List<TRowResult> scannerGet(int id) throws IllegalArgument, IOError {
- return scannerGetList(id,1);
- }
-
- @Override
- public int scannerOpenWithScan(ByteBuffer tableName, TScan tScan,
- Map<ByteBuffer, ByteBuffer> attributes)
- throws IOError {
-
- Table table = null;
- try {
- table = getTable(tableName);
- Scan scan = new Scan();
- addAttributes(scan, attributes);
- if (tScan.isSetStartRow()) {
- scan.setStartRow(tScan.getStartRow());
- }
- if (tScan.isSetStopRow()) {
- scan.setStopRow(tScan.getStopRow());
- }
- if (tScan.isSetTimestamp()) {
- scan.setTimeRange(0, tScan.getTimestamp());
- }
- if (tScan.isSetCaching()) {
- scan.setCaching(tScan.getCaching());
- }
- if (tScan.isSetBatchSize()) {
- scan.setBatch(tScan.getBatchSize());
- }
- if (tScan.isSetColumns() && !tScan.getColumns().isEmpty()) {
- for(ByteBuffer column : tScan.getColumns()) {
- byte [][] famQf = CellUtil.parseColumn(getBytes(column));
- if(famQf.length == 1) {
- scan.addFamily(famQf[0]);
- } else {
- scan.addColumn(famQf[0], famQf[1]);
- }
- }
- }
- if (tScan.isSetFilterString()) {
- ParseFilter parseFilter = new ParseFilter();
- scan.setFilter(
- parseFilter.parseFilterString(tScan.getFilterString()));
- }
- if (tScan.isSetReversed()) {
- scan.setReversed(tScan.isReversed());
- }
- if (tScan.isSetCacheBlocks()) {
- scan.setCacheBlocks(tScan.isCacheBlocks());
- }
- return addScanner(table.getScanner(scan), tScan.sortColumns);
- } catch (IOException e) {
- LOG.warn(e.getMessage(), e);
- throw getIOError(e);
- } finally{
- closeTable(table);
- }
- }
-
- @Override
- public int scannerOpen(ByteBuffer tableName, ByteBuffer startRow,
- List<ByteBuffer> columns,
- Map<ByteBuffer, ByteBuffer> attributes) throws IOError {
-
- Table table = null;
- try {
- table = getTable(tableName);
- Scan scan = new Scan(getBytes(startRow));
- addAttributes(scan, attributes);
- if(columns != null && !columns.isEmpty()) {
- for(ByteBuffer column : columns) {
- byte [][] famQf = CellUtil.parseColumn(getBytes(column));
- if(famQf.length == 1) {
- scan.addFamily(famQf[0]);
- } else {
- scan.addColumn(famQf[0], famQf[1]);
- }
- }
- }
- return addScanner(table.getScanner(scan), false);
- } catch (IOException e) {
- LOG.warn(e.getMessage(), e);
- throw getIOError(e);
- } finally{
- closeTable(table);
- }
- }
-
- @Override
- public int scannerOpenWithStop(ByteBuffer tableName, ByteBuffer startRow,
- ByteBuffer stopRow, List<ByteBuffer> columns,
- Map<ByteBuffer, ByteBuffer> attributes)
- throws IOError, TException {
-
- Table table = null;
- try {
- table = getTable(tableName);
- Scan scan = new Scan(getBytes(startRow), getBytes(stopRow));
- addAttributes(scan, attributes);
- if(columns != null && !columns.isEmpty()) {
- for(ByteBuffer column : columns) {
- byte [][] famQf = CellUtil.parseColumn(getBytes(column));
- if(famQf.length == 1) {
- scan.addFamily(famQf[0]);
- } else {
- scan.addColumn(famQf[0], famQf[1]);
- }
- }
- }
- return addScanner(table.getScanner(scan), false);
- } catch (IOException e) {
- LOG.warn(e.getMessage(), e);
- throw getIOError(e);
- } finally{
- closeTable(table);
- }
- }
-
- @Override
- public int scannerOpenWithPrefix(ByteBuffer tableName,
- ByteBuffer startAndPrefix,
- List<ByteBuffer> columns,
- Map<ByteBuffer, ByteBuffer> attributes)
- throws IOError, TException {
-
- Table table = null;
- try {
- table = getTable(tableName);
- Scan scan = new Scan(getBytes(startAndPrefix));
- addAttributes(scan, attributes);
- Filter f = new WhileMatchFilter(
- new PrefixFilter(getBytes(startAndPrefix)));
- scan.setFilter(f);
- if (columns != null && !columns.isEmpty()) {
- for(ByteBuffer column : columns) {
- byte [][] famQf = CellUtil.parseColumn(getBytes(column));
- if(famQf.length == 1) {
- scan.addFamily(famQf[0]);
- } else {
- scan.addColumn(famQf[0], famQf[1]);
- }
- }
- }
- return addScanner(table.getScanner(scan), false);
- } catch (IOException e) {
- LOG.warn(e.getMessage(), e);
- throw getIOError(e);
- } finally{
- closeTable(table);
- }
- }
-
- @Override
- 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);
- Scan scan = new Scan(getBytes(startRow));
- addAttributes(scan, attributes);
- scan.setTimeRange(0, timestamp);
- if (columns != null && !columns.isEmpty()) {
- for (ByteBuffer column : columns) {
- byte [][] famQf = CellUtil.parseColumn(getBytes(column));
- if(famQf.length == 1) {
- scan.addFamily(famQf[0]);
- } else {
- scan.addColumn(famQf[0], famQf[1]);
- }
- }
- }
- return addScanner(table.getScanner(scan), false);
- } catch (IOException e) {
- LOG.warn(e.getMessage(), e);
- throw getIOError(e);
- } finally{
- closeTable(table);
- }
- }
-
- @Override
- public int scannerOpenWithStopTs(ByteBuffer tableName, ByteBuffer startRow,
- ByteBuffer stopRow, List<ByteBuffer> columns, long timestamp,
- Map<ByteBuffer, ByteBuffer> attributes)
- throws IOError, TException {
-
- Table table = null;
- try {
- table = getTable(tableName);
- Scan scan = new Scan(getBytes(startRow), getBytes(stopRow));
- addAttributes(scan, attributes);
- scan.setTimeRange(0, timestamp);
- if (columns != null && !columns.isEmpty()) {
- for (ByteBuffer column : columns) {
- byte [][] famQf = CellUtil.parseColumn(getBytes(column));
- if(famQf.length == 1) {
- scan.addFamily(famQf[0]);
- } else {
- scan.addColumn(famQf[0], famQf[1]);
- }
- }
- }
- scan.setTimeRange(0, timestamp);
- return addScanner(table.getScanner(scan), false);
- } catch (IOException e) {
- LOG.warn(e.getMessage(), e);
- throw getIOError(e);
- } finally{
- closeTable(table);
- }
- }
-
- @Override
- public Map<ByteBuffer, ColumnDescriptor> getColumnDescriptors(
- ByteBuffer tableName) throws IOError, TException {
-
- Table table = null;
- try {
- TreeMap<ByteBuffer, ColumnDescriptor> columns = new TreeMap<>();
-
- table = getTable(tableName);
- HTableDescriptor desc = table.getTableDescriptor();
-
- for (HColumnDescriptor e : desc.getFamilies()) {
- ColumnDescriptor col = ThriftUtilities.colDescFromHbase(e);
- columns.put(col.name, col);
- }
- return columns;
- } catch (IOException e) {
- LOG.warn(e.getMessage(), e);
- throw getIOError(e);
- } finally {
- closeTable(table);
- }
- }
-
- private void closeTable(Table table) throws IOError {
- try{
- if(table != null){
- table.close();
- }
- } catch (IOException e){
- LOG.error(e.getMessage(), e);
- throw getIOError(e);
- }
- }
-
- @Override
- public TRegionInfo getRegionInfo(ByteBuffer searchRow) throws IOError {
- try {
- byte[] row = getBytes(searchRow);
- Result startRowResult = getReverseScanResult(TableName.META_TABLE_NAME.getName(), row,
- HConstants.CATALOG_FAMILY);
-
- if (startRowResult == null) {
- throw new IOException("Cannot find row in "+ TableName.META_TABLE_NAME+", row="
- + Bytes.toStringBinary(row));
- }
-
- // find region start and end keys
- RegionInfo regionInfo = MetaTableAccessor.getRegionInfo(startRowResult);
- if (regionInfo == null) {
- throw new IOException("RegionInfo REGIONINFO was null or " +
- " empty in Meta for row="
- + Bytes.toStringBinary(row));
- }
- TRegionInfo region = new TRegionInfo();
- region.setStartKey(regionInfo.getStartKey());
- region.setEndKey(regionInfo.getEndKey());
- region.id = regionInfo.getRegionId();
- region.setName(regionInfo.getRegionName());
- region.version = HREGION_VERSION; // version not used anymore, PB encoding used.
-
- // find region assignment to server
- ServerName serverName = MetaTableAccessor.getServerName(startRowResult, 0);
- if (serverName != null) {
- region.setServerName(Bytes.toBytes(serverName.getHostname()));
- region.port = serverName.getPort();
- }
- return region;
- } catch (IOException e) {
- LOG.warn(e.getMessage(), e);
- throw getIOError(e);
- }
- }
-
- private Result getReverseScanResult(byte[] tableName, byte[] row, byte[] family)
- throws IOException {
- Scan scan = new Scan(row);
- scan.setReversed(true);
- scan.addFamily(family);
- scan.setStartRow(row);
- try (Table table = getTable(tableName);
- ResultScanner scanner = table.getScanner(scan)) {
- return scanner.next();
- }
- }
-
- private void initMetrics(ThriftMetrics metrics) {
- this.metrics = metrics;
- }
-
- @Override
- public void increment(TIncrement tincrement) throws IOError, TException {
-
- if (tincrement.getRow().length == 0 || tincrement.getTable().length == 0) {
- throw new TException("Must supply a table and a row key; can't increment");
- }
-
- if (conf.getBoolean(COALESCE_INC_KEY, false)) {
- this.coalescer.queueIncrement(tincrement);
- return;
- }
-
- Table table = null;
- try {
- table = getTable(tincrement.getTable());
- Increment inc = ThriftUtilities.incrementFromThrift(tincrement);
- table.increment(inc);
- } catch (IOException e) {
- LOG.warn(e.getMessage(), e);
- throw getIOError(e);
- } finally{
- closeTable(table);
- }
- }
-
- @Override
- public void incrementRows(List<TIncrement> tincrements) throws IOError, TException {
- if (conf.getBoolean(COALESCE_INC_KEY, false)) {
- this.coalescer.queueIncrements(tincrements);
- return;
- }
- for (TIncrement tinc : tincrements) {
- increment(tinc);
- }
- }
-
- @Override
- public List<TCell> append(TAppend tappend) throws IOError, TException {
- if (tappend.getRow().length == 0 || tappend.getTable().length == 0) {
- throw new TException("Must supply a table and a row key; can't append");
- }
-
- Table table = null;
- try {
- table = getTable(tappend.getTable());
- Append append = ThriftUtilities.appendFromThrift(tappend);
- Result result = table.append(append);
- return ThriftUtilities.cellFromHBase(result.rawCells());
- } catch (IOException e) {
- LOG.warn(e.getMessage(), e);
- throw getIOError(e);
- } finally{
- closeTable(table);
- }
- }
-
- @Override
- public boolean checkAndPut(ByteBuffer tableName, ByteBuffer row, ByteBuffer column,
- ByteBuffer value, Mutation mput, Map<ByteBuffer, ByteBuffer> attributes) throws IOError,
- IllegalArgument, TException {
- Put put;
- try {
- put = new Put(getBytes(row), HConstants.LATEST_TIMESTAMP);
- addAttributes(put, attributes);
-
- byte[][] famAndQf = CellUtil.parseColumn(getBytes(mput.column));
- put.add(CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY)
- .setRow(put.getRow())
- .setFamily(famAndQf[0])
- .setQualifier(famAndQf[1])
- .setTimestamp(put.getTimestamp())
- .setType(Type.Put)
- .setValue(mput.value != null ? getBytes(mput.value)
- : HConstants.EMPTY_BYTE_ARRAY)
- .build());
- put.setDurability(mput.writeToWAL ? Durability.SYNC_WAL : Durability.SKIP_WAL);
- } catch (IOException | IllegalArgumentException e) {
- LOG.warn(e.getMessage(), e);
- throw new IllegalArgument(Throwables.getStackTraceAsString(e));
- }
-
- Table table = null;
- try {
- table = getTable(tableName);
- byte[][] famAndQf = CellUtil.parseColumn(getBytes(column));
- Table.CheckAndMutateBuilder mutateBuilder =
- table.checkAndMutate(getBytes(row), famAndQf[0]).qualifier(famAndQf[1]);
- if (value != null) {
- return mutateBuilder.ifEquals(getBytes(value)).thenPut(put);
- } else {
- return mutateBuilder.ifNotExists().thenPut(put);
- }
- } catch (IOException e) {
- LOG.warn(e.getMessage(), e);
- throw getIOError(e);
- } catch (IllegalArgumentException e) {
- LOG.warn(e.getMessage(), e);
- throw new IllegalArgument(Throwables.getStackTraceAsString(e));
- } finally {
- closeTable(table);
- }
- }
- }
-
- private static IOError getIOError(Throwable throwable) {
- IOError error = new IOErrorWithCause(throwable);
- error.setMessage(Throwables.getStackTraceAsString(throwable));
- return error;
- }
-
- /**
- * Adds all the attributes into the Operation object
- */
- private static void addAttributes(OperationWithAttributes op,
- Map<ByteBuffer, ByteBuffer> attributes) {
- if (attributes == null || attributes.isEmpty()) {
- return;
- }
- for (Map.Entry<ByteBuffer, ByteBuffer> entry : attributes.entrySet()) {
- String name = Bytes.toStringBinary(getBytes(entry.getKey()));
- byte[] value = getBytes(entry.getValue());
- op.setAttribute(name, value);
- }
- }
-
- public static void registerFilters(Configuration conf) {
- String[] filters = conf.getStrings("hbase.thrift.filters");
- Splitter splitter = Splitter.on(':');
- if(filters != null) {
- for(String filterClass: filters) {
- List<String> filterPart = splitter.splitToList(filterClass);
- if(filterPart.size() != 2) {
- LOG.warn("Invalid filter specification " + filterClass + " - skipping");
- } else {
- ParseFilter.registerFilter(filterPart.get(0), filterPart.get(1));
- }
- }
- }
- }
-
- public static class IOErrorWithCause extends IOError {
- private final Throwable cause;
- public IOErrorWithCause(Throwable cause) {
- this.cause = cause;
- }
-
- @Override
- public synchronized Throwable getCause() {
- return cause;
- }
-
- @Override
- public boolean equals(Object other) {
- if (super.equals(other) &&
- other instanceof IOErrorWithCause) {
- Throwable otherCause = ((IOErrorWithCause) other).getCause();
- if (this.getCause() != null) {
- return otherCause != null && this.getCause().equals(otherCause);
- } else {
- return otherCause == null;
- }
- }
- return false;
- }
-
- @Override
- public int hashCode() {
- int result = super.hashCode();
- result = 31 * result + (cause != null ? cause.hashCode() : 0);
- return result;
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/hbase/blob/e4b6b4af/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftHBaseServiceHandler.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftHBaseServiceHandler.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftHBaseServiceHandler.java
index 2bfeefe..a9ec646 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftHBaseServiceHandler.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftHBaseServiceHandler.java
@@ -18,6 +18,8 @@
*/
package org.apache.hadoop.hbase.thrift2;
+import static org.apache.hadoop.hbase.thrift.Constants.THRIFT_READONLY_ENABLED;
+import static org.apache.hadoop.hbase.thrift.Constants.THRIFT_READONLY_ENABLED_DEFAULT;
import static org.apache.hadoop.hbase.thrift2.ThriftUtilities.appendFromThrift;
import static org.apache.hadoop.hbase.thrift2.ThriftUtilities.columnFamilyDescriptorFromThrift;
import static org.apache.hadoop.hbase.thrift2.ThriftUtilities.compareOpFromThrift;
@@ -44,10 +46,6 @@ import static org.apache.hadoop.hbase.thrift2.ThriftUtilities.tableNamesFromHBas
import static org.apache.thrift.TBaseHelper.byteBufferToByteArray;
import java.io.IOException;
-import java.lang.reflect.InvocationHandler;
-import java.lang.reflect.InvocationTargetException;
-import java.lang.reflect.Method;
-import java.lang.reflect.Proxy;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.Collections;
@@ -68,7 +66,7 @@ import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.security.UserProvider;
-import org.apache.hadoop.hbase.thrift.ThriftMetrics;
+import org.apache.hadoop.hbase.thrift.HBaseServiceHandler;
import org.apache.hadoop.hbase.thrift2.generated.TAppend;
import org.apache.hadoop.hbase.thrift2.generated.TColumnFamilyDescriptor;
import org.apache.hadoop.hbase.thrift2.generated.TCompareOp;
@@ -87,7 +85,6 @@ import org.apache.hadoop.hbase.thrift2.generated.TScan;
import org.apache.hadoop.hbase.thrift2.generated.TTableDescriptor;
import org.apache.hadoop.hbase.thrift2.generated.TTableName;
import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.ConnectionCache;
import org.apache.thrift.TException;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
@@ -99,7 +96,7 @@ import org.slf4j.LoggerFactory;
*/
@InterfaceAudience.Private
@SuppressWarnings("deprecation")
-public class ThriftHBaseServiceHandler implements THBaseService.Iface {
+public class ThriftHBaseServiceHandler extends HBaseServiceHandler implements THBaseService.Iface {
// TODO: Size of pool configuraple
private static final Logger LOG = LoggerFactory.getLogger(ThriftHBaseServiceHandler.class);
@@ -109,50 +106,10 @@ public class ThriftHBaseServiceHandler implements THBaseService.Iface {
private final AtomicInteger nextScannerId = new AtomicInteger(0);
private final Map<Integer, ResultScanner> scannerMap = new ConcurrentHashMap<>();
- private final ConnectionCache connectionCache;
-
- static final String CLEANUP_INTERVAL = "hbase.thrift.connection.cleanup-interval";
- static final String MAX_IDLETIME = "hbase.thrift.connection.max-idletime";
-
private static final IOException ioe
= new DoNotRetryIOException("Thrift Server is in Read-only mode.");
private boolean isReadOnly;
- public static THBaseService.Iface newInstance(
- THBaseService.Iface handler, ThriftMetrics metrics) {
- return (THBaseService.Iface) Proxy.newProxyInstance(handler.getClass().getClassLoader(),
- new Class[] { THBaseService.Iface.class }, new THBaseServiceMetricsProxy(handler, metrics));
- }
-
- private static final class THBaseServiceMetricsProxy implements InvocationHandler {
- private final THBaseService.Iface handler;
- private final ThriftMetrics metrics;
-
- private THBaseServiceMetricsProxy(THBaseService.Iface handler, ThriftMetrics metrics) {
- this.handler = handler;
- this.metrics = metrics;
- }
-
- @Override
- public Object invoke(Object proxy, Method m, Object[] args) throws Throwable {
- Object result;
- long start = now();
- try {
- result = m.invoke(handler, args);
- } catch (InvocationTargetException e) {
- metrics.exception(e.getCause());
- throw e.getTargetException();
- } catch (Exception e) {
- metrics.exception(e);
- throw new RuntimeException("unexpected invocation exception: " + e.getMessage());
- } finally {
- long processTime = now() - start;
- metrics.incMethodTime(m.getName(), processTime);
- }
- return result;
- }
- }
-
private static class TIOErrorWithCause extends TIOError {
private Throwable cause;
@@ -188,20 +145,14 @@ public class ThriftHBaseServiceHandler implements THBaseService.Iface {
}
}
- private static long now() {
- return System.nanoTime();
- }
-
ThriftHBaseServiceHandler(final Configuration conf,
final UserProvider userProvider) throws IOException {
- int cleanInterval = conf.getInt(CLEANUP_INTERVAL, 10 * 1000);
- int maxIdleTime = conf.getInt(MAX_IDLETIME, 10 * 60 * 1000);
- connectionCache = new ConnectionCache(
- conf, userProvider, cleanInterval, maxIdleTime);
- isReadOnly = conf.getBoolean("hbase.thrift.readonly", false);
+ super(conf, userProvider);
+ isReadOnly = conf.getBoolean(THRIFT_READONLY_ENABLED, THRIFT_READONLY_ENABLED_DEFAULT);
}
- private Table getTable(ByteBuffer tableName) {
+ @Override
+ protected Table getTable(ByteBuffer tableName) {
try {
return connectionCache.getTable(Bytes.toString(byteBufferToByteArray(tableName)));
} catch (IOException ie) {
@@ -251,10 +202,6 @@ public class ThriftHBaseServiceHandler implements THBaseService.Iface {
return scannerMap.get(id);
}
- void setEffectiveUser(String effectiveUser) {
- connectionCache.setEffectiveUser(effectiveUser);
- }
-
/**
* Removes the scanner associated with the specified ID from the internal HashMap.
* @param id of the Scanner to remove
[04/15] hbase git commit: HBASE-21652 Refactor ThriftServer making
thrift2 server inherited from thrift1 server
Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/e4b6b4af/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServer.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServer.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServer.java
index fc00327..6d11ac6 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServer.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServer.java
@@ -18,16 +18,132 @@
package org.apache.hadoop.hbase.thrift;
+import static org.apache.hadoop.hbase.thrift.Constants.BACKLOG_CONF_DEAFULT;
+import static org.apache.hadoop.hbase.thrift.Constants.BACKLOG_CONF_KEY;
+import static org.apache.hadoop.hbase.thrift.Constants.BIND_CONF_KEY;
+import static org.apache.hadoop.hbase.thrift.Constants.BIND_OPTION;
+import static org.apache.hadoop.hbase.thrift.Constants.COMPACT_CONF_DEFAULT;
+import static org.apache.hadoop.hbase.thrift.Constants.COMPACT_CONF_KEY;
+import static org.apache.hadoop.hbase.thrift.Constants.COMPACT_OPTION;
+import static org.apache.hadoop.hbase.thrift.Constants.DEFAULT_BIND_ADDR;
+import static org.apache.hadoop.hbase.thrift.Constants.DEFAULT_HTTP_MAX_HEADER_SIZE;
+import static org.apache.hadoop.hbase.thrift.Constants.DEFAULT_LISTEN_PORT;
+import static org.apache.hadoop.hbase.thrift.Constants.FRAMED_CONF_DEFAULT;
+import static org.apache.hadoop.hbase.thrift.Constants.FRAMED_CONF_KEY;
+import static org.apache.hadoop.hbase.thrift.Constants.FRAMED_OPTION;
+import static org.apache.hadoop.hbase.thrift.Constants.HTTP_MAX_THREADS_KEY;
+import static org.apache.hadoop.hbase.thrift.Constants.HTTP_MAX_THREADS_KEY_DEFAULT;
+import static org.apache.hadoop.hbase.thrift.Constants.HTTP_MIN_THREADS_KEY;
+import static org.apache.hadoop.hbase.thrift.Constants.HTTP_MIN_THREADS_KEY_DEFAULT;
+import static org.apache.hadoop.hbase.thrift.Constants.INFOPORT_OPTION;
+import static org.apache.hadoop.hbase.thrift.Constants.KEEP_ALIVE_SEC_OPTION;
+import static org.apache.hadoop.hbase.thrift.Constants.MAX_FRAME_SIZE_CONF_DEFAULT;
+import static org.apache.hadoop.hbase.thrift.Constants.MAX_FRAME_SIZE_CONF_KEY;
+import static org.apache.hadoop.hbase.thrift.Constants.MAX_QUEUE_SIZE_OPTION;
+import static org.apache.hadoop.hbase.thrift.Constants.MAX_WORKERS_OPTION;
+import static org.apache.hadoop.hbase.thrift.Constants.MIN_WORKERS_OPTION;
+import static org.apache.hadoop.hbase.thrift.Constants.PORT_CONF_KEY;
+import static org.apache.hadoop.hbase.thrift.Constants.PORT_OPTION;
+import static org.apache.hadoop.hbase.thrift.Constants.READ_TIMEOUT_OPTION;
+import static org.apache.hadoop.hbase.thrift.Constants.SELECTOR_NUM_OPTION;
+import static org.apache.hadoop.hbase.thrift.Constants.THRIFT_DNS_INTERFACE_KEY;
+import static org.apache.hadoop.hbase.thrift.Constants.THRIFT_DNS_NAMESERVER_KEY;
+import static org.apache.hadoop.hbase.thrift.Constants.THRIFT_FILTERS;
+import static org.apache.hadoop.hbase.thrift.Constants.THRIFT_HTTP_ALLOW_OPTIONS_METHOD;
+import static org.apache.hadoop.hbase.thrift.Constants.THRIFT_HTTP_ALLOW_OPTIONS_METHOD_DEFAULT;
+import static org.apache.hadoop.hbase.thrift.Constants.THRIFT_INFO_SERVER_BINDING_ADDRESS;
+import static org.apache.hadoop.hbase.thrift.Constants.THRIFT_INFO_SERVER_BINDING_ADDRESS_DEFAULT;
+import static org.apache.hadoop.hbase.thrift.Constants.THRIFT_INFO_SERVER_PORT;
+import static org.apache.hadoop.hbase.thrift.Constants.THRIFT_INFO_SERVER_PORT_DEFAULT;
+import static org.apache.hadoop.hbase.thrift.Constants.THRIFT_KERBEROS_PRINCIPAL_KEY;
+import static org.apache.hadoop.hbase.thrift.Constants.THRIFT_KEYTAB_FILE_KEY;
+import static org.apache.hadoop.hbase.thrift.Constants.THRIFT_QOP_KEY;
+import static org.apache.hadoop.hbase.thrift.Constants.THRIFT_SELECTOR_NUM;
+import static org.apache.hadoop.hbase.thrift.Constants.THRIFT_SERVER_SOCKET_READ_TIMEOUT_DEFAULT;
+import static org.apache.hadoop.hbase.thrift.Constants.THRIFT_SERVER_SOCKET_READ_TIMEOUT_KEY;
+import static org.apache.hadoop.hbase.thrift.Constants.THRIFT_SSL_ENABLED_KEY;
+import static org.apache.hadoop.hbase.thrift.Constants.THRIFT_SSL_EXCLUDE_CIPHER_SUITES_KEY;
+import static org.apache.hadoop.hbase.thrift.Constants.THRIFT_SSL_EXCLUDE_PROTOCOLS_KEY;
+import static org.apache.hadoop.hbase.thrift.Constants.THRIFT_SSL_INCLUDE_CIPHER_SUITES_KEY;
+import static org.apache.hadoop.hbase.thrift.Constants.THRIFT_SSL_INCLUDE_PROTOCOLS_KEY;
+import static org.apache.hadoop.hbase.thrift.Constants.THRIFT_SSL_KEYSTORE_KEYPASSWORD_KEY;
+import static org.apache.hadoop.hbase.thrift.Constants.THRIFT_SSL_KEYSTORE_PASSWORD_KEY;
+import static org.apache.hadoop.hbase.thrift.Constants.THRIFT_SSL_KEYSTORE_STORE_KEY;
+import static org.apache.hadoop.hbase.thrift.Constants.THRIFT_SUPPORT_PROXYUSER_KEY;
+import static org.apache.hadoop.hbase.thrift.Constants.USE_HTTP_CONF_KEY;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.net.UnknownHostException;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.ExecutorService;
+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.SaslServer;
+
+import org.apache.commons.lang3.ArrayUtils;
import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
+import org.apache.hadoop.hbase.filter.ParseFilter;
+import org.apache.hadoop.hbase.http.HttpServerUtil;
import org.apache.hadoop.hbase.http.InfoServer;
-import org.apache.hadoop.hbase.thrift.ThriftServerRunner.ImplType;
+import org.apache.hadoop.hbase.security.SaslUtil;
+import org.apache.hadoop.hbase.security.SecurityUtil;
+import org.apache.hadoop.hbase.security.UserProvider;
+import org.apache.hadoop.hbase.thrift.generated.Hbase;
+import org.apache.hadoop.hbase.util.DNS;
+import org.apache.hadoop.hbase.util.JvmPauseMonitor;
+import org.apache.hadoop.hbase.util.Strings;
import org.apache.hadoop.hbase.util.VersionInfo;
+import org.apache.hadoop.security.SaslRpcServer;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.authorize.ProxyUsers;
import org.apache.hadoop.util.Shell.ExitCodeException;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+import org.apache.thrift.TProcessor;
+import org.apache.thrift.protocol.TBinaryProtocol;
+import org.apache.thrift.protocol.TCompactProtocol;
+import org.apache.thrift.protocol.TProtocolFactory;
+import org.apache.thrift.server.THsHaServer;
+import org.apache.thrift.server.TNonblockingServer;
+import org.apache.thrift.server.TServer;
+import org.apache.thrift.server.TServlet;
+import org.apache.thrift.server.TThreadedSelectorServer;
+import org.apache.thrift.transport.TFramedTransport;
+import org.apache.thrift.transport.TNonblockingServerSocket;
+import org.apache.thrift.transport.TNonblockingServerTransport;
+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.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.slf4j.Logger;
import org.slf4j.LoggerFactory;
+
+import org.apache.hbase.thirdparty.com.google.common.base.Joiner;
+import org.apache.hbase.thirdparty.com.google.common.base.Splitter;
+import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLine;
import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLineParser;
import org.apache.hbase.thirdparty.org.apache.commons.cli.DefaultParser;
@@ -40,29 +156,36 @@ import org.apache.hbase.thirdparty.org.apache.commons.cli.Options;
* independent process.
*/
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.TOOLS)
-public class ThriftServer {
+public class ThriftServer extends Configured implements Tool {
private static final Logger LOG = LoggerFactory.getLogger(ThriftServer.class);
- private static final String MIN_WORKERS_OPTION = "minWorkers";
- private static final String MAX_WORKERS_OPTION = "workers";
- private static final String MAX_QUEUE_SIZE_OPTION = "queue";
- private static final String KEEP_ALIVE_SEC_OPTION = "keepAliveSec";
- static final String BIND_OPTION = "bind";
- static final String COMPACT_OPTION = "compact";
- static final String FRAMED_OPTION = "framed";
- static final String PORT_OPTION = "port";
- static final String INFOPORT_OPTION = "infoport";
- private static final String DEFAULT_BIND_ADDR = "0.0.0.0";
- private static final int DEFAULT_LISTEN_PORT = 9090;
- private Configuration conf;
- ThriftServerRunner serverRunner;
+ protected Configuration conf;
+
+ protected InfoServer infoServer;
+
+ protected TProcessor processor;
+
+ protected ThriftMetrics metrics;
+ protected HBaseServiceHandler hbaseServiceHandler;
+ protected UserGroupInformation serviceUGI;
+ protected boolean httpEnabled;
+
+ protected SaslUtil.QualityOfProtection qop;
+ protected String host;
+ protected int listenPort;
+
+
+ protected boolean securityEnabled;
+ protected boolean doAsEnabled;
+
+ protected JvmPauseMonitor pauseMonitor;
- private InfoServer infoServer;
+ protected volatile TServer tserver;
+ protected volatile Server httpServer;
- private static final String READ_TIMEOUT_OPTION = "readTimeout";
//
// Main program and support routines
@@ -72,7 +195,89 @@ public class ThriftServer {
this.conf = HBaseConfiguration.create(conf);
}
- private static void printUsageAndExit(Options options, int exitCode)
+ protected void setupParamters() throws IOException {
+ // login the server principal (if using secure Hadoop)
+ UserProvider userProvider = UserProvider.instantiate(conf);
+ securityEnabled = userProvider.isHadoopSecurityEnabled()
+ && userProvider.isHBaseSecurityEnabled();
+ if (securityEnabled) {
+ host = Strings.domainNamePointerToHostName(DNS.getDefaultHost(
+ conf.get(THRIFT_DNS_INTERFACE_KEY, "default"),
+ conf.get(THRIFT_DNS_NAMESERVER_KEY, "default")));
+ userProvider.login(THRIFT_KEYTAB_FILE_KEY, THRIFT_KERBEROS_PRINCIPAL_KEY, host);
+ }
+ this.serviceUGI = userProvider.getCurrent().getUGI();
+
+ this.listenPort = conf.getInt(PORT_CONF_KEY, DEFAULT_LISTEN_PORT);
+ this.metrics = new ThriftMetrics(conf, ThriftMetrics.ThriftServerType.ONE);
+ this.pauseMonitor = new JvmPauseMonitor(conf, this.metrics.getSource());
+ this.hbaseServiceHandler = createHandler(conf, userProvider);
+ this.hbaseServiceHandler.initMetrics(metrics);
+ this.processor = createProcessor();
+
+ httpEnabled = conf.getBoolean(USE_HTTP_CONF_KEY, false);
+ doAsEnabled = conf.getBoolean(THRIFT_SUPPORT_PROXYUSER_KEY, false);
+ if (doAsEnabled && !httpEnabled) {
+ LOG.warn("Fail to enable the doAs feature. " + USE_HTTP_CONF_KEY + " is not configured");
+ }
+
+ String strQop = conf.get(THRIFT_QOP_KEY);
+ if (strQop != null) {
+ this.qop = SaslUtil.getQop(strQop);
+ }
+ if (qop != null) {
+ if (qop != SaslUtil.QualityOfProtection.AUTHENTICATION &&
+ qop != SaslUtil.QualityOfProtection.INTEGRITY &&
+ qop != SaslUtil.QualityOfProtection.PRIVACY) {
+ throw new IOException(String.format("Invalid %s: It must be one of %s, %s, or %s.",
+ THRIFT_QOP_KEY,
+ SaslUtil.QualityOfProtection.AUTHENTICATION.name(),
+ SaslUtil.QualityOfProtection.INTEGRITY.name(),
+ SaslUtil.QualityOfProtection.PRIVACY.name()));
+ }
+ checkHttpSecurity(qop, conf);
+ if (!securityEnabled) {
+ throw new IOException("Thrift server must run in secure mode to support authentication");
+ }
+ }
+ registerFilters(conf);
+ pauseMonitor.start();
+ }
+
+ protected void startInfoServer() throws IOException {
+ // Put up info server.
+ int port = conf.getInt(THRIFT_INFO_SERVER_PORT , THRIFT_INFO_SERVER_PORT_DEFAULT);
+
+ if (port >= 0) {
+ conf.setLong("startcode", System.currentTimeMillis());
+ String a = conf
+ .get(THRIFT_INFO_SERVER_BINDING_ADDRESS, THRIFT_INFO_SERVER_BINDING_ADDRESS_DEFAULT);
+ infoServer = new InfoServer("thrift", a, port, false, conf);
+ infoServer.setAttribute("hbase.conf", conf);
+ infoServer.start();
+ }
+ }
+
+ protected void checkHttpSecurity(SaslUtil.QualityOfProtection qop, Configuration conf) {
+ if (qop == SaslUtil.QualityOfProtection.PRIVACY &&
+ conf.getBoolean(USE_HTTP_CONF_KEY, false) &&
+ !conf.getBoolean(THRIFT_SSL_ENABLED_KEY, false)) {
+ throw new IllegalArgumentException("Thrift HTTP Server's QoP is privacy, but " +
+ THRIFT_SSL_ENABLED_KEY + " is false");
+ }
+ }
+
+ protected HBaseServiceHandler createHandler(Configuration conf, UserProvider userProvider)
+ throws IOException {
+ return new ThriftHBaseServiceHandler(conf, userProvider);
+ }
+
+ protected TProcessor createProcessor() {
+ return new Hbase.Processor<>(
+ HbaseHandlerMetricsProxy.newInstance((Hbase.Iface) hbaseServiceHandler, metrics, conf));
+ }
+
+ protected void printUsageAndExit(Options options, int exitCode)
throws ExitCodeException {
HelpFormatter formatter = new HelpFormatter();
formatter.printHelp("Thrift", null, options,
@@ -85,32 +290,333 @@ public class ThriftServer {
}
/**
- * Start up or shuts down the Thrift server, depending on the arguments.
- * @param args the arguments to pass in when starting the Thrift server
+ * Setup a HTTP Server using Jetty to serve calls from THttpClient
+ *
+ * @throws IOException IOException
*/
- void doMain(final String[] args) throws Exception {
- processOptions(args);
- serverRunner = new ThriftServerRunner(conf);
+ protected void setupHTTPServer() throws IOException {
+ TProtocolFactory protocolFactory = new TBinaryProtocol.Factory();
+ TServlet thriftHttpServlet = new ThriftHttpServlet(processor, protocolFactory, serviceUGI,
+ conf, hbaseServiceHandler, securityEnabled, doAsEnabled);
+
+ // Set the default max thread number to 100 to limit
+ // the number of concurrent requests so that Thrfit HTTP server doesn't OOM easily.
+ // Jetty set the default max thread number to 250, if we don't set it.
+ //
+ // Our default min thread number 2 is the same as that used by Jetty.
+ int minThreads = conf.getInt(HTTP_MIN_THREADS_KEY,
+ conf.getInt(TBoundedThreadPoolServer.MIN_WORKER_THREADS_CONF_KEY,
+ HTTP_MIN_THREADS_KEY_DEFAULT));
+ int maxThreads = conf.getInt(HTTP_MAX_THREADS_KEY,
+ conf.getInt(TBoundedThreadPoolServer.MAX_WORKER_THREADS_CONF_KEY,
+ HTTP_MAX_THREADS_KEY_DEFAULT));
+ QueuedThreadPool threadPool = new QueuedThreadPool(maxThreads);
+ threadPool.setMinThreads(minThreads);
+ httpServer = new Server(threadPool);
+
+ // Context handler
+ ServletContextHandler ctxHandler = new ServletContextHandler(httpServer, "/",
+ ServletContextHandler.SESSIONS);
+ ctxHandler.addServlet(new ServletHolder(thriftHttpServlet), "/*");
+ HttpServerUtil.constrainHttpMethods(ctxHandler,
+ conf.getBoolean(THRIFT_HTTP_ALLOW_OPTIONS_METHOD,
+ THRIFT_HTTP_ALLOW_OPTIONS_METHOD_DEFAULT));
+
+ // set up Jetty and run the embedded server
+ HttpConfiguration httpConfig = new HttpConfiguration();
+ httpConfig.setSecureScheme("https");
+ httpConfig.setSecurePort(listenPort);
+ httpConfig.setHeaderCacheSize(DEFAULT_HTTP_MAX_HEADER_SIZE);
+ httpConfig.setRequestHeaderSize(DEFAULT_HTTP_MAX_HEADER_SIZE);
+ httpConfig.setResponseHeaderSize(DEFAULT_HTTP_MAX_HEADER_SIZE);
+ httpConfig.setSendServerVersion(false);
+ httpConfig.setSendDateHeader(false);
+
+ ServerConnector serverConnector;
+ if(conf.getBoolean(THRIFT_SSL_ENABLED_KEY, false)) {
+ HttpConfiguration httpsConfig = new HttpConfiguration(httpConfig);
+ httpsConfig.addCustomizer(new SecureRequestCustomizer());
+
+ SslContextFactory sslCtxFactory = new SslContextFactory();
+ String keystore = conf.get(THRIFT_SSL_KEYSTORE_STORE_KEY);
+ String password = HBaseConfiguration.getPassword(conf,
+ THRIFT_SSL_KEYSTORE_PASSWORD_KEY, null);
+ String keyPassword = HBaseConfiguration.getPassword(conf,
+ THRIFT_SSL_KEYSTORE_KEYPASSWORD_KEY, password);
+ sslCtxFactory.setKeyStorePath(keystore);
+ sslCtxFactory.setKeyStorePassword(password);
+ sslCtxFactory.setKeyManagerPassword(keyPassword);
+
+ String[] excludeCiphers = conf.getStrings(
+ THRIFT_SSL_EXCLUDE_CIPHER_SUITES_KEY, ArrayUtils.EMPTY_STRING_ARRAY);
+ if (excludeCiphers.length != 0) {
+ sslCtxFactory.setExcludeCipherSuites(excludeCiphers);
+ }
+ String[] includeCiphers = conf.getStrings(
+ THRIFT_SSL_INCLUDE_CIPHER_SUITES_KEY, ArrayUtils.EMPTY_STRING_ARRAY);
+ if (includeCiphers.length != 0) {
+ sslCtxFactory.setIncludeCipherSuites(includeCiphers);
+ }
- // Put up info server.
- int port = conf.getInt("hbase.thrift.info.port", 9095);
+ // Disable SSLv3 by default due to "Poodle" Vulnerability - CVE-2014-3566
+ String[] excludeProtocols = conf.getStrings(
+ THRIFT_SSL_EXCLUDE_PROTOCOLS_KEY, "SSLv3");
+ if (excludeProtocols.length != 0) {
+ sslCtxFactory.setExcludeProtocols(excludeProtocols);
+ }
+ String[] includeProtocols = conf.getStrings(
+ THRIFT_SSL_INCLUDE_PROTOCOLS_KEY, ArrayUtils.EMPTY_STRING_ARRAY);
+ if (includeProtocols.length != 0) {
+ sslCtxFactory.setIncludeProtocols(includeProtocols);
+ }
- if (port >= 0) {
- conf.setLong("startcode", System.currentTimeMillis());
- String a = conf.get("hbase.thrift.info.bindAddress", "0.0.0.0");
- infoServer = new InfoServer("thrift", a, port, false, conf);
- infoServer.setAttribute("hbase.conf", conf);
- infoServer.start();
+ serverConnector = new ServerConnector(httpServer,
+ new SslConnectionFactory(sslCtxFactory, HttpVersion.HTTP_1_1.toString()),
+ new HttpConnectionFactory(httpsConfig));
+ } else {
+ serverConnector = new ServerConnector(httpServer, new HttpConnectionFactory(httpConfig));
}
+ serverConnector.setPort(listenPort);
+ serverConnector.setHost(getBindAddress(conf).getHostAddress());
+ httpServer.addConnector(serverConnector);
+ httpServer.setStopAtShutdown(true);
- serverRunner.run();
+ if (doAsEnabled) {
+ ProxyUsers.refreshSuperUserGroupsConfiguration(conf);
+ }
+ LOG.info("Starting Thrift HTTP Server on {}", Integer.toString(listenPort));
}
/**
- * Parse the command line options to set parameters the conf.
+ * Setting up the thrift TServer
*/
- private void processOptions(final String[] args) throws Exception {
- Options options = new Options();
+ protected void setupServer() throws Exception {
+ // Construct correct ProtocolFactory
+ TProtocolFactory protocolFactory = getProtocolFactory();
+
+ ImplType implType = ImplType.getServerImpl(conf);
+ TProcessor processorToUse = processor;
+
+ // Construct correct TransportFactory
+ TTransportFactory transportFactory;
+ if (conf.getBoolean(FRAMED_CONF_KEY, FRAMED_CONF_DEFAULT) || implType.isAlwaysFramed) {
+ if (qop != null) {
+ throw new RuntimeException("Thrift server authentication"
+ + " doesn't work with framed transport yet");
+ }
+ transportFactory = new TFramedTransport.Factory(
+ conf.getInt(MAX_FRAME_SIZE_CONF_KEY, MAX_FRAME_SIZE_CONF_DEFAULT) * 1024 * 1024);
+ LOG.debug("Using framed transport");
+ } else if (qop == null) {
+ transportFactory = new TTransportFactory();
+ } else {
+ // Extract the name from the principal
+ String thriftKerberosPrincipal = conf.get(THRIFT_KERBEROS_PRINCIPAL_KEY);
+ if (thriftKerberosPrincipal == null) {
+ throw new IllegalArgumentException(THRIFT_KERBEROS_PRINCIPAL_KEY + " cannot be null");
+ }
+ String name = SecurityUtil.getUserFromPrincipal(thriftKerberosPrincipal);
+ Map<String, String> saslProperties = SaslUtil.initSaslProperties(qop.name());
+ TSaslServerTransport.Factory saslFactory = new TSaslServerTransport.Factory();
+ saslFactory.addServerDefinition("GSSAPI", name, host, saslProperties,
+ new SaslRpcServer.SaslGssCallbackHandler() {
+ @Override
+ public void handle(Callback[] callbacks)
+ throws UnsupportedCallbackException {
+ AuthorizeCallback ac = null;
+ for (Callback callback : callbacks) {
+ if (callback instanceof AuthorizeCallback) {
+ ac = (AuthorizeCallback) callback;
+ } else {
+ throw new UnsupportedCallbackException(callback,
+ "Unrecognized SASL GSSAPI Callback");
+ }
+ }
+ if (ac != null) {
+ String authid = ac.getAuthenticationID();
+ String authzid = ac.getAuthorizationID();
+ if (!authid.equals(authzid)) {
+ ac.setAuthorized(false);
+ } else {
+ ac.setAuthorized(true);
+ String userName = SecurityUtil.getUserFromPrincipal(authzid);
+ LOG.info("Effective user: {}", userName);
+ ac.setAuthorizedID(userName);
+ }
+ }
+ }
+ });
+ transportFactory = saslFactory;
+
+ // Create a processor wrapper, to get the caller
+ processorToUse = (inProt, outProt) -> {
+ TSaslServerTransport saslServerTransport =
+ (TSaslServerTransport)inProt.getTransport();
+ SaslServer saslServer = saslServerTransport.getSaslServer();
+ String principal = saslServer.getAuthorizationID();
+ hbaseServiceHandler.setEffectiveUser(principal);
+ return processor.process(inProt, outProt);
+ };
+ }
+
+ if (conf.get(BIND_CONF_KEY) != null && !implType.canSpecifyBindIP) {
+ LOG.error("Server types {} don't support IP address binding at the moment. See " +
+ "https://issues.apache.org/jira/browse/HBASE-2155 for details.",
+ Joiner.on(", ").join(ImplType.serversThatCannotSpecifyBindIP()));
+ throw new RuntimeException("-" + BIND_CONF_KEY + " not supported with " + implType);
+ }
+
+ InetSocketAddress inetSocketAddress = new InetSocketAddress(getBindAddress(conf), listenPort);
+ if (implType == ImplType.HS_HA || implType == ImplType.NONBLOCKING ||
+ implType == ImplType.THREADED_SELECTOR) {
+ TNonblockingServerTransport serverTransport = new TNonblockingServerSocket(inetSocketAddress);
+ if (implType == ImplType.NONBLOCKING) {
+ tserver = getTNonBlockingServer(serverTransport, protocolFactory, processorToUse,
+ transportFactory, inetSocketAddress);
+ } else if (implType == ImplType.HS_HA) {
+ tserver = getTHsHaServer(serverTransport, protocolFactory, processorToUse, transportFactory,
+ inetSocketAddress);
+ } else { // THREADED_SELECTOR
+ tserver = getTThreadedSelectorServer(serverTransport, protocolFactory, processorToUse,
+ transportFactory, inetSocketAddress);
+ }
+ LOG.info("starting HBase {} server on {}", implType.simpleClassName(),
+ Integer.toString(listenPort));
+ } else if (implType == ImplType.THREAD_POOL) {
+ this.tserver = getTThreadPoolServer(protocolFactory, processorToUse, transportFactory,
+ inetSocketAddress);
+ } else {
+ throw new AssertionError("Unsupported Thrift server implementation: " +
+ implType.simpleClassName());
+ }
+
+ // A sanity check that we instantiated the right type of server.
+ if (tserver.getClass() != implType.serverClass) {
+ throw new AssertionError("Expected to create Thrift server class " +
+ implType.serverClass.getName() + " but got " +
+ tserver.getClass().getName());
+ }
+ }
+
+ private TServer getTNonBlockingServer(TNonblockingServerTransport serverTransport,
+ TProtocolFactory protocolFactory, TProcessor processor, TTransportFactory transportFactory,
+ InetSocketAddress inetSocketAddress) {
+ LOG.info("starting HBase Nonblocking Thrift server on " + inetSocketAddress.toString());
+ TNonblockingServer.Args serverArgs = new TNonblockingServer.Args(serverTransport);
+ serverArgs.processor(processor);
+ serverArgs.transportFactory(transportFactory);
+ serverArgs.protocolFactory(protocolFactory);
+ return new TNonblockingServer(serverArgs);
+ }
+
+ private TServer getTHsHaServer(TNonblockingServerTransport serverTransport,
+ TProtocolFactory protocolFactory, TProcessor processor, TTransportFactory transportFactory,
+ InetSocketAddress inetSocketAddress) {
+ LOG.info("starting HBase HsHA Thrift server on " + inetSocketAddress.toString());
+ THsHaServer.Args serverArgs = new THsHaServer.Args(serverTransport);
+ int queueSize = conf.getInt(TBoundedThreadPoolServer.MAX_QUEUED_REQUESTS_CONF_KEY,
+ TBoundedThreadPoolServer.DEFAULT_MAX_QUEUED_REQUESTS);
+ CallQueue callQueue = new CallQueue(new LinkedBlockingQueue<>(queueSize), metrics);
+ int workerThread = conf.getInt(TBoundedThreadPoolServer.MAX_WORKER_THREADS_CONF_KEY,
+ serverArgs.getMaxWorkerThreads());
+ ExecutorService executorService = createExecutor(
+ callQueue, workerThread, workerThread);
+ serverArgs.executorService(executorService).processor(processor)
+ .transportFactory(transportFactory).protocolFactory(protocolFactory);
+ return new THsHaServer(serverArgs);
+ }
+
+ private TServer getTThreadedSelectorServer(TNonblockingServerTransport serverTransport,
+ TProtocolFactory protocolFactory, TProcessor processor, TTransportFactory transportFactory,
+ InetSocketAddress inetSocketAddress) {
+ LOG.info("starting HBase ThreadedSelector Thrift server on " + inetSocketAddress.toString());
+ TThreadedSelectorServer.Args serverArgs =
+ new HThreadedSelectorServerArgs(serverTransport, conf);
+ int queueSize = conf.getInt(TBoundedThreadPoolServer.MAX_QUEUED_REQUESTS_CONF_KEY,
+ TBoundedThreadPoolServer.DEFAULT_MAX_QUEUED_REQUESTS);
+ CallQueue callQueue = new CallQueue(new LinkedBlockingQueue<>(queueSize), metrics);
+ int workerThreads = conf.getInt(TBoundedThreadPoolServer.MAX_WORKER_THREADS_CONF_KEY,
+ serverArgs.getWorkerThreads());
+ int selectorThreads = conf.getInt(THRIFT_SELECTOR_NUM, serverArgs.getSelectorThreads());
+ serverArgs.selectorThreads(selectorThreads);
+ ExecutorService executorService = createExecutor(
+ callQueue, workerThreads, workerThreads);
+ serverArgs.executorService(executorService).processor(processor)
+ .transportFactory(transportFactory).protocolFactory(protocolFactory);
+ return new TThreadedSelectorServer(serverArgs);
+ }
+
+ private TServer getTThreadPoolServer(TProtocolFactory protocolFactory, TProcessor processor,
+ TTransportFactory transportFactory, InetSocketAddress inetSocketAddress) throws Exception {
+ LOG.info("starting HBase ThreadPool Thrift server on " + inetSocketAddress.toString());
+ // Thrift's implementation uses '0' as a placeholder for 'use the default.'
+ int backlog = conf.getInt(BACKLOG_CONF_KEY, BACKLOG_CONF_DEAFULT);
+ int readTimeout = conf.getInt(THRIFT_SERVER_SOCKET_READ_TIMEOUT_KEY,
+ THRIFT_SERVER_SOCKET_READ_TIMEOUT_DEFAULT);
+ TServerTransport serverTransport = new TServerSocket(
+ new TServerSocket.ServerSocketTransportArgs().
+ bindAddr(inetSocketAddress).backlog(backlog).
+ clientTimeout(readTimeout));
+
+ TBoundedThreadPoolServer.Args serverArgs =
+ new TBoundedThreadPoolServer.Args(serverTransport, conf);
+ serverArgs.processor(processor).transportFactory(transportFactory)
+ .protocolFactory(protocolFactory);
+ return new TBoundedThreadPoolServer(serverArgs, metrics);
+ }
+
+ private TProtocolFactory getProtocolFactory() {
+ TProtocolFactory protocolFactory;
+
+ if (conf.getBoolean(COMPACT_CONF_KEY, COMPACT_CONF_DEFAULT)) {
+ LOG.debug("Using compact protocol");
+ protocolFactory = new TCompactProtocol.Factory();
+ } else {
+ LOG.debug("Using binary protocol");
+ protocolFactory = new TBinaryProtocol.Factory();
+ }
+
+ return protocolFactory;
+ }
+
+ ExecutorService createExecutor(BlockingQueue<Runnable> callQueue,
+ int minWorkers, int maxWorkers) {
+ ThreadFactoryBuilder tfb = new ThreadFactoryBuilder();
+ tfb.setDaemon(true);
+ tfb.setNameFormat("thrift-worker-%d");
+ ThreadPoolExecutor threadPool = new THBaseThreadPoolExecutor(minWorkers, maxWorkers,
+ Long.MAX_VALUE, TimeUnit.SECONDS, callQueue, tfb.build(), metrics);
+ threadPool.allowCoreThreadTimeOut(true);
+ return threadPool;
+ }
+
+ private InetAddress getBindAddress(Configuration conf)
+ throws UnknownHostException {
+ String bindAddressStr = conf.get(BIND_CONF_KEY, DEFAULT_BIND_ADDR);
+ return InetAddress.getByName(bindAddressStr);
+ }
+
+
+ public static void registerFilters(Configuration conf) {
+ String[] filters = conf.getStrings(THRIFT_FILTERS);
+ Splitter splitter = Splitter.on(':');
+ if(filters != null) {
+ for(String filterClass: filters) {
+ List<String> filterPart = splitter.splitToList(filterClass);
+ if(filterPart.size() != 2) {
+ LOG.warn("Invalid filter specification " + filterClass + " - skipping");
+ } else {
+ ParseFilter.registerFilter(filterPart.get(0), filterPart.get(1));
+ }
+ }
+ }
+ }
+
+ /**
+ * Add options to command lines
+ * @param options options
+ */
+ protected void addOptions(Options options) {
options.addOption("b", BIND_OPTION, true, "Address to bind " +
"the Thrift server to. [default: " + DEFAULT_BIND_ADDR + "]");
options.addOption("p", PORT_OPTION, true, "Port to bind to [default: " +
@@ -118,62 +624,56 @@ public class ThriftServer {
options.addOption("f", FRAMED_OPTION, false, "Use framed transport");
options.addOption("c", COMPACT_OPTION, false, "Use the compact protocol");
options.addOption("h", "help", false, "Print help information");
+ options.addOption("s", SELECTOR_NUM_OPTION, true, "How many selector threads to use.");
options.addOption(null, INFOPORT_OPTION, true, "Port for web UI");
options.addOption("m", MIN_WORKERS_OPTION, true,
"The minimum number of worker threads for " +
- ImplType.THREAD_POOL.simpleClassName());
+ ImplType.THREAD_POOL.simpleClassName());
options.addOption("w", MAX_WORKERS_OPTION, true,
"The maximum number of worker threads for " +
- ImplType.THREAD_POOL.simpleClassName());
+ ImplType.THREAD_POOL.simpleClassName());
options.addOption("q", MAX_QUEUE_SIZE_OPTION, true,
"The maximum number of queued requests in " +
- ImplType.THREAD_POOL.simpleClassName());
+ ImplType.THREAD_POOL.simpleClassName());
options.addOption("k", KEEP_ALIVE_SEC_OPTION, true,
"The amount of time in secods to keep a thread alive when idle in " +
- ImplType.THREAD_POOL.simpleClassName());
+ ImplType.THREAD_POOL.simpleClassName());
options.addOption("t", READ_TIMEOUT_OPTION, true,
"Amount of time in milliseconds before a server thread will timeout " +
- "waiting for client to send data on a connected socket. Currently, " +
- "only applies to TBoundedThreadPoolServer");
+ "waiting for client to send data on a connected socket. Currently, " +
+ "only applies to TBoundedThreadPoolServer");
options.addOptionGroup(ImplType.createOptionGroup());
+ }
- CommandLineParser parser = new DefaultParser();
- CommandLine cmd = parser.parse(options, args);
-
- if (cmd.hasOption("help")) {
- printUsageAndExit(options, 1);
- }
-
+ protected void parseCommandLine(CommandLine cmd, Options options) throws ExitCodeException {
// Get port to bind to
try {
if (cmd.hasOption(PORT_OPTION)) {
int listenPort = Integer.parseInt(cmd.getOptionValue(PORT_OPTION));
- conf.setInt(ThriftServerRunner.PORT_CONF_KEY, listenPort);
+ conf.setInt(PORT_CONF_KEY, listenPort);
}
} catch (NumberFormatException e) {
LOG.error("Could not parse the value provided for the port option", e);
printUsageAndExit(options, -1);
}
-
// check for user-defined info server port setting, if so override the conf
try {
if (cmd.hasOption(INFOPORT_OPTION)) {
String val = cmd.getOptionValue(INFOPORT_OPTION);
- conf.setInt("hbase.thrift.info.port", Integer.parseInt(val));
+ conf.setInt(THRIFT_INFO_SERVER_PORT, Integer.parseInt(val));
LOG.debug("Web UI port set to " + val);
}
} catch (NumberFormatException e) {
LOG.error("Could not parse the value provided for the " + INFOPORT_OPTION +
- " option", e);
+ " option", e);
printUsageAndExit(options, -1);
}
-
// Make optional changes to the configuration based on command-line options
optionToConf(cmd, MIN_WORKERS_OPTION,
conf, TBoundedThreadPoolServer.MIN_WORKER_THREADS_CONF_KEY);
@@ -183,23 +683,42 @@ public class ThriftServer {
conf, TBoundedThreadPoolServer.MAX_QUEUED_REQUESTS_CONF_KEY);
optionToConf(cmd, KEEP_ALIVE_SEC_OPTION,
conf, TBoundedThreadPoolServer.THREAD_KEEP_ALIVE_TIME_SEC_CONF_KEY);
- optionToConf(cmd, READ_TIMEOUT_OPTION, conf,
- ThriftServerRunner.THRIFT_SERVER_SOCKET_READ_TIMEOUT_KEY);
+ optionToConf(cmd, READ_TIMEOUT_OPTION, conf, THRIFT_SERVER_SOCKET_READ_TIMEOUT_KEY);
+ optionToConf(cmd, SELECTOR_NUM_OPTION, conf, THRIFT_SELECTOR_NUM);
// Set general thrift server options
boolean compact = cmd.hasOption(COMPACT_OPTION) ||
- conf.getBoolean(ThriftServerRunner.COMPACT_CONF_KEY, false);
- conf.setBoolean(ThriftServerRunner.COMPACT_CONF_KEY, compact);
+ conf.getBoolean(COMPACT_CONF_KEY, false);
+ conf.setBoolean(COMPACT_CONF_KEY, compact);
boolean framed = cmd.hasOption(FRAMED_OPTION) ||
- conf.getBoolean(ThriftServerRunner.FRAMED_CONF_KEY, false);
- conf.setBoolean(ThriftServerRunner.FRAMED_CONF_KEY, framed);
- if (cmd.hasOption(BIND_OPTION)) {
- conf.set(ThriftServerRunner.BIND_CONF_KEY, cmd.getOptionValue(BIND_OPTION));
- }
+ conf.getBoolean(FRAMED_CONF_KEY, false);
+ conf.setBoolean(FRAMED_CONF_KEY, framed);
+
+ optionToConf(cmd, BIND_OPTION, conf, BIND_CONF_KEY);
+
ImplType.setServerImpl(cmd, conf);
}
+ /**
+ * Parse the command line options to set parameters the conf.
+ */
+ private void processOptions(final String[] args) throws Exception {
+ if (args == null || args.length == 0) {
+ return;
+ }
+ Options options = new Options();
+ addOptions(options);
+
+ CommandLineParser parser = new DefaultParser();
+ CommandLine cmd = parser.parse(options, args);
+
+ if (cmd.hasOption("help")) {
+ printUsageAndExit(options, 1);
+ }
+ parseCommandLine(cmd, options);
+ }
+
public void stop() {
if (this.infoServer != null) {
LOG.info("Stopping infoServer");
@@ -209,10 +728,25 @@ public class ThriftServer {
LOG.error("Failed to stop infoServer", ex);
}
}
- serverRunner.shutdown();
+ if (pauseMonitor != null) {
+ pauseMonitor.stop();
+ }
+ if (tserver != null) {
+ tserver.stop();
+ tserver = null;
+ }
+ if (httpServer != null) {
+ try {
+ httpServer.stop();
+ httpServer = null;
+ } catch (Exception e) {
+ LOG.error("Problem encountered in shutting down HTTP server", e);
+ }
+ httpServer = null;
+ }
}
- private static void optionToConf(CommandLine cmd, String option,
+ protected static void optionToConf(CommandLine cmd, String option,
Configuration conf, String destConfKey) {
if (cmd.hasOption(option)) {
String value = cmd.getOptionValue(option);
@@ -221,16 +755,38 @@ public class ThriftServer {
}
}
+ /**
+ * Run without any command line arguments
+ * @return exit code
+ * @throws Exception exception
+ */
+ public int run() throws Exception {
+ return run(null);
+ }
+
+ @Override
+ public int run(String[] strings) throws Exception {
+ processOptions(strings);
+ setupParamters();
+ startInfoServer();
+ if (httpEnabled) {
+ setupHTTPServer();
+ httpServer.start();
+ httpServer.join();
+ } else {
+ setupServer();
+ tserver.serve();
+ }
+ return 0;
+ }
+
public static void main(String [] args) throws Exception {
LOG.info("***** STARTING service '" + ThriftServer.class.getSimpleName() + "' *****");
VersionInfo.logVersion();
- int exitCode = 0;
- try {
- new ThriftServer(HBaseConfiguration.create()).doMain(args);
- } catch (ExitCodeException ex) {
- exitCode = ex.getExitCode();
- }
+ final Configuration conf = HBaseConfiguration.create();
+ // for now, only time we return is on an argument error.
+ final int status = ToolRunner.run(conf, new ThriftServer(conf), args);
LOG.info("***** STOPPING service '" + ThriftServer.class.getSimpleName() + "' *****");
- System.exit(exitCode);
+ System.exit(status);
}
}
[14/15] hbase git commit: HBASE-21526 Use AsyncClusterConnection in
ServerManager for getRsAdmin
Posted by zh...@apache.org.
HBASE-21526 Use AsyncClusterConnection in ServerManager for getRsAdmin
Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/0d051abb
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/0d051abb
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/0d051abb
Branch: refs/heads/HBASE-21512
Commit: 0d051abb71d2f8b0846c078ce7ce452c4372aa33
Parents: 721af37
Author: zhangduo <zh...@apache.org>
Authored: Thu Dec 6 21:25:34 2018 +0800
Committer: Duo Zhang <zh...@apache.org>
Committed: Fri Jan 4 14:34:02 2019 +0800
----------------------------------------------------------------------
.../hbase/client/AsyncClusterConnection.java | 6 +
.../hbase/client/AsyncConnectionImpl.java | 5 +
.../hbase/client/AsyncRegionServerAdmin.java | 210 +++++++++++++++++++
.../apache/hadoop/hbase/util/FutureUtils.java | 2 +-
.../org/apache/hadoop/hbase/master/HMaster.java | 15 +-
.../hadoop/hbase/master/ServerManager.java | 67 ------
.../master/procedure/RSProcedureDispatcher.java | 44 ++--
7 files changed, 263 insertions(+), 86 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/hbase/blob/0d051abb/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncClusterConnection.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncClusterConnection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncClusterConnection.java
index c7dea25..1327fd7 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncClusterConnection.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncClusterConnection.java
@@ -17,6 +17,7 @@
*/
package org.apache.hadoop.hbase.client;
+import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.ipc.RpcClient;
import org.apache.yetus.audience.InterfaceAudience;
@@ -27,6 +28,11 @@ import org.apache.yetus.audience.InterfaceAudience;
public interface AsyncClusterConnection extends AsyncConnection {
/**
+ * Get the admin service for the given region server.
+ */
+ AsyncRegionServerAdmin getRegionServerAdmin(ServerName serverName);
+
+ /**
* Get the nonce generator for this connection.
*/
NonceGenerator getNonceGenerator();
http://git-wip-us.apache.org/repos/asf/hbase/blob/0d051abb/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java
index 188e830..4e7f421 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java
@@ -330,4 +330,9 @@ class AsyncConnectionImpl implements AsyncClusterConnection {
return new AsyncBufferedMutatorBuilderImpl(connConf, getTableBuilder(tableName, pool),
RETRY_TIMER);
}
+
+ @Override
+ public AsyncRegionServerAdmin getRegionServerAdmin(ServerName serverName) {
+ return new AsyncRegionServerAdmin(serverName, this);
+ }
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/0d051abb/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionServerAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionServerAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionServerAdmin.java
new file mode 100644
index 0000000..9accd89
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionServerAdmin.java
@@ -0,0 +1,210 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.client;
+
+import java.io.IOException;
+import java.util.concurrent.CompletableFuture;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.ipc.HBaseRpcController;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
+
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ClearCompactionQueuesRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ClearCompactionQueuesResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ClearRegionBlockCacheRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ClearRegionBlockCacheResponse;
+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.CompactRegionRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactRegionResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactionSwitchRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactionSwitchResponse;
+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.FlushRegionRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetOnlineRegionRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetOnlineRegionResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionLoadRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionLoadResponse;
+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.shaded.protobuf.generated.AdminProtos.GetStoreFileRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetStoreFileResponse;
+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.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.RollWALWriterRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.RollWALWriterResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.StopServerRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.StopServerResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateConfigurationRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateConfigurationResponse;
+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.shaded.protobuf.generated.AdminProtos.WarmupRegionRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WarmupRegionResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsResponse;
+
+/**
+ * A simple wrapper of the {@link AdminService} for a region server, which returns a
+ * {@link CompletableFuture}. This is easier to use, as if you use the raw protobuf interface, you
+ * need to get the result from the {@link RpcCallback}, and if there is an exception, you need to
+ * get it from the {@link RpcController} passed in.
+ * <p/>
+ * Notice that there is no retry, and this is intentional. We have different retry for different
+ * usage for now, if later we want to unify them, we can move the retry logic into this class.
+ */
+@InterfaceAudience.Private
+public class AsyncRegionServerAdmin {
+
+ private final ServerName server;
+
+ private final AsyncConnectionImpl conn;
+
+ AsyncRegionServerAdmin(ServerName server, AsyncConnectionImpl conn) {
+ this.server = server;
+ this.conn = conn;
+ }
+
+ @FunctionalInterface
+ private interface RpcCall<RESP> {
+ void call(AdminService.Interface stub, HBaseRpcController controller, RpcCallback<RESP> done);
+ }
+
+ private <RESP> CompletableFuture<RESP> call(RpcCall<RESP> rpcCall) {
+ CompletableFuture<RESP> future = new CompletableFuture<>();
+ HBaseRpcController controller = conn.rpcControllerFactory.newController();
+ try {
+ rpcCall.call(conn.getAdminStub(server), controller, new RpcCallback<RESP>() {
+
+ @Override
+ public void run(RESP resp) {
+ if (controller.failed()) {
+ future.completeExceptionally(controller.getFailed());
+ } else {
+ future.complete(resp);
+ }
+ }
+ });
+ } catch (IOException e) {
+ future.completeExceptionally(e);
+ }
+ return future;
+ }
+
+ public CompletableFuture<GetRegionInfoResponse> getRegionInfo(GetRegionInfoRequest request) {
+ return call((stub, controller, done) -> stub.getRegionInfo(controller, request, done));
+ }
+
+ public CompletableFuture<GetStoreFileResponse> getStoreFile(GetStoreFileRequest request) {
+ return call((stub, controller, done) -> stub.getStoreFile(controller, request, done));
+ }
+
+ public CompletableFuture<GetOnlineRegionResponse> getOnlineRegion(
+ GetOnlineRegionRequest request) {
+ return call((stub, controller, done) -> stub.getOnlineRegion(controller, request, done));
+ }
+
+ public CompletableFuture<OpenRegionResponse> openRegion(OpenRegionRequest request) {
+ return call((stub, controller, done) -> stub.openRegion(controller, request, done));
+ }
+
+ public CompletableFuture<WarmupRegionResponse> warmupRegion(WarmupRegionRequest request) {
+ return call((stub, controller, done) -> stub.warmupRegion(controller, request, done));
+ }
+
+ public CompletableFuture<CloseRegionResponse> closeRegion(CloseRegionRequest request) {
+ return call((stub, controller, done) -> stub.closeRegion(controller, request, done));
+ }
+
+ public CompletableFuture<FlushRegionResponse> flushRegion(FlushRegionRequest request) {
+ return call((stub, controller, done) -> stub.flushRegion(controller, request, done));
+ }
+
+ public CompletableFuture<CompactionSwitchResponse> compactionSwitch(
+ CompactionSwitchRequest request) {
+ return call((stub, controller, done) -> stub.compactionSwitch(controller, request, done));
+ }
+
+ public CompletableFuture<CompactRegionResponse> compactRegion(CompactRegionRequest request) {
+ return call((stub, controller, done) -> stub.compactRegion(controller, request, done));
+ }
+
+ public CompletableFuture<ReplicateWALEntryResponse> replicateWALEntry(
+ ReplicateWALEntryRequest request) {
+ return call((stub, controller, done) -> stub.replicateWALEntry(controller, request, done));
+ }
+
+ public CompletableFuture<ReplicateWALEntryResponse> replay(ReplicateWALEntryRequest request) {
+ return call((stub, controller, done) -> stub.replay(controller, request, done));
+ }
+
+ public CompletableFuture<RollWALWriterResponse> rollWALWriter(RollWALWriterRequest request) {
+ return call((stub, controller, done) -> stub.rollWALWriter(controller, request, done));
+ }
+
+ public CompletableFuture<GetServerInfoResponse> getServerInfo(GetServerInfoRequest request) {
+ return call((stub, controller, done) -> stub.getServerInfo(controller, request, done));
+ }
+
+ public CompletableFuture<StopServerResponse> stopServer(StopServerRequest request) {
+ return call((stub, controller, done) -> stub.stopServer(controller, request, done));
+ }
+
+ public CompletableFuture<UpdateFavoredNodesResponse> updateFavoredNodes(
+ UpdateFavoredNodesRequest request) {
+ return call((stub, controller, done) -> stub.updateFavoredNodes(controller, request, done));
+ }
+
+ public CompletableFuture<UpdateConfigurationResponse> updateConfiguration(
+ UpdateConfigurationRequest request) {
+ return call((stub, controller, done) -> stub.updateConfiguration(controller, request, done));
+ }
+
+ public CompletableFuture<GetRegionLoadResponse> getRegionLoad(GetRegionLoadRequest request) {
+ return call((stub, controller, done) -> stub.getRegionLoad(controller, request, done));
+ }
+
+ public CompletableFuture<ClearCompactionQueuesResponse> clearCompactionQueues(
+ ClearCompactionQueuesRequest request) {
+ return call((stub, controller, done) -> stub.clearCompactionQueues(controller, request, done));
+ }
+
+ public CompletableFuture<ClearRegionBlockCacheResponse> clearRegionBlockCache(
+ ClearRegionBlockCacheRequest request) {
+ return call((stub, controller, done) -> stub.clearRegionBlockCache(controller, request, done));
+ }
+
+ public CompletableFuture<GetSpaceQuotaSnapshotsResponse> getSpaceQuotaSnapshots(
+ GetSpaceQuotaSnapshotsRequest request) {
+ return call((stub, controller, done) -> stub.getSpaceQuotaSnapshots(controller, request, done));
+ }
+
+ public CompletableFuture<ExecuteProceduresResponse> executeProcedures(
+ ExecuteProceduresRequest request) {
+ return call((stub, controller, done) -> stub.executeProcedures(controller, request, done));
+ }
+}
http://git-wip-us.apache.org/repos/asf/hbase/blob/0d051abb/hbase-common/src/main/java/org/apache/hadoop/hbase/util/FutureUtils.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/FutureUtils.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/FutureUtils.java
index 067e66b..f4a7332 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/FutureUtils.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/FutureUtils.java
@@ -57,4 +57,4 @@ public final class FutureUtils {
}
});
}
-}
\ No newline at end of file
+}
http://git-wip-us.apache.org/repos/asf/hbase/blob/0d051abb/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 52005d6..4fd3a37 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
@@ -193,6 +193,7 @@ import org.apache.hadoop.hbase.util.BloomFilterUtil;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.CompressionTest;
import org.apache.hadoop.hbase.util.EncryptionTest;
+import org.apache.hadoop.hbase.util.FutureUtils;
import org.apache.hadoop.hbase.util.HBaseFsck;
import org.apache.hadoop.hbase.util.HFileArchiveUtil;
import org.apache.hadoop.hbase.util.HasThread;
@@ -225,6 +226,7 @@ import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
import org.apache.hbase.thirdparty.com.google.common.collect.Maps;
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.GetRegionInfoResponse.CompactionState;
import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas;
import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceViolationPolicy;
@@ -1937,6 +1939,15 @@ public class HMaster extends HRegionServer implements MasterServices {
});
}
+ private void warmUpRegion(ServerName server, RegionInfo region) {
+ FutureUtils.addListener(asyncClusterConnection.getRegionServerAdmin(server)
+ .warmupRegion(RequestConverter.buildWarmupRegionRequest(region)), (r, e) -> {
+ if (e != null) {
+ LOG.warn("Failed to warm up region {} on server {}", region, server, e);
+ }
+ });
+ }
+
// Public so can be accessed by tests. Blocks until move is done.
// Replace with an async implementation from which you can get
// a success/failure result.
@@ -2008,7 +2019,9 @@ public class HMaster extends HRegionServer implements MasterServices {
// Warmup the region on the destination before initiating the move. this call
// is synchronous and takes some time. doing it before the source region gets
// closed
- serverManager.sendRegionWarmup(rp.getDestination(), hri);
+ // A region server could reject the close request because it either does not
+ // have the specified region or the region is being split.
+ warmUpRegion(rp.getDestination(), hri);
LOG.info(getClientIdAuditPrefix() + " move " + rp + ", running balancer");
Future<byte []> future = this.assignmentManager.moveAsync(rp);
http://git-wip-us.apache.org/repos/asf/hbase/blob/0d051abb/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 86d72d1..c26ef6c 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
@@ -24,7 +24,6 @@ import java.io.IOException;
import java.net.InetAddress;
import java.util.ArrayList;
import java.util.Collections;
-import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
@@ -51,12 +50,9 @@ import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.YouAreDeadException;
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.master.assignment.RegionStates;
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
-import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
@@ -159,25 +155,16 @@ public class ServerManager {
private final ConcurrentNavigableMap<ServerName, ServerMetrics> onlineServers =
new ConcurrentSkipListMap<>();
- /**
- * Map of admin interfaces per registered regionserver; these interfaces we use to control
- * regionservers out on the cluster
- */
- private final Map<ServerName, AdminService.BlockingInterface> rsAdmins = new HashMap<>();
-
/** List of region servers that should not get any more new regions. */
private final ArrayList<ServerName> drainingServers = new ArrayList<>();
private final MasterServices master;
- private final ClusterConnection connection;
private final DeadServer deadservers = new DeadServer();
private final long maxSkew;
private final long warningSkew;
- private final RpcControllerFactory rpcControllerFactory;
-
/** Listeners that are called on server events. */
private List<ServerListener> listeners = new CopyOnWriteArrayList<>();
@@ -189,8 +176,6 @@ public class ServerManager {
Configuration c = master.getConfiguration();
maxSkew = c.getLong("hbase.master.maxclockskew", 30000);
warningSkew = c.getLong("hbase.master.warningclockskew", 10000);
- this.connection = master.getClusterConnection();
- this.rpcControllerFactory = this.connection == null? null: connection.getRpcControllerFactory();
persistFlushedSequenceId = c.getBoolean(PERSIST_FLUSHEDSEQUENCEID,
PERSIST_FLUSHEDSEQUENCEID_DEFAULT);
}
@@ -438,7 +423,6 @@ public class ServerManager {
void recordNewServerWithLock(final ServerName serverName, final ServerMetrics sl) {
LOG.info("Registering regionserver=" + serverName);
this.onlineServers.put(serverName, sl);
- this.rsAdmins.remove(serverName);
}
@VisibleForTesting
@@ -633,7 +617,6 @@ public class ServerManager {
this.onlineServers.remove(sn);
onlineServers.notifyAll();
}
- this.rsAdmins.remove(sn);
}
/*
@@ -676,34 +659,6 @@ public class ServerManager {
return this.drainingServers.add(sn);
}
- // RPC methods to region servers
-
- private HBaseRpcController newRpcController() {
- return rpcControllerFactory == null ? null : rpcControllerFactory.newController();
- }
-
- /**
- * Sends a WARMUP RPC to the specified server to warmup the specified region.
- * <p>
- * A region server could reject the close request because it either does not
- * have the specified region or the region is being split.
- * @param server server to warmup a region
- * @param region region to warmup
- */
- public void sendRegionWarmup(ServerName server,
- RegionInfo region) {
- if (server == null) return;
- try {
- AdminService.BlockingInterface admin = getRsAdmin(server);
- HBaseRpcController controller = newRpcController();
- ProtobufUtil.warmupRegion(controller, admin, region);
- } catch (IOException e) {
- LOG.error("Received exception in RPC for warmup server:" +
- server + "region: " + region +
- "exception: " + e);
- }
- }
-
/**
* Contacts a region server and waits up to timeout ms
* to close the region. This bypasses the active hmaster.
@@ -737,28 +692,6 @@ public class ServerManager {
}
/**
- * @param sn
- * @return Admin interface for the remote regionserver named <code>sn</code>
- * @throws IOException
- * @throws RetriesExhaustedException wrapping a ConnectException if failed
- */
- public AdminService.BlockingInterface getRsAdmin(final ServerName sn)
- throws IOException {
- AdminService.BlockingInterface admin = this.rsAdmins.get(sn);
- if (admin == null) {
- LOG.debug("New admin connection to " + sn.toString());
- if (sn.equals(master.getServerName()) && master instanceof HRegionServer) {
- // A master is also a region server now, see HBASE-10569 for details
- admin = ((HRegionServer)master).getRSRpcServices();
- } else {
- admin = this.connection.getAdmin(sn);
- }
- this.rsAdmins.put(sn, admin);
- }
- return admin;
- }
-
- /**
* Calculate min necessary to start. This is not an absolute. It is just
* a friction that will cause us hang around a bit longer waiting on
* RegionServers to check-in.
http://git-wip-us.apache.org/repos/asf/hbase/blob/0d051abb/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 638f9d3..f3ab4b3 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,12 +18,15 @@
package org.apache.hadoop.hbase.master.procedure;
import java.io.IOException;
+import java.io.InterruptedIOException;
import java.util.List;
import java.util.Set;
import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
import java.util.concurrent.TimeUnit;
import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.client.AsyncRegionServerAdmin;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.exceptions.ClientExceptionsUtil;
import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
@@ -37,11 +40,11 @@ import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+import org.apache.hbase.thirdparty.com.google.common.base.Throwables;
import org.apache.hbase.thirdparty.com.google.common.collect.ArrayListMultimap;
import org.apache.hbase.thirdparty.com.google.protobuf.ByteString;
import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
-import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
@@ -159,13 +162,8 @@ public class RSProcedureDispatcher
this.serverName = serverName;
}
- protected AdminService.BlockingInterface getRsAdmin() throws IOException {
- final AdminService.BlockingInterface admin = master.getServerManager().getRsAdmin(serverName);
- if (admin == null) {
- throw new IOException("Attempting to send OPEN RPC to server " + getServerName() +
- " failed because no RPC connection found to this server");
- }
- return admin;
+ protected AsyncRegionServerAdmin getRsAdmin() throws IOException {
+ return master.getAsyncClusterConnection().getRegionServerAdmin(serverName);
}
protected ServerName getServerName() {
@@ -344,9 +342,13 @@ public class RSProcedureDispatcher
protected ExecuteProceduresResponse sendRequest(final ServerName serverName,
final ExecuteProceduresRequest request) throws IOException {
try {
- return getRsAdmin().executeProcedures(null, request);
- } catch (ServiceException se) {
- throw ProtobufUtil.getRemoteException(se);
+ return getRsAdmin().executeProcedures(request).get();
+ } catch (InterruptedException e) {
+ throw (IOException) new InterruptedIOException().initCause(e);
+ } catch (ExecutionException e) {
+ Throwable cause = e.getCause();
+ Throwables.propagateIfPossible(cause, IOException.class);
+ throw new IOException(cause);
}
}
@@ -407,9 +409,13 @@ public class RSProcedureDispatcher
private OpenRegionResponse sendRequest(final ServerName serverName,
final OpenRegionRequest request) throws IOException {
try {
- return getRsAdmin().openRegion(null, request);
- } catch (ServiceException se) {
- throw ProtobufUtil.getRemoteException(se);
+ return getRsAdmin().openRegion(request).get();
+ } catch (InterruptedException e) {
+ throw (IOException) new InterruptedIOException().initCause(e);
+ } catch (ExecutionException e) {
+ Throwable cause = e.getCause();
+ Throwables.propagateIfPossible(cause, IOException.class);
+ throw new IOException(cause);
}
}
@@ -453,9 +459,13 @@ public class RSProcedureDispatcher
private CloseRegionResponse sendRequest(final ServerName serverName,
final CloseRegionRequest request) throws IOException {
try {
- return getRsAdmin().closeRegion(null, request);
- } catch (ServiceException se) {
- throw ProtobufUtil.getRemoteException(se);
+ return getRsAdmin().closeRegion(request).get();
+ } catch (InterruptedException e) {
+ throw (IOException) new InterruptedIOException().initCause(e);
+ } catch (ExecutionException e) {
+ Throwable cause = e.getCause();
+ Throwables.propagateIfPossible(cause, IOException.class);
+ throw new IOException(cause);
}
}
[08/15] hbase git commit: HBASE-17356 Add replica get support
Posted by zh...@apache.org.
HBASE-17356 Add replica get support
Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/db66e6cc
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/db66e6cc
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/db66e6cc
Branch: refs/heads/HBASE-21512
Commit: db66e6cc9e1c6ea027631388aba688cb623b7d0a
Parents: e4b6b4a
Author: zhangduo <zh...@apache.org>
Authored: Tue Jan 1 21:59:37 2019 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Thu Jan 3 08:38:20 2019 +0800
----------------------------------------------------------------------
.../apache/hadoop/hbase/RegionLocations.java | 30 +-
.../client/AsyncBatchRpcRetryingCaller.java | 114 +-
.../client/AsyncConnectionConfiguration.java | 12 +
.../hbase/client/AsyncConnectionImpl.java | 1 -
.../hbase/client/AsyncMetaRegionLocator.java | 125 +-
.../hbase/client/AsyncNonMetaRegionLocator.java | 291 +--
.../hadoop/hbase/client/AsyncRegionLocator.java | 129 +-
.../hbase/client/AsyncRegionLocatorHelper.java | 147 ++
.../hbase/client/AsyncRpcRetryingCaller.java | 15 +-
.../client/AsyncRpcRetryingCallerFactory.java | 55 +-
.../AsyncSingleRequestRpcRetryingCaller.java | 71 +-
.../hbase/client/AsyncTableRegionLocator.java | 28 +-
.../client/AsyncTableRegionLocatorImpl.java | 6 +-
.../hbase/client/ConnectionConfiguration.java | 5 +-
.../hadoop/hbase/client/RawAsyncHBaseAdmin.java | 2033 +++++++++---------
.../hadoop/hbase/client/RawAsyncTableImpl.java | 208 +-
.../apache/hadoop/hbase/util/FutureUtils.java | 60 +
.../hbase/client/RegionReplicaTestHelper.java | 161 ++
.../client/TestAsyncMetaRegionLocator.java | 55 +-
.../client/TestAsyncNonMetaRegionLocator.java | 126 +-
...syncNonMetaRegionLocatorConcurrenyLimit.java | 20 +-
...TestAsyncSingleRequestRpcRetryingCaller.java | 56 +-
.../client/TestAsyncTableLocatePrefetch.java | 4 +-
.../client/TestAsyncTableRegionReplicasGet.java | 204 ++
.../hbase/client/TestZKAsyncRegistry.java | 44 +-
25 files changed, 2366 insertions(+), 1634 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/hbase/blob/db66e6cc/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 fd6f3c7..f98bf03 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
@@ -56,8 +56,8 @@ public class RegionLocations {
int index = 0;
for (HRegionLocation loc : locations) {
if (loc != null) {
- if (loc.getRegionInfo().getReplicaId() >= maxReplicaId) {
- maxReplicaId = loc.getRegionInfo().getReplicaId();
+ if (loc.getRegion().getReplicaId() >= maxReplicaId) {
+ maxReplicaId = loc.getRegion().getReplicaId();
maxReplicaIdIndex = index;
}
}
@@ -72,7 +72,7 @@ public class RegionLocations {
this.locations = new HRegionLocation[maxReplicaId + 1];
for (HRegionLocation loc : locations) {
if (loc != null) {
- this.locations[loc.getRegionInfo().getReplicaId()] = loc;
+ this.locations[loc.getRegion().getReplicaId()] = loc;
}
}
}
@@ -146,7 +146,7 @@ public class RegionLocations {
public RegionLocations remove(HRegionLocation location) {
if (location == null) return this;
if (location.getRegion() == null) return this;
- int replicaId = location.getRegionInfo().getReplicaId();
+ int replicaId = location.getRegion().getReplicaId();
if (replicaId >= locations.length) return this;
// check whether something to remove. HRL.compareTo() compares ONLY the
@@ -203,14 +203,14 @@ public class RegionLocations {
// in case of region replication going down, we might have a leak here.
int max = other.locations.length;
- HRegionInfo regionInfo = null;
+ RegionInfo regionInfo = null;
for (int i = 0; i < max; i++) {
HRegionLocation thisLoc = this.getRegionLocation(i);
HRegionLocation otherLoc = other.getRegionLocation(i);
- if (regionInfo == null && otherLoc != null && otherLoc.getRegionInfo() != null) {
+ if (regionInfo == null && otherLoc != null && otherLoc.getRegion() != null) {
// regionInfo is the first non-null HRI from other RegionLocations. We use it to ensure that
// all replica region infos belong to the same region with same region id.
- regionInfo = otherLoc.getRegionInfo();
+ regionInfo = otherLoc.getRegion();
}
HRegionLocation selectedLoc = selectRegionLocation(thisLoc,
@@ -232,7 +232,7 @@ public class RegionLocations {
for (int i=0; i < newLocations.length; i++) {
if (newLocations[i] != null) {
if (!RegionReplicaUtil.isReplicasForSameRegion(regionInfo,
- newLocations[i].getRegionInfo())) {
+ newLocations[i].getRegion())) {
newLocations[i] = null;
}
}
@@ -273,9 +273,9 @@ public class RegionLocations {
boolean checkForEquals, boolean force) {
assert location != null;
- int replicaId = location.getRegionInfo().getReplicaId();
+ int replicaId = location.getRegion().getReplicaId();
- HRegionLocation oldLoc = getRegionLocation(location.getRegionInfo().getReplicaId());
+ HRegionLocation oldLoc = getRegionLocation(location.getRegion().getReplicaId());
HRegionLocation selectedLoc = selectRegionLocation(oldLoc, location,
checkForEquals, force);
@@ -288,8 +288,8 @@ public class RegionLocations {
// ensure that all replicas share the same start code. Otherwise delete them
for (int i=0; i < newLocations.length; i++) {
if (newLocations[i] != null) {
- if (!RegionReplicaUtil.isReplicasForSameRegion(location.getRegionInfo(),
- newLocations[i].getRegionInfo())) {
+ if (!RegionReplicaUtil.isReplicasForSameRegion(location.getRegion(),
+ newLocations[i].getRegion())) {
newLocations[i] = null;
}
}
@@ -317,8 +317,8 @@ public class RegionLocations {
public HRegionLocation getRegionLocationByRegionName(byte[] regionName) {
for (HRegionLocation loc : locations) {
if (loc != null) {
- if (Bytes.equals(loc.getRegionInfo().getRegionName(), regionName)
- || Bytes.equals(loc.getRegionInfo().getEncodedNameAsBytes(), regionName)) {
+ if (Bytes.equals(loc.getRegion().getRegionName(), regionName)
+ || Bytes.equals(loc.getRegion().getEncodedNameAsBytes(), regionName)) {
return loc;
}
}
@@ -331,7 +331,7 @@ public class RegionLocations {
}
public HRegionLocation getDefaultRegionLocation() {
- return locations[HRegionInfo.DEFAULT_REPLICA_ID];
+ return locations[RegionInfo.DEFAULT_REPLICA_ID];
}
/**
http://git-wip-us.apache.org/repos/asf/hbase/blob/db66e6cc/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncBatchRpcRetryingCaller.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncBatchRpcRetryingCaller.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncBatchRpcRetryingCaller.java
index 51b89a9..e268b2e 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncBatchRpcRetryingCaller.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncBatchRpcRetryingCaller.java
@@ -23,8 +23,7 @@ import static org.apache.hadoop.hbase.client.ConnectionUtils.getPauseTime;
import static org.apache.hadoop.hbase.client.ConnectionUtils.resetController;
import static org.apache.hadoop.hbase.client.ConnectionUtils.translateException;
import static org.apache.hadoop.hbase.util.CollectionUtils.computeIfAbsent;
-
-import org.apache.hbase.thirdparty.io.netty.util.HashedWheelTimer;
+import static org.apache.hadoop.hbase.util.FutureUtils.addListener;
import java.io.IOException;
import java.util.ArrayList;
@@ -43,24 +42,26 @@ import java.util.concurrent.TimeUnit;
import java.util.function.Supplier;
import java.util.stream.Collectors;
import java.util.stream.Stream;
-
import org.apache.hadoop.hbase.CellScannable;
import org.apache.hadoop.hbase.DoNotRetryIOException;
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.slf4j.Logger;
-import org.slf4j.LoggerFactory;
import org.apache.hadoop.hbase.client.MultiResponse.RegionResult;
import org.apache.hadoop.hbase.client.RetriesExhaustedException.ThrowableWithExtraContext;
import org.apache.hadoop.hbase.ipc.HBaseRpcController;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hbase.thirdparty.io.netty.util.HashedWheelTimer;
+
import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
import org.apache.hadoop.hbase.shaded.protobuf.ResponseConverter;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientService;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
/**
* Retry caller for batch.
@@ -121,10 +122,10 @@ class AsyncBatchRpcRetryingCaller<T> {
private static final class ServerRequest {
public final ConcurrentMap<byte[], RegionRequest> actionsByRegion =
- new ConcurrentSkipListMap<>(Bytes.BYTES_COMPARATOR);
+ new ConcurrentSkipListMap<>(Bytes.BYTES_COMPARATOR);
public void addAction(HRegionLocation loc, Action action) {
- computeIfAbsent(actionsByRegion, loc.getRegionInfo().getRegionName(),
+ computeIfAbsent(actionsByRegion, loc.getRegion().getRegionName(),
() -> new RegionRequest(loc)).actions.add(action);
}
}
@@ -173,11 +174,10 @@ class AsyncBatchRpcRetryingCaller<T> {
Throwable error, ServerName serverName) {
if (tries > startLogErrorsCnt) {
String regions =
- regionsSupplier.get().map(r -> "'" + r.loc.getRegionInfo().getRegionNameAsString() + "'")
- .collect(Collectors.joining(",", "[", "]"));
- LOG.warn("Process batch for " + regions + " in " + tableName + " from " + serverName
- + " failed, tries=" + tries,
- error);
+ regionsSupplier.get().map(r -> "'" + r.loc.getRegion().getRegionNameAsString() + "'")
+ .collect(Collectors.joining(",", "[", "]"));
+ LOG.warn("Process batch for " + regions + " in " + tableName + " from " + serverName +
+ " failed, tries=" + tries, error);
}
}
@@ -191,7 +191,7 @@ class AsyncBatchRpcRetryingCaller<T> {
errors = action2Errors.computeIfAbsent(action, k -> new ArrayList<>());
}
errors.add(new ThrowableWithExtraContext(error, EnvironmentEdgeManager.currentTime(),
- getExtraContextForError(serverName)));
+ getExtraContextForError(serverName)));
}
private void addError(Iterable<Action> actions, Throwable error, ServerName serverName) {
@@ -204,7 +204,7 @@ class AsyncBatchRpcRetryingCaller<T> {
return;
}
ThrowableWithExtraContext errorWithCtx =
- new ThrowableWithExtraContext(error, currentTime, extras);
+ new ThrowableWithExtraContext(error, currentTime, extras);
List<ThrowableWithExtraContext> errors = removeErrors(action);
if (errors == null) {
errors = Collections.singletonList(errorWithCtx);
@@ -227,7 +227,7 @@ class AsyncBatchRpcRetryingCaller<T> {
return;
}
future.completeExceptionally(new RetriesExhaustedException(tries,
- Optional.ofNullable(removeErrors(action)).orElse(Collections.emptyList())));
+ Optional.ofNullable(removeErrors(action)).orElse(Collections.emptyList())));
});
}
@@ -242,9 +242,9 @@ class AsyncBatchRpcRetryingCaller<T> {
// multiRequestBuilder will be populated with region actions.
// rowMutationsIndexMap will be non-empty after the call if there is RowMutations in the
// action list.
- RequestConverter.buildNoDataRegionActions(entry.getKey(),
- entry.getValue().actions, cells, multiRequestBuilder, regionActionBuilder, actionBuilder,
- mutationBuilder, nonceGroup, rowMutationsIndexMap);
+ RequestConverter.buildNoDataRegionActions(entry.getKey(), entry.getValue().actions, cells,
+ multiRequestBuilder, regionActionBuilder, actionBuilder, mutationBuilder, nonceGroup,
+ rowMutationsIndexMap);
}
return multiRequestBuilder.build();
}
@@ -254,15 +254,15 @@ class AsyncBatchRpcRetryingCaller<T> {
RegionResult regionResult, List<Action> failedActions, Throwable regionException) {
Object result = regionResult.result.getOrDefault(action.getOriginalIndex(), regionException);
if (result == null) {
- LOG.error("Server " + serverName + " sent us neither result nor exception for row '"
- + Bytes.toStringBinary(action.getAction().getRow()) + "' of "
- + regionReq.loc.getRegionInfo().getRegionNameAsString());
+ LOG.error("Server " + serverName + " sent us neither result nor exception for row '" +
+ Bytes.toStringBinary(action.getAction().getRow()) + "' of " +
+ regionReq.loc.getRegion().getRegionNameAsString());
addError(action, new RuntimeException("Invalid response"), serverName);
failedActions.add(action);
} else if (result instanceof Throwable) {
Throwable error = translateException((Throwable) result);
logException(tries, () -> Stream.of(regionReq), error, serverName);
- conn.getLocator().updateCachedLocation(regionReq.loc, error);
+ conn.getLocator().updateCachedLocationOnError(regionReq.loc, error);
if (error instanceof DoNotRetryIOException || tries >= maxAttempts) {
failOne(action, tries, error, EnvironmentEdgeManager.currentTime(),
getExtraContextForError(serverName));
@@ -281,20 +281,19 @@ class AsyncBatchRpcRetryingCaller<T> {
RegionResult regionResult = resp.getResults().get(rn);
Throwable regionException = resp.getException(rn);
if (regionResult != null) {
- regionReq.actions.forEach(
- action -> onComplete(action, regionReq, tries, serverName, regionResult, failedActions,
- regionException));
+ regionReq.actions.forEach(action -> onComplete(action, regionReq, tries, serverName,
+ regionResult, failedActions, regionException));
} else {
Throwable error;
if (regionException == null) {
- LOG.error(
- "Server sent us neither results nor exceptions for " + Bytes.toStringBinary(rn));
+ LOG
+ .error("Server sent us neither results nor exceptions for " + Bytes.toStringBinary(rn));
error = new RuntimeException("Invalid response");
} else {
error = translateException(regionException);
}
logException(tries, () -> Stream.of(regionReq), error, serverName);
- conn.getLocator().updateCachedLocation(regionReq.loc, error);
+ conn.getLocator().updateCachedLocationOnError(regionReq.loc, error);
if (error instanceof DoNotRetryIOException || tries >= maxAttempts) {
failAll(regionReq.actions.stream(), tries, error, serverName);
return;
@@ -314,8 +313,7 @@ class AsyncBatchRpcRetryingCaller<T> {
remainingNs = remainingTimeNs();
if (remainingNs <= 0) {
failAll(actionsByServer.values().stream().flatMap(m -> m.actionsByRegion.values().stream())
- .flatMap(r -> r.actions.stream()),
- tries);
+ .flatMap(r -> r.actions.stream()), tries);
return;
}
} else {
@@ -366,15 +364,15 @@ class AsyncBatchRpcRetryingCaller<T> {
ServerName serverName) {
Throwable error = translateException(t);
logException(tries, () -> actionsByRegion.values().stream(), error, serverName);
- actionsByRegion
- .forEach((rn, regionReq) -> conn.getLocator().updateCachedLocation(regionReq.loc, error));
+ actionsByRegion.forEach(
+ (rn, regionReq) -> conn.getLocator().updateCachedLocationOnError(regionReq.loc, error));
if (error instanceof DoNotRetryIOException || tries >= maxAttempts) {
failAll(actionsByRegion.values().stream().flatMap(r -> r.actions.stream()), tries, error,
serverName);
return;
}
List<Action> copiedActions = actionsByRegion.values().stream().flatMap(r -> r.actions.stream())
- .collect(Collectors.toList());
+ .collect(Collectors.toList());
addError(copiedActions, error, serverName);
tryResubmit(copiedActions.stream(), tries);
}
@@ -407,30 +405,30 @@ class AsyncBatchRpcRetryingCaller<T> {
}
ConcurrentMap<ServerName, ServerRequest> actionsByServer = new ConcurrentHashMap<>();
ConcurrentLinkedQueue<Action> locateFailed = new ConcurrentLinkedQueue<>();
- CompletableFuture.allOf(actions
- .map(action -> conn.getLocator().getRegionLocation(tableName, action.getAction().getRow(),
- RegionLocateType.CURRENT, locateTimeoutNs).whenComplete((loc, error) -> {
- if (error != null) {
- error = translateException(error);
- if (error instanceof DoNotRetryIOException) {
- failOne(action, tries, error, EnvironmentEdgeManager.currentTime(), "");
- return;
- }
- addError(action, error, null);
- locateFailed.add(action);
- } else {
- computeIfAbsent(actionsByServer, loc.getServerName(), ServerRequest::new)
- .addAction(loc, action);
+ addListener(CompletableFuture.allOf(actions
+ .map(action -> conn.getLocator().getRegionLocation(tableName, action.getAction().getRow(),
+ RegionLocateType.CURRENT, locateTimeoutNs).whenComplete((loc, error) -> {
+ if (error != null) {
+ error = translateException(error);
+ if (error instanceof DoNotRetryIOException) {
+ failOne(action, tries, error, EnvironmentEdgeManager.currentTime(), "");
+ return;
}
- }))
- .toArray(CompletableFuture[]::new)).whenComplete((v, r) -> {
- if (!actionsByServer.isEmpty()) {
- send(actionsByServer, tries);
- }
- if (!locateFailed.isEmpty()) {
- tryResubmit(locateFailed.stream(), tries);
+ addError(action, error, null);
+ locateFailed.add(action);
+ } else {
+ computeIfAbsent(actionsByServer, loc.getServerName(), ServerRequest::new).addAction(loc,
+ action);
}
- });
+ }))
+ .toArray(CompletableFuture[]::new)), (v, r) -> {
+ if (!actionsByServer.isEmpty()) {
+ send(actionsByServer, tries);
+ }
+ if (!locateFailed.isEmpty()) {
+ tryResubmit(locateFailed.stream(), tries);
+ }
+ });
}
public List<CompletableFuture<T>> call() {
http://git-wip-us.apache.org/repos/asf/hbase/blob/db66e6cc/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionConfiguration.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionConfiguration.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionConfiguration.java
index 915e9dd..fa051a5 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionConfiguration.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionConfiguration.java
@@ -39,6 +39,8 @@ import static org.apache.hadoop.hbase.HConstants.HBASE_RPC_TIMEOUT_KEY;
import static org.apache.hadoop.hbase.HConstants.HBASE_RPC_WRITE_TIMEOUT_KEY;
import static org.apache.hadoop.hbase.client.AsyncProcess.DEFAULT_START_LOG_ERRORS_AFTER_COUNT;
import static org.apache.hadoop.hbase.client.AsyncProcess.START_LOG_ERRORS_AFTER_COUNT_KEY;
+import static org.apache.hadoop.hbase.client.ConnectionConfiguration.PRIMARY_CALL_TIMEOUT_MICROSECOND;
+import static org.apache.hadoop.hbase.client.ConnectionConfiguration.PRIMARY_CALL_TIMEOUT_MICROSECOND_DEFAULT;
import static org.apache.hadoop.hbase.client.ConnectionConfiguration.WRITE_BUFFER_PERIODIC_FLUSH_TIMEOUT_MS;
import static org.apache.hadoop.hbase.client.ConnectionConfiguration.WRITE_BUFFER_PERIODIC_FLUSH_TIMEOUT_MS_DEFAULT;
import static org.apache.hadoop.hbase.client.ConnectionConfiguration.WRITE_BUFFER_SIZE_DEFAULT;
@@ -94,6 +96,10 @@ class AsyncConnectionConfiguration {
private final long writeBufferPeriodicFlushTimeoutNs;
+ // this is for supporting region replica get, if the primary does not finished within this
+ // timeout, we will send request to secondaries.
+ private final long primaryCallTimeoutNs;
+
@SuppressWarnings("deprecation")
AsyncConnectionConfiguration(Configuration conf) {
this.metaOperationTimeoutNs = TimeUnit.MILLISECONDS.toNanos(
@@ -124,6 +130,8 @@ class AsyncConnectionConfiguration {
this.writeBufferPeriodicFlushTimeoutNs =
TimeUnit.MILLISECONDS.toNanos(conf.getLong(WRITE_BUFFER_PERIODIC_FLUSH_TIMEOUT_MS,
WRITE_BUFFER_PERIODIC_FLUSH_TIMEOUT_MS_DEFAULT));
+ this.primaryCallTimeoutNs = TimeUnit.MICROSECONDS.toNanos(
+ conf.getLong(PRIMARY_CALL_TIMEOUT_MICROSECOND, PRIMARY_CALL_TIMEOUT_MICROSECOND_DEFAULT));
}
long getMetaOperationTimeoutNs() {
@@ -181,4 +189,8 @@ class AsyncConnectionConfiguration {
long getWriteBufferPeriodicFlushTimeoutNs() {
return writeBufferPeriodicFlushTimeoutNs;
}
+
+ long getPrimaryCallTimeoutNs() {
+ return primaryCallTimeoutNs;
+ }
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/db66e6cc/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java
index 078395b..361d5b2 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java
@@ -152,7 +152,6 @@ class AsyncConnectionImpl implements AsyncConnection {
}
// we will override this method for testing retry caller, so do not remove this method.
- @VisibleForTesting
AsyncRegionLocator getLocator() {
return locator;
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/db66e6cc/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncMetaRegionLocator.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncMetaRegionLocator.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncMetaRegionLocator.java
index 06b5b57..9fef15d 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncMetaRegionLocator.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncMetaRegionLocator.java
@@ -17,11 +17,16 @@
*/
package org.apache.hadoop.hbase.client;
-import static org.apache.hadoop.hbase.client.AsyncRegionLocator.*;
+import static org.apache.hadoop.hbase.client.AsyncRegionLocatorHelper.canUpdateOnError;
+import static org.apache.hadoop.hbase.client.AsyncRegionLocatorHelper.createRegionLocations;
+import static org.apache.hadoop.hbase.client.AsyncRegionLocatorHelper.isGood;
+import static org.apache.hadoop.hbase.client.AsyncRegionLocatorHelper.removeRegionLocation;
+import static org.apache.hadoop.hbase.client.AsyncRegionLocatorHelper.replaceRegionLocation;
+
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.atomic.AtomicReference;
-
import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.RegionLocations;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -36,43 +41,43 @@ class AsyncMetaRegionLocator {
private final AsyncRegistry registry;
- private final AtomicReference<HRegionLocation> metaRegionLocation = new AtomicReference<>();
+ private final AtomicReference<RegionLocations> metaRegionLocations = new AtomicReference<>();
- private final AtomicReference<CompletableFuture<HRegionLocation>> metaRelocateFuture =
- new AtomicReference<>();
+ private final AtomicReference<CompletableFuture<RegionLocations>> metaRelocateFuture =
+ new AtomicReference<>();
AsyncMetaRegionLocator(AsyncRegistry registry) {
this.registry = registry;
}
- CompletableFuture<HRegionLocation> getRegionLocation(boolean reload) {
+ /**
+ * Get the region locations for meta region. If the location for the given replica is not
+ * available in the cached locations, then fetch from the HBase cluster.
+ * <p/>
+ * The <code>replicaId</code> parameter is important. If the region replication config for meta
+ * region is changed, then the cached region locations may not have the locations for new
+ * replicas. If we do not check the location for the given replica, we will always return the
+ * cached region locations and cause an infinite loop.
+ */
+ CompletableFuture<RegionLocations> getRegionLocations(int replicaId, boolean reload) {
for (;;) {
if (!reload) {
- HRegionLocation metaRegionLocation = this.metaRegionLocation.get();
- if (metaRegionLocation != null) {
- return CompletableFuture.completedFuture(metaRegionLocation);
+ RegionLocations locs = this.metaRegionLocations.get();
+ if (isGood(locs, replicaId)) {
+ return CompletableFuture.completedFuture(locs);
}
}
- if (LOG.isTraceEnabled()) {
- LOG.trace("Meta region location cache is null, try fetching from registry.");
- }
+ LOG.trace("Meta region location cache is null, try fetching from registry.");
if (metaRelocateFuture.compareAndSet(null, new CompletableFuture<>())) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("Start fetching meta region location from registry.");
- }
- CompletableFuture<HRegionLocation> future = metaRelocateFuture.get();
+ LOG.debug("Start fetching meta region location from registry.");
+ CompletableFuture<RegionLocations> future = metaRelocateFuture.get();
registry.getMetaRegionLocation().whenComplete((locs, error) -> {
if (error != null) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("Failed to fetch meta region location from registry", error);
- }
+ LOG.debug("Failed to fetch meta region location from registry", error);
metaRelocateFuture.getAndSet(null).completeExceptionally(error);
return;
}
- HRegionLocation loc = locs.getDefaultRegionLocation();
- if (LOG.isDebugEnabled()) {
- LOG.debug("The fetched meta region location is " + loc);
- }
+ LOG.debug("The fetched meta region location is {}" + locs);
// Here we update cache before reset future, so it is possible that someone can get a
// stale value. Consider this:
// 1. update cache
@@ -82,12 +87,12 @@ class AsyncMetaRegionLocator {
// cleared in step 2.
// But we do not think it is a big deal as it rarely happens, and even if it happens, the
// caller will retry again later, no correctness problems.
- this.metaRegionLocation.set(loc);
+ this.metaRegionLocations.set(locs);
metaRelocateFuture.set(null);
- future.complete(loc);
+ future.complete(locs);
});
} else {
- CompletableFuture<HRegionLocation> future = metaRelocateFuture.get();
+ CompletableFuture<RegionLocations> future = metaRelocateFuture.get();
if (future != null) {
return future;
}
@@ -95,30 +100,56 @@ class AsyncMetaRegionLocator {
}
}
- void updateCachedLocation(HRegionLocation loc, Throwable exception) {
- AsyncRegionLocator.updateCachedLocation(loc, exception, l -> metaRegionLocation.get(),
- newLoc -> {
- for (;;) {
- HRegionLocation oldLoc = metaRegionLocation.get();
- if (oldLoc != null && (oldLoc.getSeqNum() > newLoc.getSeqNum() ||
- oldLoc.getServerName().equals(newLoc.getServerName()))) {
- return;
- }
- if (metaRegionLocation.compareAndSet(oldLoc, newLoc)) {
- return;
- }
- }
- }, l -> {
- for (;;) {
- HRegionLocation oldLoc = metaRegionLocation.get();
- if (!canUpdate(l, oldLoc) || metaRegionLocation.compareAndSet(oldLoc, null)) {
- return;
- }
+ private HRegionLocation getCacheLocation(HRegionLocation loc) {
+ RegionLocations locs = metaRegionLocations.get();
+ return locs != null ? locs.getRegionLocation(loc.getRegion().getReplicaId()) : null;
+ }
+
+ private void addLocationToCache(HRegionLocation loc) {
+ for (;;) {
+ int replicaId = loc.getRegion().getReplicaId();
+ RegionLocations oldLocs = metaRegionLocations.get();
+ if (oldLocs == null) {
+ RegionLocations newLocs = createRegionLocations(loc);
+ if (metaRegionLocations.compareAndSet(null, newLocs)) {
+ return;
}
- });
+ }
+ HRegionLocation oldLoc = oldLocs.getRegionLocation(replicaId);
+ if (oldLoc != null && (oldLoc.getSeqNum() > loc.getSeqNum() ||
+ oldLoc.getServerName().equals(loc.getServerName()))) {
+ return;
+ }
+ RegionLocations newLocs = replaceRegionLocation(oldLocs, loc);
+ if (metaRegionLocations.compareAndSet(oldLocs, newLocs)) {
+ return;
+ }
+ }
+ }
+
+ private void removeLocationFromCache(HRegionLocation loc) {
+ for (;;) {
+ RegionLocations oldLocs = metaRegionLocations.get();
+ if (oldLocs == null) {
+ return;
+ }
+ HRegionLocation oldLoc = oldLocs.getRegionLocation(loc.getRegion().getReplicaId());
+ if (!canUpdateOnError(loc, oldLoc)) {
+ return;
+ }
+ RegionLocations newLocs = removeRegionLocation(oldLocs, loc.getRegion().getReplicaId());
+ if (metaRegionLocations.compareAndSet(oldLocs, newLocs)) {
+ return;
+ }
+ }
+ }
+
+ void updateCachedLocationOnError(HRegionLocation loc, Throwable exception) {
+ AsyncRegionLocatorHelper.updateCachedLocationOnError(loc, exception, this::getCacheLocation,
+ this::addLocationToCache, this::removeLocationFromCache);
}
void clearCache() {
- metaRegionLocation.set(null);
+ metaRegionLocations.set(null);
}
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/db66e6cc/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncNonMetaRegionLocator.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncNonMetaRegionLocator.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncNonMetaRegionLocator.java
index 7e3d56c..1fcfbb0 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncNonMetaRegionLocator.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncNonMetaRegionLocator.java
@@ -20,6 +20,11 @@ package org.apache.hadoop.hbase.client;
import static org.apache.hadoop.hbase.HConstants.NINES;
import static org.apache.hadoop.hbase.HConstants.ZEROES;
import static org.apache.hadoop.hbase.TableName.META_TABLE_NAME;
+import static org.apache.hadoop.hbase.client.AsyncRegionLocatorHelper.canUpdateOnError;
+import static org.apache.hadoop.hbase.client.AsyncRegionLocatorHelper.createRegionLocations;
+import static org.apache.hadoop.hbase.client.AsyncRegionLocatorHelper.isGood;
+import static org.apache.hadoop.hbase.client.AsyncRegionLocatorHelper.mergeRegionLocations;
+import static org.apache.hadoop.hbase.client.AsyncRegionLocatorHelper.removeRegionLocation;
import static org.apache.hadoop.hbase.client.ConnectionUtils.createClosestRowAfter;
import static org.apache.hadoop.hbase.client.ConnectionUtils.isEmptyStopRow;
import static org.apache.hadoop.hbase.client.RegionInfo.createRegionName;
@@ -39,7 +44,9 @@ import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.ConcurrentNavigableMap;
import java.util.concurrent.ConcurrentSkipListMap;
+import org.apache.commons.lang3.ObjectUtils;
import org.apache.hadoop.hbase.DoNotRetryIOException;
+import org.apache.hadoop.hbase.HBaseIOException;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.MetaTableAccessor;
@@ -53,6 +60,7 @@ import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
+import org.apache.hbase.thirdparty.com.google.common.base.Objects;
/**
* The asynchronous locator for regions other than meta.
@@ -83,9 +91,9 @@ class AsyncNonMetaRegionLocator {
private static final class LocateRequest {
- public final byte[] row;
+ private final byte[] row;
- public final RegionLocateType locateType;
+ private final RegionLocateType locateType;
public LocateRequest(byte[] row, RegionLocateType locateType) {
this.row = row;
@@ -109,12 +117,12 @@ class AsyncNonMetaRegionLocator {
private static final class TableCache {
- public final ConcurrentNavigableMap<byte[], HRegionLocation> cache =
+ private final ConcurrentNavigableMap<byte[], RegionLocations> cache =
new ConcurrentSkipListMap<>(BYTES_COMPARATOR);
- public final Set<LocateRequest> pendingRequests = new HashSet<>();
+ private final Set<LocateRequest> pendingRequests = new HashSet<>();
- public final Map<LocateRequest, CompletableFuture<HRegionLocation>> allRequests =
+ private final Map<LocateRequest, CompletableFuture<RegionLocations>> allRequests =
new LinkedHashMap<>();
public boolean hasQuota(int max) {
@@ -133,25 +141,29 @@ class AsyncNonMetaRegionLocator {
return allRequests.keySet().stream().filter(r -> !isPending(r)).findFirst();
}
- public void clearCompletedRequests(Optional<HRegionLocation> location) {
- for (Iterator<Map.Entry<LocateRequest, CompletableFuture<HRegionLocation>>> iter =
+ public void clearCompletedRequests(Optional<RegionLocations> locations) {
+ for (Iterator<Map.Entry<LocateRequest, CompletableFuture<RegionLocations>>> iter =
allRequests.entrySet().iterator(); iter.hasNext();) {
- Map.Entry<LocateRequest, CompletableFuture<HRegionLocation>> entry = iter.next();
- if (tryComplete(entry.getKey(), entry.getValue(), location)) {
+ Map.Entry<LocateRequest, CompletableFuture<RegionLocations>> entry = iter.next();
+ if (tryComplete(entry.getKey(), entry.getValue(), locations)) {
iter.remove();
}
}
}
- private boolean tryComplete(LocateRequest req, CompletableFuture<HRegionLocation> future,
- Optional<HRegionLocation> location) {
+ private boolean tryComplete(LocateRequest req, CompletableFuture<RegionLocations> future,
+ Optional<RegionLocations> locations) {
if (future.isDone()) {
return true;
}
- if (!location.isPresent()) {
+ if (!locations.isPresent()) {
return false;
}
- HRegionLocation loc = location.get();
+ RegionLocations locs = locations.get();
+ HRegionLocation loc = ObjectUtils.firstNonNull(locs.getRegionLocations());
+ // we should at least have one location available, otherwise the request should fail and
+ // should not arrive here
+ assert loc != null;
boolean completed;
if (req.locateType.equals(RegionLocateType.BEFORE)) {
// for locating the row before current row, the common case is to find the previous region
@@ -166,7 +178,7 @@ class AsyncNonMetaRegionLocator {
completed = loc.getRegion().containsRow(req.row);
}
if (completed) {
- future.complete(loc);
+ future.complete(locs);
return true;
} else {
return false;
@@ -186,59 +198,59 @@ class AsyncNonMetaRegionLocator {
return computeIfAbsent(cache, tableName, TableCache::new);
}
- private void removeFromCache(HRegionLocation loc) {
- TableCache tableCache = cache.get(loc.getRegion().getTable());
- if (tableCache == null) {
- return;
+ private boolean isEqual(RegionLocations locs1, RegionLocations locs2) {
+ HRegionLocation[] locArr1 = locs1.getRegionLocations();
+ HRegionLocation[] locArr2 = locs2.getRegionLocations();
+ if (locArr1.length != locArr2.length) {
+ return false;
}
- tableCache.cache.computeIfPresent(loc.getRegion().getStartKey(), (k, oldLoc) -> {
- if (oldLoc.getSeqNum() > loc.getSeqNum() ||
- !oldLoc.getServerName().equals(loc.getServerName())) {
- return oldLoc;
+ for (int i = 0; i < locArr1.length; i++) {
+ // do not need to compare region info
+ HRegionLocation loc1 = locArr1[i];
+ HRegionLocation loc2 = locArr2[i];
+ if (loc1 == null) {
+ if (loc2 != null) {
+ return false;
+ }
+ } else {
+ if (loc2 == null) {
+ return false;
+ }
+ if (loc1.getSeqNum() != loc2.getSeqNum()) {
+ return false;
+ }
+ if (Objects.equal(loc1.getServerName(), loc2.getServerName())) {
+ return false;
+ }
}
- return null;
- });
+ }
+ return true;
}
// return whether we add this loc to cache
- private boolean addToCache(TableCache tableCache, HRegionLocation loc) {
- if (LOG.isTraceEnabled()) {
- LOG.trace("Try adding " + loc + " to cache");
- }
- byte[] startKey = loc.getRegion().getStartKey();
- HRegionLocation oldLoc = tableCache.cache.putIfAbsent(startKey, loc);
- if (oldLoc == null) {
- return true;
- }
- if (oldLoc.getSeqNum() > loc.getSeqNum() ||
- oldLoc.getServerName().equals(loc.getServerName())) {
- if (LOG.isTraceEnabled()) {
- LOG.trace("Will not add " + loc + " to cache because the old value " + oldLoc +
- " is newer than us or has the same server name");
- }
- return false;
- }
- return loc == tableCache.cache.compute(startKey, (k, oldValue) -> {
- if (oldValue == null || oldValue.getSeqNum() <= loc.getSeqNum()) {
- return loc;
+ private boolean addToCache(TableCache tableCache, RegionLocations locs) {
+ LOG.trace("Try adding {} to cache", locs);
+ byte[] startKey = locs.getDefaultRegionLocation().getRegion().getStartKey();
+ for (;;) {
+ RegionLocations oldLocs = tableCache.cache.putIfAbsent(startKey, locs);
+ if (oldLocs == null) {
+ return true;
}
- if (LOG.isTraceEnabled()) {
- LOG.trace("Will not add " + loc + " to cache because the old value " + oldValue +
+ RegionLocations mergedLocs = mergeRegionLocations(locs, oldLocs);
+ if (isEqual(mergedLocs, oldLocs)) {
+ // the merged one is the same with the old one, give up
+ LOG.trace("Will not add {} to cache because the old value {} " +
" is newer than us or has the same server name." +
- " Maybe it is updated before we replace it");
+ " Maybe it is updated before we replace it", locs, oldLocs);
+ return false;
}
- return oldValue;
- });
- }
-
- @edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "UPM_UNCALLED_PRIVATE_METHOD",
- justification = "Called by lambda expression")
- private void addToCache(HRegionLocation loc) {
- addToCache(getTableCache(loc.getRegion().getTable()), loc);
- LOG.trace("Try adding {} to cache", loc);
+ if (tableCache.cache.replace(startKey, oldLocs, mergedLocs)) {
+ return true;
+ }
+ }
}
- private void complete(TableName tableName, LocateRequest req, HRegionLocation loc,
+ private void complete(TableName tableName, LocateRequest req, RegionLocations locs,
Throwable error) {
if (error != null) {
LOG.warn("Failed to locate region in '" + tableName + "', row='" +
@@ -246,8 +258,8 @@ class AsyncNonMetaRegionLocator {
}
Optional<LocateRequest> toSend = Optional.empty();
TableCache tableCache = getTableCache(tableName);
- if (loc != null) {
- if (!addToCache(tableCache, loc)) {
+ if (locs != null) {
+ if (!addToCache(tableCache, locs)) {
// someone is ahead of us.
synchronized (tableCache) {
tableCache.pendingRequests.remove(req);
@@ -269,7 +281,7 @@ class AsyncNonMetaRegionLocator {
future.completeExceptionally(error);
}
}
- tableCache.clearCompletedRequests(Optional.ofNullable(loc));
+ tableCache.clearCompletedRequests(Optional.ofNullable(locs));
// Remove a complete locate request in a synchronized block, so the table cache must have
// quota to send a candidate request.
toSend = tableCache.getCandidate();
@@ -286,9 +298,11 @@ class AsyncNonMetaRegionLocator {
Bytes.toStringBinary(req.row), req.locateType, locs);
}
+ // the default region location should always be presented when fetching from meta, otherwise
+ // let's fail the request.
if (locs == null || locs.getDefaultRegionLocation() == null) {
complete(tableName, req, null,
- new IOException(String.format("No location found for '%s', row='%s', locateType=%s",
+ new HBaseIOException(String.format("No location found for '%s', row='%s', locateType=%s",
tableName, Bytes.toStringBinary(req.row), req.locateType)));
return true;
}
@@ -296,58 +310,60 @@ class AsyncNonMetaRegionLocator {
RegionInfo info = loc.getRegion();
if (info == null) {
complete(tableName, req, null,
- new IOException(String.format("HRegionInfo is null for '%s', row='%s', locateType=%s",
+ new HBaseIOException(String.format("HRegionInfo is null for '%s', row='%s', locateType=%s",
tableName, Bytes.toStringBinary(req.row), req.locateType)));
return true;
}
if (info.isSplitParent()) {
return false;
}
- if (loc.getServerName() == null) {
- complete(tableName, req, null,
- new IOException(
- String.format("No server address listed for region '%s', row='%s', locateType=%s",
- info.getRegionNameAsString(), Bytes.toStringBinary(req.row), req.locateType)));
- return true;
- }
- complete(tableName, req, loc, null);
+ complete(tableName, req, locs, null);
return true;
}
- private HRegionLocation locateRowInCache(TableCache tableCache, TableName tableName, byte[] row) {
- Map.Entry<byte[], HRegionLocation> entry = tableCache.cache.floorEntry(row);
+ private RegionLocations locateRowInCache(TableCache tableCache, TableName tableName, byte[] row,
+ int replicaId) {
+ Map.Entry<byte[], RegionLocations> entry = tableCache.cache.floorEntry(row);
if (entry == null) {
return null;
}
- HRegionLocation loc = entry.getValue();
+ RegionLocations locs = entry.getValue();
+ HRegionLocation loc = locs.getRegionLocation(replicaId);
+ if (loc == null) {
+ return null;
+ }
byte[] endKey = loc.getRegion().getEndKey();
if (isEmptyStopRow(endKey) || Bytes.compareTo(row, endKey) < 0) {
if (LOG.isTraceEnabled()) {
- LOG.trace("Found " + loc + " in cache for '" + tableName + "', row='" +
- Bytes.toStringBinary(row) + "', locateType=" + RegionLocateType.CURRENT);
+ LOG.trace("Found {} in cache for {}, row='{}', locateType={}, replicaId={}", loc, tableName,
+ Bytes.toStringBinary(row), RegionLocateType.CURRENT, replicaId);
}
- return loc;
+ return locs;
} else {
return null;
}
}
- private HRegionLocation locateRowBeforeInCache(TableCache tableCache, TableName tableName,
- byte[] row) {
+ private RegionLocations locateRowBeforeInCache(TableCache tableCache, TableName tableName,
+ byte[] row, int replicaId) {
boolean isEmptyStopRow = isEmptyStopRow(row);
- Map.Entry<byte[], HRegionLocation> entry =
- isEmptyStopRow ? tableCache.cache.lastEntry() : tableCache.cache.lowerEntry(row);
+ Map.Entry<byte[], RegionLocations> entry =
+ isEmptyStopRow ? tableCache.cache.lastEntry() : tableCache.cache.lowerEntry(row);
if (entry == null) {
return null;
}
- HRegionLocation loc = entry.getValue();
+ RegionLocations locs = entry.getValue();
+ HRegionLocation loc = locs.getRegionLocation(replicaId);
+ if (loc == null) {
+ return null;
+ }
if (isEmptyStopRow(loc.getRegion().getEndKey()) ||
(!isEmptyStopRow && Bytes.compareTo(loc.getRegion().getEndKey(), row) >= 0)) {
if (LOG.isTraceEnabled()) {
- LOG.trace("Found " + loc + " in cache for '" + tableName + "', row='" +
- Bytes.toStringBinary(row) + "', locateType=" + RegionLocateType.BEFORE);
+ LOG.trace("Found {} in cache for {}, row='{}', locateType={}, replicaId={}", loc, tableName,
+ Bytes.toStringBinary(row), RegionLocateType.BEFORE, replicaId);
}
- return loc;
+ return locs;
} else {
return null;
}
@@ -390,8 +406,8 @@ class AsyncNonMetaRegionLocator {
if (tableNotFound) {
complete(tableName, req, null, new TableNotFoundException(tableName));
} else if (!completeNormally) {
- complete(tableName, req, null, new IOException(
- "Unable to find region for " + Bytes.toStringBinary(req.row) + " in " + tableName));
+ complete(tableName, req, null, new IOException("Unable to find region for '" +
+ Bytes.toStringBinary(req.row) + "' in " + tableName));
}
}
@@ -423,13 +439,12 @@ class AsyncNonMetaRegionLocator {
continue;
}
RegionInfo info = loc.getRegion();
- if (info == null || info.isOffline() || info.isSplitParent() ||
- loc.getServerName() == null) {
+ if (info == null || info.isOffline() || info.isSplitParent()) {
continue;
}
- if (addToCache(tableCache, loc)) {
+ if (addToCache(tableCache, locs)) {
synchronized (tableCache) {
- tableCache.clearCompletedRequests(Optional.of(loc));
+ tableCache.clearCompletedRequests(Optional.of(locs));
}
}
}
@@ -438,36 +453,36 @@ class AsyncNonMetaRegionLocator {
});
}
- private HRegionLocation locateInCache(TableCache tableCache, TableName tableName, byte[] row,
- RegionLocateType locateType) {
+ private RegionLocations locateInCache(TableCache tableCache, TableName tableName, byte[] row,
+ int replicaId, RegionLocateType locateType) {
return locateType.equals(RegionLocateType.BEFORE)
- ? locateRowBeforeInCache(tableCache, tableName, row)
- : locateRowInCache(tableCache, tableName, row);
+ ? locateRowBeforeInCache(tableCache, tableName, row, replicaId)
+ : locateRowInCache(tableCache, tableName, row, replicaId);
}
// locateToPrevious is true means we will use the start key of a region to locate the region
// placed before it. Used for reverse scan. See the comment of
// AsyncRegionLocator.getPreviousRegionLocation.
- private CompletableFuture<HRegionLocation> getRegionLocationInternal(TableName tableName,
- byte[] row, RegionLocateType locateType, boolean reload) {
+ private CompletableFuture<RegionLocations> getRegionLocationsInternal(TableName tableName,
+ byte[] row, int replicaId, RegionLocateType locateType, boolean reload) {
// AFTER should be convert to CURRENT before calling this method
assert !locateType.equals(RegionLocateType.AFTER);
TableCache tableCache = getTableCache(tableName);
if (!reload) {
- HRegionLocation loc = locateInCache(tableCache, tableName, row, locateType);
- if (loc != null) {
- return CompletableFuture.completedFuture(loc);
+ RegionLocations locs = locateInCache(tableCache, tableName, row, replicaId, locateType);
+ if (isGood(locs, replicaId)) {
+ return CompletableFuture.completedFuture(locs);
}
}
- CompletableFuture<HRegionLocation> future;
+ CompletableFuture<RegionLocations> future;
LocateRequest req;
boolean sendRequest = false;
synchronized (tableCache) {
// check again
if (!reload) {
- HRegionLocation loc = locateInCache(tableCache, tableName, row, locateType);
- if (loc != null) {
- return CompletableFuture.completedFuture(loc);
+ RegionLocations locs = locateInCache(tableCache, tableName, row, replicaId, locateType);
+ if (isGood(locs, replicaId)) {
+ return CompletableFuture.completedFuture(locs);
}
}
req = new LocateRequest(row, locateType);
@@ -487,28 +502,58 @@ class AsyncNonMetaRegionLocator {
return future;
}
- CompletableFuture<HRegionLocation> getRegionLocation(TableName tableName, byte[] row,
- RegionLocateType locateType, boolean reload) {
- if (locateType.equals(RegionLocateType.BEFORE)) {
- return getRegionLocationInternal(tableName, row, locateType, reload);
- } else {
- // as we know the exact row after us, so we can just create the new row, and use the same
- // algorithm to locate it.
- if (locateType.equals(RegionLocateType.AFTER)) {
- row = createClosestRowAfter(row);
- }
- return getRegionLocationInternal(tableName, row, RegionLocateType.CURRENT, reload);
+ CompletableFuture<RegionLocations> getRegionLocations(TableName tableName, byte[] row,
+ int replicaId, RegionLocateType locateType, boolean reload) {
+ // as we know the exact row after us, so we can just create the new row, and use the same
+ // algorithm to locate it.
+ if (locateType.equals(RegionLocateType.AFTER)) {
+ row = createClosestRowAfter(row);
+ locateType = RegionLocateType.CURRENT;
}
+ return getRegionLocationsInternal(tableName, row, replicaId, locateType, reload);
}
- void updateCachedLocation(HRegionLocation loc, Throwable exception) {
- AsyncRegionLocator.updateCachedLocation(loc, exception, l -> {
- TableCache tableCache = cache.get(l.getRegion().getTable());
- if (tableCache == null) {
- return null;
+ private void removeLocationFromCache(HRegionLocation loc) {
+ TableCache tableCache = cache.get(loc.getRegion().getTable());
+ if (tableCache == null) {
+ return;
+ }
+ byte[] startKey = loc.getRegion().getStartKey();
+ for (;;) {
+ RegionLocations oldLocs = tableCache.cache.get(startKey);
+ HRegionLocation oldLoc = oldLocs.getRegionLocation(loc.getRegion().getReplicaId());
+ if (!canUpdateOnError(loc, oldLoc)) {
+ return;
}
- return tableCache.cache.get(l.getRegion().getStartKey());
- }, this::addToCache, this::removeFromCache);
+ RegionLocations newLocs = removeRegionLocation(oldLocs, loc.getRegion().getReplicaId());
+ if (newLocs == null) {
+ if (tableCache.cache.remove(startKey, oldLocs)) {
+ return;
+ }
+ } else {
+ if (tableCache.cache.replace(startKey, oldLocs, newLocs)) {
+ return;
+ }
+ }
+ }
+ }
+
+ private void addLocationToCache(HRegionLocation loc) {
+ addToCache(getTableCache(loc.getRegion().getTable()), createRegionLocations(loc));
+ }
+
+ private HRegionLocation getCachedLocation(HRegionLocation loc) {
+ TableCache tableCache = cache.get(loc.getRegion().getTable());
+ if (tableCache == null) {
+ return null;
+ }
+ RegionLocations locs = tableCache.cache.get(loc.getRegion().getStartKey());
+ return locs != null ? locs.getRegionLocation(loc.getRegion().getReplicaId()) : null;
+ }
+
+ void updateCachedLocationOnError(HRegionLocation loc, Throwable exception) {
+ AsyncRegionLocatorHelper.updateCachedLocationOnError(loc, exception, this::getCachedLocation,
+ this::addLocationToCache, this::removeLocationFromCache);
}
void clearCache(TableName tableName) {
@@ -526,11 +571,11 @@ class AsyncNonMetaRegionLocator {
// only used for testing whether we have cached the location for a region.
@VisibleForTesting
- HRegionLocation getRegionLocationInCache(TableName tableName, byte[] row) {
+ RegionLocations getRegionLocationInCache(TableName tableName, byte[] row) {
TableCache tableCache = cache.get(tableName);
if (tableCache == null) {
return null;
}
- return locateRowInCache(tableCache, tableName, row);
+ return locateRowInCache(tableCache, tableName, row, RegionReplicaUtil.DEFAULT_REPLICA_ID);
}
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/db66e6cc/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionLocator.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionLocator.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionLocator.java
index 56228ab..d624974 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionLocator.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionLocator.java
@@ -18,26 +18,24 @@
package org.apache.hadoop.hbase.client;
import static org.apache.hadoop.hbase.TableName.META_TABLE_NAME;
-import static org.apache.hadoop.hbase.exceptions.ClientExceptionsUtil.findException;
-import static org.apache.hadoop.hbase.exceptions.ClientExceptionsUtil.isMetaClearingException;
-
-import org.apache.hbase.thirdparty.io.netty.util.HashedWheelTimer;
-import org.apache.hbase.thirdparty.io.netty.util.Timeout;
+import static org.apache.hadoop.hbase.util.FutureUtils.addListener;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.TimeUnit;
-import java.util.function.Consumer;
-import java.util.function.Function;
import java.util.function.Supplier;
-
+import org.apache.hadoop.hbase.HBaseIOException;
import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.RegionException;
+import org.apache.hadoop.hbase.RegionLocations;
import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.exceptions.TimeoutIOException;
+import org.apache.hadoop.hbase.util.Bytes;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import org.apache.hadoop.hbase.exceptions.RegionMovedException;
-import org.apache.hadoop.hbase.exceptions.TimeoutIOException;
-import org.apache.hadoop.hbase.util.Bytes;
+
+import org.apache.hbase.thirdparty.io.netty.util.HashedWheelTimer;
+import org.apache.hbase.thirdparty.io.netty.util.Timeout;
/**
* The asynchronous region locator.
@@ -59,8 +57,8 @@ class AsyncRegionLocator {
this.retryTimer = retryTimer;
}
- private CompletableFuture<HRegionLocation> withTimeout(CompletableFuture<HRegionLocation> future,
- long timeoutNs, Supplier<String> timeoutMsg) {
+ private <T> CompletableFuture<T> withTimeout(CompletableFuture<T> future, long timeoutNs,
+ Supplier<String> timeoutMsg) {
if (future.isDone() || timeoutNs <= 0) {
return future;
}
@@ -78,74 +76,75 @@ class AsyncRegionLocator {
});
}
- CompletableFuture<HRegionLocation> getRegionLocation(TableName tableName, byte[] row,
+ private boolean isMeta(TableName tableName) {
+ return TableName.isMetaTableName(tableName);
+ }
+
+ CompletableFuture<RegionLocations> getRegionLocations(TableName tableName, byte[] row,
RegionLocateType type, boolean reload, long timeoutNs) {
+ CompletableFuture<RegionLocations> future = isMeta(tableName)
+ ? metaRegionLocator.getRegionLocations(RegionReplicaUtil.DEFAULT_REPLICA_ID, reload)
+ : nonMetaRegionLocator.getRegionLocations(tableName, row,
+ RegionReplicaUtil.DEFAULT_REPLICA_ID, type, reload);
+ return withTimeout(future, timeoutNs,
+ () -> "Timeout(" + TimeUnit.NANOSECONDS.toMillis(timeoutNs) +
+ "ms) waiting for region locations for " + tableName + ", row='" +
+ Bytes.toStringBinary(row) + "'");
+ }
+
+ CompletableFuture<HRegionLocation> getRegionLocation(TableName tableName, byte[] row,
+ int replicaId, RegionLocateType type, boolean reload, long timeoutNs) {
// meta region can not be split right now so we always call the same method.
// Change it later if the meta table can have more than one regions.
- CompletableFuture<HRegionLocation> future =
- tableName.equals(META_TABLE_NAME) ? metaRegionLocator.getRegionLocation(reload)
- : nonMetaRegionLocator.getRegionLocation(tableName, row, type, reload);
+ CompletableFuture<HRegionLocation> future = new CompletableFuture<>();
+ CompletableFuture<RegionLocations> locsFuture =
+ isMeta(tableName) ? metaRegionLocator.getRegionLocations(replicaId, reload)
+ : nonMetaRegionLocator.getRegionLocations(tableName, row, replicaId, type, reload);
+ addListener(locsFuture, (locs, error) -> {
+ if (error != null) {
+ future.completeExceptionally(error);
+ return;
+ }
+ HRegionLocation loc = locs.getRegionLocation(replicaId);
+ if (loc == null) {
+ future
+ .completeExceptionally(new RegionException("No location for " + tableName + ", row='" +
+ Bytes.toStringBinary(row) + "', locateType=" + type + ", replicaId=" + replicaId));
+ } else if (loc.getServerName() == null) {
+ future.completeExceptionally(new HBaseIOException("No server address listed for region '" +
+ loc.getRegion().getRegionNameAsString() + ", row='" + Bytes.toStringBinary(row) +
+ "', locateType=" + type + ", replicaId=" + replicaId));
+ } else {
+ future.complete(loc);
+ }
+ });
return withTimeout(future, timeoutNs,
() -> "Timeout(" + TimeUnit.NANOSECONDS.toMillis(timeoutNs) +
- "ms) waiting for region location for " + tableName + ", row='" +
- Bytes.toStringBinary(row) + "'");
+ "ms) waiting for region location for " + tableName + ", row='" + Bytes.toStringBinary(row) +
+ "', replicaId=" + replicaId);
}
CompletableFuture<HRegionLocation> getRegionLocation(TableName tableName, byte[] row,
- RegionLocateType type, long timeoutNs) {
- return getRegionLocation(tableName, row, type, false, timeoutNs);
+ int replicaId, RegionLocateType type, long timeoutNs) {
+ return getRegionLocation(tableName, row, replicaId, type, false, timeoutNs);
}
- static boolean canUpdate(HRegionLocation loc, HRegionLocation oldLoc) {
- // Do not need to update if no such location, or the location is newer, or the location is not
- // same with us
- return oldLoc != null && oldLoc.getSeqNum() <= loc.getSeqNum() &&
- oldLoc.getServerName().equals(loc.getServerName());
+ CompletableFuture<HRegionLocation> getRegionLocation(TableName tableName, byte[] row,
+ RegionLocateType type, boolean reload, long timeoutNs) {
+ return getRegionLocation(tableName, row, RegionReplicaUtil.DEFAULT_REPLICA_ID, type, reload,
+ timeoutNs);
}
- static void updateCachedLocation(HRegionLocation loc, Throwable exception,
- Function<HRegionLocation, HRegionLocation> cachedLocationSupplier,
- Consumer<HRegionLocation> addToCache, Consumer<HRegionLocation> removeFromCache) {
- HRegionLocation oldLoc = cachedLocationSupplier.apply(loc);
- if (LOG.isDebugEnabled()) {
- LOG.debug("Try updating " + loc + ", the old value is " + oldLoc, exception);
- }
- if (!canUpdate(loc, oldLoc)) {
- return;
- }
- Throwable cause = findException(exception);
- if (LOG.isDebugEnabled()) {
- LOG.debug("The actual exception when updating " + loc, cause);
- }
- if (cause == null || !isMetaClearingException(cause)) {
- if (LOG.isDebugEnabled()) {
- LOG.debug(
- "Will not update " + loc + " because the exception is null or not the one we care about");
- }
- return;
- }
- if (cause instanceof RegionMovedException) {
- RegionMovedException rme = (RegionMovedException) cause;
- HRegionLocation newLoc =
- new HRegionLocation(loc.getRegionInfo(), rme.getServerName(), rme.getLocationSeqNum());
- if (LOG.isDebugEnabled()) {
- LOG.debug(
- "Try updating " + loc + " with the new location " + newLoc + " constructed by " + rme);
- }
- addToCache.accept(newLoc);
- } else {
- if (LOG.isDebugEnabled()) {
- LOG.debug("Try removing " + loc + " from cache");
- }
- removeFromCache.accept(loc);
- }
+ CompletableFuture<HRegionLocation> getRegionLocation(TableName tableName, byte[] row,
+ RegionLocateType type, long timeoutNs) {
+ return getRegionLocation(tableName, row, type, false, timeoutNs);
}
- void updateCachedLocation(HRegionLocation loc, Throwable exception) {
+ void updateCachedLocationOnError(HRegionLocation loc, Throwable exception) {
if (loc.getRegion().isMetaRegion()) {
- metaRegionLocator.updateCachedLocation(loc, exception);
+ metaRegionLocator.updateCachedLocationOnError(loc, exception);
} else {
- nonMetaRegionLocator.updateCachedLocation(loc, exception);
+ nonMetaRegionLocator.updateCachedLocationOnError(loc, exception);
}
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/db66e6cc/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionLocatorHelper.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionLocatorHelper.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionLocatorHelper.java
new file mode 100644
index 0000000..5c9c091
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionLocatorHelper.java
@@ -0,0 +1,147 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.client;
+
+import static org.apache.hadoop.hbase.exceptions.ClientExceptionsUtil.findException;
+import static org.apache.hadoop.hbase.exceptions.ClientExceptionsUtil.isMetaClearingException;
+
+import java.util.Arrays;
+import java.util.function.Consumer;
+import java.util.function.Function;
+import org.apache.commons.lang3.ObjectUtils;
+import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.RegionLocations;
+import org.apache.hadoop.hbase.exceptions.RegionMovedException;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Helper class for asynchronous region locator.
+ */
+@InterfaceAudience.Private
+final class AsyncRegionLocatorHelper {
+
+ private static final Logger LOG = LoggerFactory.getLogger(AsyncRegionLocatorHelper.class);
+
+ private AsyncRegionLocatorHelper() {
+ }
+
+ static boolean canUpdateOnError(HRegionLocation loc, HRegionLocation oldLoc) {
+ // Do not need to update if no such location, or the location is newer, or the location is not
+ // the same with us
+ return oldLoc != null && oldLoc.getSeqNum() <= loc.getSeqNum() &&
+ oldLoc.getServerName().equals(loc.getServerName());
+ }
+
+ static void updateCachedLocationOnError(HRegionLocation loc, Throwable exception,
+ Function<HRegionLocation, HRegionLocation> cachedLocationSupplier,
+ Consumer<HRegionLocation> addToCache, Consumer<HRegionLocation> removeFromCache) {
+ HRegionLocation oldLoc = cachedLocationSupplier.apply(loc);
+ LOG.debug("Try updating {} , the old value is {}", loc, oldLoc, exception);
+ if (!canUpdateOnError(loc, oldLoc)) {
+ return;
+ }
+ Throwable cause = findException(exception);
+ LOG.debug("The actual exception when updating {}", loc, cause);
+ if (cause == null || !isMetaClearingException(cause)) {
+ LOG.debug("Will not update {} because the exception is null or not the one we care about",
+ loc);
+ return;
+ }
+ if (cause instanceof RegionMovedException) {
+ RegionMovedException rme = (RegionMovedException) cause;
+ HRegionLocation newLoc =
+ new HRegionLocation(loc.getRegion(), rme.getServerName(), rme.getLocationSeqNum());
+ LOG.debug("Try updating {} with the new location {} constructed by {}", loc, newLoc, rme);
+ addToCache.accept(newLoc);
+ } else {
+ LOG.debug("Try removing {} from cache", loc);
+ removeFromCache.accept(loc);
+ }
+ }
+
+ static RegionLocations createRegionLocations(HRegionLocation loc) {
+ int replicaId = loc.getRegion().getReplicaId();
+ HRegionLocation[] locs = new HRegionLocation[replicaId + 1];
+ locs[replicaId] = loc;
+ return new RegionLocations(locs);
+ }
+
+ /**
+ * Create a new {@link RegionLocations} based on the given {@code oldLocs}, and replace the
+ * location for the given {@code replicaId} with the given {@code loc}.
+ * <p/>
+ * All the {@link RegionLocations} in async locator related class are immutable because we want to
+ * access them concurrently, so here we need to create a new one, instead of calling
+ * {@link RegionLocations#updateLocation(HRegionLocation, boolean, boolean)}.
+ */
+ static RegionLocations replaceRegionLocation(RegionLocations oldLocs, HRegionLocation loc) {
+ int replicaId = loc.getRegion().getReplicaId();
+ HRegionLocation[] locs = oldLocs.getRegionLocations();
+ locs = Arrays.copyOf(locs, Math.max(replicaId + 1, locs.length));
+ locs[replicaId] = loc;
+ return new RegionLocations(locs);
+ }
+
+ /**
+ * Create a new {@link RegionLocations} based on the given {@code oldLocs}, and remove the
+ * location for the given {@code replicaId}.
+ * <p/>
+ * All the {@link RegionLocations} in async locator related class are immutable because we want to
+ * access them concurrently, so here we need to create a new one, instead of calling
+ * {@link RegionLocations#remove(int)}.
+ */
+ static RegionLocations removeRegionLocation(RegionLocations oldLocs, int replicaId) {
+ HRegionLocation[] locs = oldLocs.getRegionLocations();
+ if (locs.length < replicaId + 1) {
+ // Here we do not modify the oldLocs so it is safe to return it.
+ return oldLocs;
+ }
+ locs = Arrays.copyOf(locs, locs.length);
+ locs[replicaId] = null;
+ if (ObjectUtils.firstNonNull(locs) != null) {
+ return new RegionLocations(locs);
+ } else {
+ // if all the locations are null, just return null
+ return null;
+ }
+ }
+
+ /**
+ * Create a new {@link RegionLocations} which is the merging result for the given two
+ * {@link RegionLocations}.
+ * <p/>
+ * All the {@link RegionLocations} in async locator related class are immutable because we want to
+ * access them concurrently, so here we need to create a new one, instead of calling
+ * {@link RegionLocations#mergeLocations(RegionLocations)} directly.
+ */
+ static RegionLocations mergeRegionLocations(RegionLocations newLocs, RegionLocations oldLocs) {
+ RegionLocations locs = new RegionLocations(newLocs.getRegionLocations());
+ locs.mergeLocations(oldLocs);
+ return locs;
+ }
+
+ static boolean isGood(RegionLocations locs, int replicaId) {
+ if (locs == null) {
+ return false;
+ }
+ HRegionLocation loc = locs.getRegionLocation(replicaId);
+ return loc != null && loc.getServerName() != null;
+ }
+}
http://git-wip-us.apache.org/repos/asf/hbase/blob/db66e6cc/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRpcRetryingCaller.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRpcRetryingCaller.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRpcRetryingCaller.java
index d30012f..e03049a 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRpcRetryingCaller.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRpcRetryingCaller.java
@@ -88,15 +88,15 @@ public abstract class AsyncRpcRetryingCaller<T> {
return TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startNs);
}
- protected long remainingTimeNs() {
+ protected final long remainingTimeNs() {
return operationTimeoutNs - (System.nanoTime() - startNs);
}
- protected void completeExceptionally() {
+ protected final void completeExceptionally() {
future.completeExceptionally(new RetriesExhaustedException(tries - 1, exceptions));
}
- protected void resetCallTimeout() {
+ protected final void resetCallTimeout() {
long callTimeoutNs;
if (operationTimeoutNs > 0) {
callTimeoutNs = remainingTimeNs();
@@ -111,8 +111,15 @@ public abstract class AsyncRpcRetryingCaller<T> {
resetController(controller, callTimeoutNs);
}
- protected void onError(Throwable error, Supplier<String> errMsg,
+ protected final void onError(Throwable error, Supplier<String> errMsg,
Consumer<Throwable> updateCachedLocation) {
+ if (future.isDone()) {
+ // Give up if the future is already done, this is possible if user has already canceled the
+ // future. And for timeline consistent read, we will also cancel some requests if we have
+ // already get one of the responses.
+ LOG.debug("The future is already done, canceled={}, give up retrying", future.isCancelled());
+ return;
+ }
error = translateException(error);
if (error instanceof DoNotRetryIOException) {
future.completeExceptionally(error);
http://git-wip-us.apache.org/repos/asf/hbase/blob/db66e6cc/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRpcRetryingCallerFactory.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRpcRetryingCallerFactory.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRpcRetryingCallerFactory.java
index f80b4e5..a660e74 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRpcRetryingCallerFactory.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRpcRetryingCallerFactory.java
@@ -17,22 +17,22 @@
*/
package org.apache.hadoop.hbase.client;
+import static org.apache.hadoop.hbase.client.ConnectionUtils.retries2Attempts;
import static org.apache.hbase.thirdparty.com.google.common.base.Preconditions.checkArgument;
import static org.apache.hbase.thirdparty.com.google.common.base.Preconditions.checkNotNull;
-import static org.apache.hadoop.hbase.client.ConnectionUtils.retries2Attempts;
-
-import org.apache.hbase.thirdparty.io.netty.util.HashedWheelTimer;
import java.util.List;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.TimeUnit;
-
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.client.metrics.ScanMetrics;
import org.apache.hadoop.hbase.ipc.HBaseRpcController;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import org.apache.hbase.thirdparty.io.netty.util.HashedWheelTimer;
+
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientService;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanResponse;
@@ -75,6 +75,8 @@ class AsyncRpcRetryingCallerFactory {
private RegionLocateType locateType = RegionLocateType.CURRENT;
+ private int replicaId = RegionReplicaUtil.DEFAULT_REPLICA_ID;
+
public SingleRequestCallerBuilder<T> table(TableName tableName) {
this.tableName = tableName;
return this;
@@ -121,11 +123,17 @@ class AsyncRpcRetryingCallerFactory {
return this;
}
+ public SingleRequestCallerBuilder<T> replicaId(int replicaId) {
+ this.replicaId = replicaId;
+ return this;
+ }
+
public AsyncSingleRequestRpcRetryingCaller<T> build() {
+ checkArgument(replicaId >= 0, "invalid replica id %s", replicaId);
return new AsyncSingleRequestRpcRetryingCaller<>(retryTimer, conn,
- checkNotNull(tableName, "tableName is null"), checkNotNull(row, "row is null"),
- checkNotNull(locateType, "locateType is null"), checkNotNull(callable, "action is null"),
- pauseNs, maxAttempts, operationTimeoutNs, rpcTimeoutNs, startLogErrorsCnt);
+ checkNotNull(tableName, "tableName is null"), checkNotNull(row, "row is null"), replicaId,
+ checkNotNull(locateType, "locateType is null"), checkNotNull(callable, "action is null"),
+ pauseNs, maxAttempts, operationTimeoutNs, rpcTimeoutNs, startLogErrorsCnt);
}
/**
@@ -241,11 +249,11 @@ class AsyncRpcRetryingCallerFactory {
public AsyncScanSingleRegionRpcRetryingCaller build() {
checkArgument(scannerId != null, "invalid scannerId %d", scannerId);
return new AsyncScanSingleRegionRpcRetryingCaller(retryTimer, conn,
- checkNotNull(scan, "scan is null"), scanMetrics, scannerId,
- checkNotNull(resultCache, "resultCache is null"),
- checkNotNull(consumer, "consumer is null"), checkNotNull(stub, "stub is null"),
- checkNotNull(loc, "location is null"), isRegionServerRemote, scannerLeaseTimeoutPeriodNs,
- pauseNs, maxAttempts, scanTimeoutNs, rpcTimeoutNs, startLogErrorsCnt);
+ checkNotNull(scan, "scan is null"), scanMetrics, scannerId,
+ checkNotNull(resultCache, "resultCache is null"),
+ checkNotNull(consumer, "consumer is null"), checkNotNull(stub, "stub is null"),
+ checkNotNull(loc, "location is null"), isRegionServerRemote, scannerLeaseTimeoutPeriodNs,
+ pauseNs, maxAttempts, scanTimeoutNs, rpcTimeoutNs, startLogErrorsCnt);
}
/**
@@ -311,7 +319,7 @@ class AsyncRpcRetryingCallerFactory {
public <T> AsyncBatchRpcRetryingCaller<T> build() {
return new AsyncBatchRpcRetryingCaller<>(retryTimer, conn, tableName, actions, pauseNs,
- maxAttempts, operationTimeoutNs, rpcTimeoutNs, startLogErrorsCnt);
+ maxAttempts, operationTimeoutNs, rpcTimeoutNs, startLogErrorsCnt);
}
public <T> List<CompletableFuture<T>> call() {
@@ -363,8 +371,8 @@ class AsyncRpcRetryingCallerFactory {
public AsyncMasterRequestRpcRetryingCaller<T> build() {
return new AsyncMasterRequestRpcRetryingCaller<T>(retryTimer, conn,
- checkNotNull(callable, "action is null"), pauseNs, maxAttempts, operationTimeoutNs,
- rpcTimeoutNs, startLogErrorsCnt);
+ checkNotNull(callable, "action is null"), pauseNs, maxAttempts, operationTimeoutNs,
+ rpcTimeoutNs, startLogErrorsCnt);
}
/**
@@ -390,7 +398,8 @@ class AsyncRpcRetryingCallerFactory {
private ServerName serverName;
- public AdminRequestCallerBuilder<T> action(AsyncAdminRequestRetryingCaller.Callable<T> callable) {
+ public AdminRequestCallerBuilder<T> action(
+ AsyncAdminRequestRetryingCaller.Callable<T> callable) {
this.callable = callable;
return this;
}
@@ -420,15 +429,15 @@ class AsyncRpcRetryingCallerFactory {
return this;
}
- public AdminRequestCallerBuilder<T> serverName(ServerName serverName){
+ public AdminRequestCallerBuilder<T> serverName(ServerName serverName) {
this.serverName = serverName;
return this;
}
public AsyncAdminRequestRetryingCaller<T> build() {
return new AsyncAdminRequestRetryingCaller<T>(retryTimer, conn, pauseNs, maxAttempts,
- operationTimeoutNs, rpcTimeoutNs, startLogErrorsCnt, checkNotNull(serverName,
- "serverName is null"), checkNotNull(callable, "action is null"));
+ operationTimeoutNs, rpcTimeoutNs, startLogErrorsCnt,
+ checkNotNull(serverName, "serverName is null"), checkNotNull(callable, "action is null"));
}
public CompletableFuture<T> call() {
@@ -436,7 +445,7 @@ class AsyncRpcRetryingCallerFactory {
}
}
- public <T> AdminRequestCallerBuilder<T> adminRequest(){
+ public <T> AdminRequestCallerBuilder<T> adminRequest() {
return new AdminRequestCallerBuilder<>();
}
@@ -488,8 +497,8 @@ class AsyncRpcRetryingCallerFactory {
public AsyncServerRequestRpcRetryingCaller<T> build() {
return new AsyncServerRequestRpcRetryingCaller<T>(retryTimer, conn, pauseNs, maxAttempts,
- operationTimeoutNs, rpcTimeoutNs, startLogErrorsCnt, checkNotNull(serverName,
- "serverName is null"), checkNotNull(callable, "action is null"));
+ operationTimeoutNs, rpcTimeoutNs, startLogErrorsCnt,
+ checkNotNull(serverName, "serverName is null"), checkNotNull(callable, "action is null"));
}
public CompletableFuture<T> call() {
http://git-wip-us.apache.org/repos/asf/hbase/blob/db66e6cc/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncSingleRequestRpcRetryingCaller.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncSingleRequestRpcRetryingCaller.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncSingleRequestRpcRetryingCaller.java
index 56c82fb..1a52e5c 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncSingleRequestRpcRetryingCaller.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncSingleRequestRpcRetryingCaller.java
@@ -17,17 +17,19 @@
*/
package org.apache.hadoop.hbase.client;
-import org.apache.hbase.thirdparty.io.netty.util.HashedWheelTimer;
+import static org.apache.hadoop.hbase.util.FutureUtils.addListener;
import java.io.IOException;
import java.util.concurrent.CompletableFuture;
-
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.TableName;
-import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.ipc.HBaseRpcController;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientService;
import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import org.apache.hbase.thirdparty.io.netty.util.HashedWheelTimer;
+
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientService;
/**
* Retry caller for a single request, such as get, put, delete, etc.
@@ -45,18 +47,21 @@ class AsyncSingleRequestRpcRetryingCaller<T> extends AsyncRpcRetryingCaller<T> {
private final byte[] row;
+ private final int replicaId;
+
private final RegionLocateType locateType;
private final Callable<T> callable;
public AsyncSingleRequestRpcRetryingCaller(HashedWheelTimer retryTimer, AsyncConnectionImpl conn,
- TableName tableName, byte[] row, RegionLocateType locateType, Callable<T> callable,
- long pauseNs, int maxAttempts, long operationTimeoutNs, long rpcTimeoutNs,
- int startLogErrorsCnt) {
+ TableName tableName, byte[] row, int replicaId, RegionLocateType locateType,
+ Callable<T> callable, long pauseNs, int maxAttempts, long operationTimeoutNs,
+ long rpcTimeoutNs, int startLogErrorsCnt) {
super(retryTimer, conn, pauseNs, maxAttempts, operationTimeoutNs, rpcTimeoutNs,
- startLogErrorsCnt);
+ startLogErrorsCnt);
this.tableName = tableName;
this.row = row;
+ this.replicaId = replicaId;
this.locateType = locateType;
this.callable = callable;
}
@@ -67,23 +72,22 @@ class AsyncSingleRequestRpcRetryingCaller<T> extends AsyncRpcRetryingCaller<T> {
stub = conn.getRegionServerStub(loc.getServerName());
} catch (IOException e) {
onError(e,
- () -> "Get async stub to " + loc.getServerName() + " for '" + Bytes.toStringBinary(row)
- + "' in " + loc.getRegion().getEncodedName() + " of " + tableName + " failed",
- err -> conn.getLocator().updateCachedLocation(loc, err));
+ () -> "Get async stub to " + loc.getServerName() + " for '" + Bytes.toStringBinary(row) +
+ "' in " + loc.getRegion().getEncodedName() + " of " + tableName + " failed",
+ err -> conn.getLocator().updateCachedLocationOnError(loc, err));
return;
}
resetCallTimeout();
- callable.call(controller, loc, stub).whenComplete(
- (result, error) -> {
- if (error != null) {
- onError(error,
- () -> "Call to " + loc.getServerName() + " for '" + Bytes.toStringBinary(row) + "' in "
- + loc.getRegion().getEncodedName() + " of " + tableName + " failed",
- err -> conn.getLocator().updateCachedLocation(loc, err));
- return;
- }
- future.complete(result);
- });
+ callable.call(controller, loc, stub).whenComplete((result, error) -> {
+ if (error != null) {
+ onError(error,
+ () -> "Call to " + loc.getServerName() + " for '" + Bytes.toStringBinary(row) + "' in " +
+ loc.getRegion().getEncodedName() + " of " + tableName + " failed",
+ err -> conn.getLocator().updateCachedLocationOnError(loc, err));
+ return;
+ }
+ future.complete(result);
+ });
}
@Override
@@ -98,18 +102,17 @@ class AsyncSingleRequestRpcRetryingCaller<T> extends AsyncRpcRetryingCaller<T> {
} else {
locateTimeoutNs = -1L;
}
- conn.getLocator()
- .getRegionLocation(tableName, row, locateType, locateTimeoutNs)
- .whenComplete(
- (loc, error) -> {
- if (error != null) {
- onError(error, () -> "Locate '" + Bytes.toStringBinary(row) + "' in " + tableName
- + " failed", err -> {
- });
- return;
- }
- call(loc);
- });
+ addListener(
+ conn.getLocator().getRegionLocation(tableName, row, replicaId, locateType, locateTimeoutNs),
+ (loc, error) -> {
+ if (error != null) {
+ onError(error,
+ () -> "Locate '" + Bytes.toStringBinary(row) + "' in " + tableName + " failed", err -> {
+ });
+ return;
+ }
+ call(loc);
+ });
}
@Override
http://git-wip-us.apache.org/repos/asf/hbase/blob/db66e6cc/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableRegionLocator.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableRegionLocator.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableRegionLocator.java
index dbfcef5..3bda38e 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableRegionLocator.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableRegionLocator.java
@@ -18,7 +18,6 @@
package org.apache.hadoop.hbase.client;
import java.util.concurrent.CompletableFuture;
-
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.TableName;
import org.apache.yetus.audience.InterfaceAudience;
@@ -55,5 +54,30 @@ public interface AsyncTableRegionLocator {
* @param row Row to find.
* @param reload true to reload information or false to use cached information
*/
- CompletableFuture<HRegionLocation> getRegionLocation(byte[] row, boolean reload);
+ default CompletableFuture<HRegionLocation> getRegionLocation(byte[] row, boolean reload) {
+ return getRegionLocation(row, RegionReplicaUtil.DEFAULT_REPLICA_ID, reload);
+ }
+
+ /**
+ * Finds the region with the given <code>replicaId</code> on which the given row is being served.
+ * <p>
+ * Returns the location of the region with the given <code>replicaId</code> to which the row
+ * belongs.
+ * @param row Row to find.
+ * @param replicaId the replica id of the region
+ */
+ default CompletableFuture<HRegionLocation> getRegionLocation(byte[] row, int replicaId) {
+ return getRegionLocation(row, replicaId, false);
+ }
+
+ /**
+ * Finds the region with the given <code>replicaId</code> on which the given row is being served.
+ * <p>
+ * Returns the location of the region with the given <code>replicaId</code> to which the row
+ * belongs.
+ * @param row Row to find.
+ * @param replicaId the replica id of the region
+ * @param reload true to reload information or false to use cached information
+ */
+ CompletableFuture<HRegionLocation> getRegionLocation(byte[] row, int replicaId, boolean reload);
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/db66e6cc/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableRegionLocatorImpl.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableRegionLocatorImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableRegionLocatorImpl.java
index 7d199df..465a411 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableRegionLocatorImpl.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableRegionLocatorImpl.java
@@ -44,7 +44,9 @@ class AsyncTableRegionLocatorImpl implements AsyncTableRegionLocator {
}
@Override
- public CompletableFuture<HRegionLocation> getRegionLocation(byte[] row, boolean reload) {
- return locator.getRegionLocation(tableName, row, RegionLocateType.CURRENT, reload, -1L);
+ public CompletableFuture<HRegionLocation> getRegionLocation(byte[] row, int replicaId,
+ boolean reload) {
+ return locator.getRegionLocation(tableName, row, replicaId, RegionLocateType.CURRENT, reload,
+ -1L);
}
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/db66e6cc/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionConfiguration.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionConfiguration.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionConfiguration.java
index d996004..55c62e7 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionConfiguration.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionConfiguration.java
@@ -38,6 +38,9 @@ public class ConnectionConfiguration {
public static final long WRITE_BUFFER_PERIODIC_FLUSH_TIMERTICK_MS_DEFAULT = 1000L; // 1 second
public static final String MAX_KEYVALUE_SIZE_KEY = "hbase.client.keyvalue.maxsize";
public static final int MAX_KEYVALUE_SIZE_DEFAULT = 10485760;
+ public static final String PRIMARY_CALL_TIMEOUT_MICROSECOND =
+ "hbase.client.primaryCallTimeout.get";
+ public static final int PRIMARY_CALL_TIMEOUT_MICROSECOND_DEFAULT = 10000; // 10ms
private final long writeBufferSize;
private final long writeBufferPeriodicFlushTimeoutMs;
@@ -86,7 +89,7 @@ public class ConnectionConfiguration {
HConstants.DEFAULT_HBASE_CLIENT_SCANNER_MAX_RESULT_SIZE);
this.primaryCallTimeoutMicroSecond =
- conf.getInt("hbase.client.primaryCallTimeout.get", 10000); // 10ms
+ conf.getInt(PRIMARY_CALL_TIMEOUT_MICROSECOND, PRIMARY_CALL_TIMEOUT_MICROSECOND_DEFAULT);
this.replicaCallTimeoutMicroSecondScan =
conf.getInt("hbase.client.replicaCallTimeout.scan", 1000000); // 1000 ms
[09/15] hbase git commit: HBASE-21662 Add
append_peer_exclude_namespaces and remove_peer_exclude_namespaces shell
commands
Posted by zh...@apache.org.
HBASE-21662 Add append_peer_exclude_namespaces and remove_peer_exclude_namespaces shell commands
Signed-off-by: Guanghao Zhang <zg...@apache.org>
Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/466fa920
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/466fa920
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/466fa920
Branch: refs/heads/HBASE-21512
Commit: 466fa920fee572fe20db3b77ebf539dc304d5f31
Parents: db66e6c
Author: meiyi <my...@gamil.com>
Authored: Wed Jan 2 14:08:22 2019 +0800
Committer: Guanghao Zhang <zg...@apache.org>
Committed: Thu Jan 3 10:21:43 2019 +0800
----------------------------------------------------------------------
.../src/main/ruby/hbase/replication_admin.rb | 39 +++++++++++
hbase-shell/src/main/ruby/shell.rb | 2 +
.../commands/append_peer_exclude_namespaces.rb | 47 +++++++++++++
.../commands/remove_peer_exclude_namespaces.rb | 45 ++++++++++++
.../test/ruby/hbase/replication_admin_test.rb | 74 ++++++++++++++++++++
5 files changed, 207 insertions(+)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/hbase/blob/466fa920/hbase-shell/src/main/ruby/hbase/replication_admin.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/hbase/replication_admin.rb b/hbase-shell/src/main/ruby/hbase/replication_admin.rb
index c01b6ea..e061168 100644
--- a/hbase-shell/src/main/ruby/hbase/replication_admin.rb
+++ b/hbase-shell/src/main/ruby/hbase/replication_admin.rb
@@ -285,6 +285,45 @@ module Hbase
end
end
+ # Append exclude namespaces config for the specified peer
+ def append_peer_exclude_namespaces(id, namespaces)
+ unless namespaces.nil?
+ rpc = get_peer_config(id)
+ unless rpc.nil?
+ if rpc.getExcludeNamespaces.nil?
+ ns_set = java.util.HashSet.new
+ else
+ ns_set = java.util.HashSet.new(rpc.getExcludeNamespaces)
+ end
+ namespaces.each do |n|
+ ns_set.add(n)
+ end
+ builder = ReplicationPeerConfig.newBuilder(rpc)
+ builder.setExcludeNamespaces(ns_set)
+ @admin.updateReplicationPeerConfig(id, builder.build)
+ end
+ end
+ end
+
+ # Remove exclude namespaces config for the specified peer
+ def remove_peer_exclude_namespaces(id, namespaces)
+ unless namespaces.nil?
+ rpc = get_peer_config(id)
+ unless rpc.nil?
+ ns_set = rpc.getExcludeNamespaces
+ unless ns_set.nil?
+ ns_set = java.util.HashSet.new(ns_set)
+ namespaces.each do |n|
+ ns_set.remove(n)
+ end
+ end
+ builder = ReplicationPeerConfig.newBuilder(rpc)
+ builder.setExcludeNamespaces(ns_set)
+ @admin.updateReplicationPeerConfig(id, builder.build)
+ end
+ end
+ end
+
def set_peer_replicate_all(id, replicate_all)
rpc = get_peer_config(id)
return if rpc.nil?
http://git-wip-us.apache.org/repos/asf/hbase/blob/466fa920/hbase-shell/src/main/ruby/shell.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/shell.rb b/hbase-shell/src/main/ruby/shell.rb
index 1507ca3..1f7eae6 100644
--- a/hbase-shell/src/main/ruby/shell.rb
+++ b/hbase-shell/src/main/ruby/shell.rb
@@ -384,6 +384,8 @@ Shell.load_command_group(
append_peer_namespaces
remove_peer_namespaces
set_peer_exclude_namespaces
+ append_peer_exclude_namespaces
+ remove_peer_exclude_namespaces
show_peer_tableCFs
set_peer_tableCFs
set_peer_exclude_tableCFs
http://git-wip-us.apache.org/repos/asf/hbase/blob/466fa920/hbase-shell/src/main/ruby/shell/commands/append_peer_exclude_namespaces.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/shell/commands/append_peer_exclude_namespaces.rb b/hbase-shell/src/main/ruby/shell/commands/append_peer_exclude_namespaces.rb
new file mode 100644
index 0000000..4f500c8
--- /dev/null
+++ b/hbase-shell/src/main/ruby/shell/commands/append_peer_exclude_namespaces.rb
@@ -0,0 +1,47 @@
+#
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+module Shell
+ module Commands
+ class AppendPeerExcludeNamespaces < Command
+ def help
+ <<-EOF
+Append the namespaces which not replicated for the specified peer.
+
+Note:
+ 1. The replicate_all flag need to be true when append exclude namespaces.
+ 2. Append a exclude namespace in the peer config means that all tables in this
+ namespace will not be replicated to the peer cluster. If peer config
+ already has a exclude table, then not allow append this table's namespace
+ as a exclude namespace.
+
+Examples:
+
+ # append ns1,ns2 to be not replicable for peer '2'.
+ hbase> append_peer_exclude_namespaces '2', ["ns1", "ns2"]
+
+ EOF
+ end
+
+ def command(id, namespaces)
+ replication_admin.append_peer_exclude_namespaces(id, namespaces)
+ end
+ end
+ end
+end
http://git-wip-us.apache.org/repos/asf/hbase/blob/466fa920/hbase-shell/src/main/ruby/shell/commands/remove_peer_exclude_namespaces.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/shell/commands/remove_peer_exclude_namespaces.rb b/hbase-shell/src/main/ruby/shell/commands/remove_peer_exclude_namespaces.rb
new file mode 100644
index 0000000..0298ce9
--- /dev/null
+++ b/hbase-shell/src/main/ruby/shell/commands/remove_peer_exclude_namespaces.rb
@@ -0,0 +1,45 @@
+#
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+module Shell
+ module Commands
+ class RemovePeerExcludeNamespaces < Command
+ def help
+ <<-EOF
+Remove the namespaces which not replicated for the specified peer.
+
+Note:
+ 1. The replicate_all flag need to be true when remove exclude namespaces.
+ 2. Remove a exclude namespace in the peer config means that all tables in this
+ namespace will be replicated to the peer cluster.
+
+Examples:
+
+ # remove ns1 from the not replicable namespaces for peer '2'.
+ hbase> remove_peer_exclude_namespaces '2', ["ns1", "ns2"]
+
+ EOF
+ end
+
+ def command(id, namespaces)
+ replication_admin.remove_peer_exclude_namespaces(id, namespaces)
+ end
+ end
+ end
+end
http://git-wip-us.apache.org/repos/asf/hbase/blob/466fa920/hbase-shell/src/test/ruby/hbase/replication_admin_test.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/test/ruby/hbase/replication_admin_test.rb b/hbase-shell/src/test/ruby/hbase/replication_admin_test.rb
index f4c771e..85b4537 100644
--- a/hbase-shell/src/test/ruby/hbase/replication_admin_test.rb
+++ b/hbase-shell/src/test/ruby/hbase/replication_admin_test.rb
@@ -641,6 +641,80 @@ module Hbase
assert_equal("value2", Bytes.to_string(peer_config.get_peer_data.get(Bytes.toBytes("data2"))))
end
+ define_test "append_peer_exclude_namespaces: works with namespaces array" do
+ cluster_key = "zk4,zk5,zk6:11000:/hbase-test"
+ args = {CLUSTER_KEY => cluster_key}
+ command(:add_peer, @peer_id, args)
+ command(:set_peer_replicate_all, @peer_id, true)
+
+ namespaces = ["ns1", "ns2"]
+ namespaces_str = "!ns1;ns2"
+ command(:append_peer_exclude_namespaces, @peer_id, namespaces)
+ assert_equal(1, command(:list_peers).length)
+ assert_equal(@peer_id, command(:list_peers).get(0).getPeerId)
+ peer_config = command(:list_peers).get(0).getPeerConfig
+ assert_equal(namespaces_str,
+ replication_admin.show_peer_exclude_namespaces(peer_config))
+
+ namespaces = ["ns3"]
+ namespaces_str = "!ns1;ns2;ns3"
+ command(:append_peer_exclude_namespaces, @peer_id, namespaces)
+ assert_equal(1, command(:list_peers).length)
+ assert_equal(@peer_id, command(:list_peers).get(0).getPeerId)
+ peer_config = command(:list_peers).get(0).getPeerConfig
+ assert_equal(namespaces_str,
+ replication_admin.show_peer_exclude_namespaces(peer_config))
+
+ # append a namespace which is already excluded in the peer config
+ command(:append_peer_exclude_namespaces, @peer_id, namespaces)
+ assert_equal(1, command(:list_peers).length)
+ assert_equal(@peer_id, command(:list_peers).get(0).getPeerId)
+ peer_config = command(:list_peers).get(0).getPeerConfig
+ assert_equal(namespaces_str,
+ replication_admin.show_peer_exclude_namespaces(peer_config))
+
+ # cleanup for future tests
+ command(:remove_peer, @peer_id)
+ end
+
+ define_test "remove_peer_exclude_namespaces: works with namespaces array" do
+ cluster_key = "zk4,zk5,zk6:11000:/hbase-test"
+ args = {CLUSTER_KEY => cluster_key}
+ command(:add_peer, @peer_id, args)
+
+ namespaces = ["ns1", "ns2", "ns3"]
+ command(:set_peer_exclude_namespaces, @peer_id, namespaces)
+
+ namespaces = ["ns1", "ns2"]
+ namespaces_str = "!ns3"
+ command(:remove_peer_exclude_namespaces, @peer_id, namespaces)
+ assert_equal(1, command(:list_peers).length)
+ assert_equal(@peer_id, command(:list_peers).get(0).getPeerId)
+ peer_config = command(:list_peers).get(0).getPeerConfig
+ assert_equal(namespaces_str,
+ replication_admin.show_peer_exclude_namespaces(peer_config))
+
+ namespaces = ["ns3"]
+ namespaces_str = nil
+ command(:remove_peer_exclude_namespaces, @peer_id, namespaces)
+ assert_equal(1, command(:list_peers).length)
+ assert_equal(@peer_id, command(:list_peers).get(0).getPeerId)
+ peer_config = command(:list_peers).get(0).getPeerConfig
+ assert_equal(namespaces_str,
+ replication_admin.show_peer_exclude_namespaces(peer_config))
+
+ # remove a namespace which is not in peer config
+ command(:remove_peer_namespaces, @peer_id, namespaces)
+ assert_equal(1, command(:list_peers).length)
+ assert_equal(@peer_id, command(:list_peers).get(0).getPeerId)
+ peer_config = command(:list_peers).get(0).getPeerConfig
+ assert_equal(namespaces_str,
+ replication_admin.show_peer_exclude_namespaces(peer_config))
+
+ # cleanup for future tests
+ command(:remove_peer, @peer_id)
+ end
+
# assert_raise fails on native exceptions - https://jira.codehaus.org/browse/JRUBY-5279
# Can't catch native Java exception with assert_raise in JRuby 1.6.8 as in the test below.
# define_test "add_peer: adding a second peer with same id should error" do
[10/15] hbase git commit: HBASE-21670 Add 2.0.4 to download page
Posted by zh...@apache.org.
HBASE-21670 Add 2.0.4 to download page
Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/c3005653
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/c3005653
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/c3005653
Branch: refs/heads/HBASE-21512
Commit: c300565311edfa056ddd5abd5480bc7f93fe07a2
Parents: 466fa92
Author: stack <st...@apache.org>
Authored: Thu Jan 3 21:35:14 2019 -0800
Committer: stack <st...@apache.org>
Committed: Thu Jan 3 21:35:14 2019 -0800
----------------------------------------------------------------------
src/site/xdoc/downloads.xml | 13 ++++++-------
1 file changed, 6 insertions(+), 7 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/hbase/blob/c3005653/src/site/xdoc/downloads.xml
----------------------------------------------------------------------
diff --git a/src/site/xdoc/downloads.xml b/src/site/xdoc/downloads.xml
index 4bb7f51..2859779 100644
--- a/src/site/xdoc/downloads.xml
+++ b/src/site/xdoc/downloads.xml
@@ -32,7 +32,6 @@ under the License.
<a href="https://www.apache.org/dyn/closer.cgi#verify">Verify The Integrity Of The Files</a> for
how to verify your mirrored downloads.
</p>
- <p style="color:red;"><strong>NOTE: 2.1.1 and 2.0.3 have a serious memory leak. See HBASE-21551. We are working on replacement releases.</strong></p>
<section name="Releases">
<table>
<tr>
@@ -67,23 +66,23 @@ under the License.
</tr>
<tr>
<td style="test-align: left">
- 2.0.3
+ 2.0.4
</td>
<td style="test-align: left">
2018/12/02
</td>
<td style="test-align: left">
- <a href="https://apache.org/dist/hbase/2.0.3/compat-check-report-2.0.2-vs-2.0.3.html">2.0.2 vs 2.0.3</a>
+ <a href="https://apache.org/dist/hbase/2.0.4/compatibility_report_2.0.3vs2.0.4.html">2.0.3 vs 2.0.4</a>
</td>
<td style="test-align: left">
- <a href="https://apache.org/dist/hbase/2.0.3/CHANGES.md">Changes</a>
+ <a href="https://apache.org/dist/hbase/2.0.4/CHANGES.md">Changes</a>
</td>
<td style="test-align: left">
- <a href="https://apache.org/dist/hbase/2.0.3/RELEASENOTES.md">Release Notes</a>
+ <a href="https://apache.org/dist/hbase/2.0.4/RELEASENOTES.md">Release Notes</a>
</td>
<td style="test-align: left">
- <a href="https://apache.org/dist/hbase/2.0.3/hbase-2.0.3-src.tar.gz">src</a> (<a href="https://apache.org/dist/hbase/2.0.3/hbase-2.0.3-src.tar.gz.sha512">sha512</a> <a href="https://apache.org/dist/hbase/2.0.3/hbase-2.0.3-src.tar.gz.asc">asc</a>) <br />
- <a href="https://apache.org/dist/hbase/2.0.3/hbase-2.0.3-bin.tar.gz">bin</a> (<a href="https://apache.org/dist/hbase/2.0.3/hbase-2.0.3-src.tar.gz.sha512">sha512</a> <a href="https://apache.org/dist/hbase/2.0.3/hbase-2.0.3-src.tar.gz.asc">asc</a>)
+ <a href="https://apache.org/dist/hbase/2.0.4/hbase-2.0.4-src.tar.gz">src</a> (<a href="https://apache.org/dist/hbase/2.0.4/hbase-2.0.4-src.tar.gz.sha512">sha512</a> <a href="https://apache.org/dist/hbase/2.0.4/hbase-2.0.4-src.tar.gz.asc">asc</a>) <br />
+ <a href="https://apache.org/dist/hbase/2.0.4/hbase-2.0.4-bin.tar.gz">bin</a> (<a href="https://apache.org/dist/hbase/2.0.4/hbase-2.0.4-src.tar.gz.sha512">sha512</a> <a href="https://apache.org/dist/hbase/2.0.4/hbase-2.0.4-src.tar.gz.asc">asc</a>)
</td>
</tr>
<tr>
[07/15] hbase git commit: HBASE-17356 Add replica get support
Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/db66e6cc/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 4f73909..869a630 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
@@ -18,6 +18,7 @@
package org.apache.hadoop.hbase.client;
import static org.apache.hadoop.hbase.TableName.META_TABLE_NAME;
+import static org.apache.hadoop.hbase.util.FutureUtils.addListener;
import com.google.protobuf.Message;
import com.google.protobuf.RpcChannel;
@@ -491,23 +492,23 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
@Override
public CompletableFuture<TableDescriptor> getDescriptor(TableName tableName) {
CompletableFuture<TableDescriptor> future = new CompletableFuture<>();
- this.<List<TableSchema>> newMasterCaller()
- .action(
- (controller, stub) -> this
- .<GetTableDescriptorsRequest, GetTableDescriptorsResponse, List<TableSchema>> call(
- controller, stub, RequestConverter.buildGetTableDescriptorsRequest(tableName), (s,
- c, req, done) -> s.getTableDescriptors(c, req, done), (resp) -> resp
- .getTableSchemaList())).call().whenComplete((tableSchemas, error) -> {
- if (error != null) {
- future.completeExceptionally(error);
- return;
- }
- if (!tableSchemas.isEmpty()) {
- future.complete(ProtobufUtil.toTableDescriptor(tableSchemas.get(0)));
- } else {
- future.completeExceptionally(new TableNotFoundException(tableName.getNameAsString()));
- }
- });
+ addListener(this.<List<TableSchema>> newMasterCaller()
+ .action((controller, stub) -> this
+ .<GetTableDescriptorsRequest, GetTableDescriptorsResponse, List<TableSchema>> call(
+ controller, stub, RequestConverter.buildGetTableDescriptorsRequest(tableName),
+ (s, c, req, done) -> s.getTableDescriptors(c, req, done),
+ (resp) -> resp.getTableSchemaList()))
+ .call(), (tableSchemas, error) -> {
+ if (error != null) {
+ future.completeExceptionally(error);
+ return;
+ }
+ if (!tableSchemas.isEmpty()) {
+ future.complete(ProtobufUtil.toTableDescriptor(tableSchemas.get(0)));
+ } else {
+ future.completeExceptionally(new TableNotFoundException(tableName.getNameAsString()));
+ }
+ });
return future;
}
@@ -590,7 +591,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
@Override
public CompletableFuture<Boolean> isTableEnabled(TableName tableName) {
CompletableFuture<Boolean> future = new CompletableFuture<>();
- AsyncMetaTableAccessor.getTableState(metaTable, tableName).whenComplete((state, error) -> {
+ addListener(AsyncMetaTableAccessor.getTableState(metaTable, tableName), (state, error) -> {
if (error != null) {
future.completeExceptionally(error);
return;
@@ -607,7 +608,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
@Override
public CompletableFuture<Boolean> isTableDisabled(TableName tableName) {
CompletableFuture<Boolean> future = new CompletableFuture<>();
- AsyncMetaTableAccessor.getTableState(metaTable, tableName).whenComplete((state, error) -> {
+ addListener(AsyncMetaTableAccessor.getTableState(metaTable, tableName), (state, error) -> {
if (error != null) {
future.completeExceptionally(error);
return;
@@ -636,40 +637,37 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
private CompletableFuture<Boolean> isTableAvailable(TableName tableName,
Optional<byte[][]> splitKeys) {
CompletableFuture<Boolean> future = new CompletableFuture<>();
- isTableEnabled(tableName).whenComplete(
- (enabled, error) -> {
- if (error != null) {
- future.completeExceptionally(error);
- return;
- }
- if (!enabled) {
- future.complete(false);
- } else {
- AsyncMetaTableAccessor.getTableHRegionLocations(metaTable, Optional.of(tableName))
- .whenComplete(
- (locations, error1) -> {
- if (error1 != null) {
- future.completeExceptionally(error1);
- return;
- }
- List<HRegionLocation> notDeployedRegions =
- locations.stream().filter(loc -> loc.getServerName() == null)
- .collect(Collectors.toList());
- if (notDeployedRegions.size() > 0) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("Table " + tableName + " has " + notDeployedRegions.size()
- + " regions");
- }
- future.complete(false);
- return;
- }
+ addListener(isTableEnabled(tableName), (enabled, error) -> {
+ if (error != null) {
+ future.completeExceptionally(error);
+ return;
+ }
+ if (!enabled) {
+ future.complete(false);
+ } else {
+ addListener(
+ AsyncMetaTableAccessor.getTableHRegionLocations(metaTable, Optional.of(tableName)),
+ (locations, error1) -> {
+ if (error1 != null) {
+ future.completeExceptionally(error1);
+ return;
+ }
+ List<HRegionLocation> notDeployedRegions = locations.stream()
+ .filter(loc -> loc.getServerName() == null).collect(Collectors.toList());
+ if (notDeployedRegions.size() > 0) {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Table " + tableName + " has " + notDeployedRegions.size() + " regions");
+ }
+ future.complete(false);
+ return;
+ }
- Optional<Boolean> available =
- splitKeys.map(keys -> compareRegionsWithSplitKeys(locations, keys));
- future.complete(available.orElse(true));
- });
- }
- });
+ Optional<Boolean> available =
+ splitKeys.map(keys -> compareRegionsWithSplitKeys(locations, keys));
+ future.complete(available.orElse(true));
+ });
+ }
+ });
return future;
}
@@ -791,20 +789,20 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
@Override
public CompletableFuture<Void> flush(TableName tableName) {
CompletableFuture<Void> future = new CompletableFuture<>();
- tableExists(tableName).whenComplete((exists, err) -> {
+ addListener(tableExists(tableName), (exists, err) -> {
if (err != null) {
future.completeExceptionally(err);
} else if (!exists) {
future.completeExceptionally(new TableNotFoundException(tableName));
} else {
- isTableEnabled(tableName).whenComplete((tableEnabled, err2) -> {
+ addListener(isTableEnabled(tableName), (tableEnabled, err2) -> {
if (err2 != null) {
future.completeExceptionally(err2);
} else if (!tableEnabled) {
future.completeExceptionally(new TableNotEnabledException(tableName));
} else {
- execProcedure(FLUSH_TABLE_PROCEDURE_SIGNATURE, tableName.getNameAsString(),
- new HashMap<>()).whenComplete((ret, err3) -> {
+ addListener(execProcedure(FLUSH_TABLE_PROCEDURE_SIGNATURE, tableName.getNameAsString(),
+ new HashMap<>()), (ret, err3) -> {
if (err3 != null) {
future.completeExceptionally(err3);
} else {
@@ -821,27 +819,25 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
@Override
public CompletableFuture<Void> flushRegion(byte[] regionName) {
CompletableFuture<Void> future = new CompletableFuture<>();
- getRegionLocation(regionName).whenComplete(
- (location, err) -> {
- if (err != null) {
- future.completeExceptionally(err);
- return;
- }
- ServerName serverName = location.getServerName();
- if (serverName == null) {
- future.completeExceptionally(new NoServerForRegionException(Bytes
- .toStringBinary(regionName)));
- return;
+ addListener(getRegionLocation(regionName), (location, err) -> {
+ if (err != null) {
+ future.completeExceptionally(err);
+ return;
+ }
+ ServerName serverName = location.getServerName();
+ if (serverName == null) {
+ future
+ .completeExceptionally(new NoServerForRegionException(Bytes.toStringBinary(regionName)));
+ return;
+ }
+ addListener(flush(serverName, location.getRegion()), (ret, err2) -> {
+ if (err2 != null) {
+ future.completeExceptionally(err2);
+ } else {
+ future.complete(ret);
}
- flush(serverName, location.getRegion())
- .whenComplete((ret, err2) -> {
- if (err2 != null) {
- future.completeExceptionally(err2);
- } else {
- future.complete(ret);
- }
- });
});
+ });
return future;
}
@@ -859,7 +855,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
@Override
public CompletableFuture<Void> flushRegionServer(ServerName sn) {
CompletableFuture<Void> future = new CompletableFuture<>();
- getRegions(sn).whenComplete((hRegionInfos, err) -> {
+ addListener(getRegions(sn), (hRegionInfos, err) -> {
if (err != null) {
future.completeExceptionally(err);
return;
@@ -868,9 +864,8 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
if (hRegionInfos != null) {
hRegionInfos.forEach(region -> compactFutures.add(flush(sn, region)));
}
- CompletableFuture
- .allOf(compactFutures.toArray(new CompletableFuture<?>[compactFutures.size()]))
- .whenComplete((ret, err2) -> {
+ addListener(CompletableFuture.allOf(
+ compactFutures.toArray(new CompletableFuture<?>[compactFutures.size()])), (ret, err2) -> {
if (err2 != null) {
future.completeExceptionally(err2);
} else {
@@ -943,7 +938,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
private CompletableFuture<Void> compactRegionServer(ServerName sn, boolean major) {
CompletableFuture<Void> future = new CompletableFuture<>();
- getRegions(sn).whenComplete((hRegionInfos, err) -> {
+ addListener(getRegions(sn), (hRegionInfos, err) -> {
if (err != null) {
future.completeExceptionally(err);
return;
@@ -952,15 +947,14 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
if (hRegionInfos != null) {
hRegionInfos.forEach(region -> compactFutures.add(compact(sn, region, major, null)));
}
- CompletableFuture
- .allOf(compactFutures.toArray(new CompletableFuture<?>[compactFutures.size()]))
- .whenComplete((ret, err2) -> {
- if (err2 != null) {
- future.completeExceptionally(err2);
- } else {
- future.complete(ret);
- }
- });
+ addListener(CompletableFuture.allOf(
+ compactFutures.toArray(new CompletableFuture<?>[compactFutures.size()])), (ret, err2) -> {
+ if (err2 != null) {
+ future.completeExceptionally(err2);
+ } else {
+ future.complete(ret);
+ }
+ });
});
return future;
}
@@ -968,28 +962,26 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
private CompletableFuture<Void> compactRegion(byte[] regionName, byte[] columnFamily,
boolean major) {
CompletableFuture<Void> future = new CompletableFuture<>();
-
- getRegionLocation(regionName).whenComplete(
- (location, err) -> {
- if (err != null) {
- future.completeExceptionally(err);
- return;
- }
- ServerName serverName = location.getServerName();
- if (serverName == null) {
- future.completeExceptionally(new NoServerForRegionException(Bytes
- .toStringBinary(regionName)));
- return;
- }
- compact(location.getServerName(), location.getRegion(), major, columnFamily)
- .whenComplete((ret, err2) -> {
- if (err2 != null) {
- future.completeExceptionally(err2);
- } else {
- future.complete(ret);
- }
- });
- });
+ addListener(getRegionLocation(regionName), (location, err) -> {
+ if (err != null) {
+ future.completeExceptionally(err);
+ return;
+ }
+ ServerName serverName = location.getServerName();
+ if (serverName == null) {
+ future
+ .completeExceptionally(new NoServerForRegionException(Bytes.toStringBinary(regionName)));
+ return;
+ }
+ addListener(compact(location.getServerName(), location.getRegion(), major, columnFamily),
+ (ret, err2) -> {
+ if (err2 != null) {
+ future.completeExceptionally(err2);
+ } else {
+ future.complete(ret);
+ }
+ });
+ });
return future;
}
@@ -1001,19 +993,18 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
CompletableFuture<List<HRegionLocation>> future = new CompletableFuture<>();
// For meta table, we use zk to fetch all locations.
AsyncRegistry registry = AsyncRegistryFactory.getRegistry(connection.getConfiguration());
- registry.getMetaRegionLocation().whenComplete(
- (metaRegions, err) -> {
- if (err != null) {
- future.completeExceptionally(err);
- } else if (metaRegions == null || metaRegions.isEmpty()
- || metaRegions.getDefaultRegionLocation() == null) {
- future.completeExceptionally(new IOException("meta region does not found"));
- } else {
- future.complete(Collections.singletonList(metaRegions.getDefaultRegionLocation()));
- }
- // close the registry.
- IOUtils.closeQuietly(registry);
- });
+ addListener(registry.getMetaRegionLocation(), (metaRegions, err) -> {
+ if (err != null) {
+ future.completeExceptionally(err);
+ } else if (metaRegions == null || metaRegions.isEmpty() ||
+ metaRegions.getDefaultRegionLocation() == null) {
+ future.completeExceptionally(new IOException("meta region does not found"));
+ } else {
+ future.complete(Collections.singletonList(metaRegions.getDefaultRegionLocation()));
+ }
+ // close the registry.
+ IOUtils.closeQuietly(registry);
+ });
return future;
} else {
// For non-meta table, we fetch all locations by scanning hbase:meta table
@@ -1024,40 +1015,40 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
/**
* Compact column family of a table, Asynchronous operation even if CompletableFuture.get()
*/
- private CompletableFuture<Void> compact(TableName tableName, byte[] columnFamily,
- boolean major, CompactType compactType) {
+ private CompletableFuture<Void> compact(TableName tableName, byte[] columnFamily, boolean major,
+ CompactType compactType) {
CompletableFuture<Void> future = new CompletableFuture<>();
switch (compactType) {
case MOB:
- connection.registry.getMasterAddress().whenComplete((serverName, err) -> {
+ addListener(connection.registry.getMasterAddress(), (serverName, err) -> {
if (err != null) {
future.completeExceptionally(err);
return;
}
RegionInfo regionInfo = RegionInfo.createMobRegionInfo(tableName);
- compact(serverName, regionInfo, major, columnFamily)
- .whenComplete((ret, err2) -> {
- if (err2 != null) {
- future.completeExceptionally(err2);
- } else {
- future.complete(ret);
- }
- });
+ addListener(compact(serverName, regionInfo, major, columnFamily), (ret, err2) -> {
+ if (err2 != null) {
+ future.completeExceptionally(err2);
+ } else {
+ future.complete(ret);
+ }
+ });
});
break;
case NORMAL:
- getTableHRegionLocations(tableName).whenComplete((locations, err) -> {
+ addListener(getTableHRegionLocations(tableName), (locations, err) -> {
if (err != null) {
future.completeExceptionally(err);
return;
}
- CompletableFuture<?>[] compactFutures = locations.stream().filter(l -> l.getRegion() != null)
+ CompletableFuture<?>[] compactFutures =
+ locations.stream().filter(l -> l.getRegion() != null)
.filter(l -> !l.getRegion().isOffline()).filter(l -> l.getServerName() != null)
.map(l -> compact(l.getServerName(), l.getRegion(), major, columnFamily))
.toArray(CompletableFuture<?>[]::new);
// future complete unless all of the compact futures are completed.
- CompletableFuture.allOf(compactFutures).whenComplete((ret, err2) -> {
+ addListener(CompletableFuture.allOf(compactFutures), (ret, err2) -> {
if (err2 != null) {
future.completeExceptionally(err2);
} else {
@@ -1098,29 +1089,28 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
private void checkAndGetTableName(byte[] encodeRegionName, AtomicReference<TableName> tableName,
CompletableFuture<TableName> result) {
- getRegionLocation(encodeRegionName).whenComplete(
- (location, err) -> {
- if (err != null) {
- result.completeExceptionally(err);
- return;
- }
- RegionInfo regionInfo = location.getRegion();
- if (regionInfo.getReplicaId() != RegionInfo.DEFAULT_REPLICA_ID) {
- result.completeExceptionally(new IllegalArgumentException(
- "Can't invoke merge on non-default regions directly"));
- return;
- }
- if (!tableName.compareAndSet(null, regionInfo.getTable())) {
- if (!tableName.get().equals(regionInfo.getTable())) {
- // tables of this two region should be same.
- result.completeExceptionally(new IllegalArgumentException(
- "Cannot merge regions from two different tables " + tableName.get() + " and "
- + regionInfo.getTable()));
- } else {
- result.complete(tableName.get());
- }
+ addListener(getRegionLocation(encodeRegionName), (location, err) -> {
+ if (err != null) {
+ result.completeExceptionally(err);
+ return;
+ }
+ RegionInfo regionInfo = location.getRegion();
+ if (regionInfo.getReplicaId() != RegionInfo.DEFAULT_REPLICA_ID) {
+ result.completeExceptionally(
+ new IllegalArgumentException("Can't invoke merge on non-default regions directly"));
+ return;
+ }
+ if (!tableName.compareAndSet(null, regionInfo.getTable())) {
+ if (!tableName.get().equals(regionInfo.getTable())) {
+ // tables of this two region should be same.
+ result.completeExceptionally(
+ new IllegalArgumentException("Cannot merge regions from two different tables " +
+ tableName.get() + " and " + regionInfo.getTable()));
+ } else {
+ result.complete(tableName.get());
}
- });
+ }
+ });
}
private CompletableFuture<TableName> checkRegionsAndGetTableName(byte[] encodeRegionNameA,
@@ -1185,41 +1175,42 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
final byte[] encodeRegionNameA = toEncodeRegionName(nameOfRegionA);
final byte[] encodeRegionNameB = toEncodeRegionName(nameOfRegionB);
- checkRegionsAndGetTableName(encodeRegionNameA, encodeRegionNameB)
- .whenComplete((tableName, err) -> {
- if (err != null) {
- future.completeExceptionally(err);
- return;
- }
+ addListener(checkRegionsAndGetTableName(encodeRegionNameA, encodeRegionNameB),
+ (tableName, err) -> {
+ if (err != null) {
+ future.completeExceptionally(err);
+ return;
+ }
- MergeTableRegionsRequest request = null;
- try {
- request = RequestConverter.buildMergeTableRegionsRequest(
- new byte[][] { encodeRegionNameA, encodeRegionNameB }, forcible, ng.getNonceGroup(),
- ng.newNonce());
- } catch (DeserializationException e) {
- future.completeExceptionally(e);
- return;
- }
+ MergeTableRegionsRequest request = null;
+ try {
+ request = RequestConverter.buildMergeTableRegionsRequest(
+ new byte[][] { encodeRegionNameA, encodeRegionNameB }, forcible, ng.getNonceGroup(),
+ ng.newNonce());
+ } catch (DeserializationException e) {
+ future.completeExceptionally(e);
+ return;
+ }
+ addListener(
this.<MergeTableRegionsRequest, MergeTableRegionsResponse> procedureCall(request,
(s, c, req, done) -> s.mergeTableRegions(c, req, done), (resp) -> resp.getProcId(),
- new MergeTableRegionProcedureBiConsumer(tableName)).whenComplete((ret, err2) -> {
- if (err2 != null) {
- future.completeExceptionally(err2);
- } else {
- future.complete(ret);
- }
- });
-
- });
+ new MergeTableRegionProcedureBiConsumer(tableName)),
+ (ret, err2) -> {
+ if (err2 != null) {
+ future.completeExceptionally(err2);
+ } else {
+ future.complete(ret);
+ }
+ });
+ });
return future;
}
@Override
public CompletableFuture<Void> split(TableName tableName) {
CompletableFuture<Void> future = new CompletableFuture<>();
- tableExists(tableName).whenComplete((exist, error) -> {
+ addListener(tableExists(tableName), (exist, error) -> {
if (error != null) {
future.completeExceptionally(error);
return;
@@ -1228,45 +1219,50 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
future.completeExceptionally(new TableNotFoundException(tableName));
return;
}
- metaTable
+ addListener(
+ metaTable
.scanAll(new Scan().setReadType(ReadType.PREAD).addFamily(HConstants.CATALOG_FAMILY)
- .withStartRow(MetaTableAccessor.getTableStartRowForMeta(tableName, QueryType.REGION))
- .withStopRow(MetaTableAccessor.getTableStopRowForMeta(tableName, QueryType.REGION)))
- .whenComplete((results, err2) -> {
- if (err2 != null) {
- future.completeExceptionally(err2);
- return;
- }
- if (results != null && !results.isEmpty()) {
- List<CompletableFuture<Void>> splitFutures = new ArrayList<>();
- for (Result r : results) {
- 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) {
- RegionInfo hri = h.getRegion();
- if (hri == null || hri.isSplitParent()
- || hri.getReplicaId() != RegionInfo.DEFAULT_REPLICA_ID)
- continue;
- splitFutures.add(split(hri, null));
+ .withStartRow(MetaTableAccessor.getTableStartRowForMeta(tableName, QueryType.REGION))
+ .withStopRow(MetaTableAccessor.getTableStopRowForMeta(tableName, QueryType.REGION))),
+ (results, err2) -> {
+ if (err2 != null) {
+ future.completeExceptionally(err2);
+ return;
+ }
+ if (results != null && !results.isEmpty()) {
+ List<CompletableFuture<Void>> splitFutures = new ArrayList<>();
+ for (Result r : results) {
+ 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) {
+ RegionInfo hri = h.getRegion();
+ if (hri == null || hri.isSplitParent() ||
+ hri.getReplicaId() != RegionInfo.DEFAULT_REPLICA_ID) {
+ continue;
}
+ splitFutures.add(split(hri, null));
}
}
}
- CompletableFuture
- .allOf(splitFutures.toArray(new CompletableFuture<?>[splitFutures.size()]))
- .whenComplete((ret, exception) -> {
- if (exception != null) {
- future.completeExceptionally(exception);
- return;
- }
- future.complete(ret);
- });
- } else {
- future.complete(null);
}
- });
+ addListener(
+ CompletableFuture
+ .allOf(splitFutures.toArray(new CompletableFuture<?>[splitFutures.size()])),
+ (ret, exception) -> {
+ if (exception != null) {
+ future.completeExceptionally(exception);
+ return;
+ }
+ future.complete(ret);
+ });
+ } else {
+ future.complete(null);
+ }
+ });
});
return future;
}
@@ -1277,54 +1273,52 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
if (splitPoint == null) {
return failedFuture(new IllegalArgumentException("splitPoint can not be null."));
}
- connection.getRegionLocator(tableName).getRegionLocation(splitPoint)
- .whenComplete((loc, err) -> {
- if (err != null) {
- result.completeExceptionally(err);
- } else if (loc == null || loc.getRegion() == null) {
- result.completeExceptionally(new IllegalArgumentException(
- "Region does not found: rowKey=" + Bytes.toStringBinary(splitPoint)));
- } else {
- splitRegion(loc.getRegion().getRegionName(), splitPoint)
- .whenComplete((ret, err2) -> {
- if (err2 != null) {
- result.completeExceptionally(err2);
- } else {
- result.complete(ret);
- }
+ addListener(connection.getRegionLocator(tableName).getRegionLocation(splitPoint),
+ (loc, err) -> {
+ if (err != null) {
+ result.completeExceptionally(err);
+ } else if (loc == null || loc.getRegion() == null) {
+ result.completeExceptionally(new IllegalArgumentException(
+ "Region does not found: rowKey=" + Bytes.toStringBinary(splitPoint)));
+ } else {
+ addListener(splitRegion(loc.getRegion().getRegionName(), splitPoint), (ret, err2) -> {
+ if (err2 != null) {
+ result.completeExceptionally(err2);
+ } else {
+ result.complete(ret);
+ }
- });
- }
- });
+ });
+ }
+ });
return result;
}
@Override
public CompletableFuture<Void> splitRegion(byte[] regionName) {
CompletableFuture<Void> future = new CompletableFuture<>();
- getRegionLocation(regionName).whenComplete(
- (location, err) -> {
- RegionInfo regionInfo = location.getRegion();
- 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."));
- return;
- }
- ServerName serverName = location.getServerName();
- if (serverName == null) {
- future.completeExceptionally(new NoServerForRegionException(Bytes
- .toStringBinary(regionName)));
- return;
+ addListener(getRegionLocation(regionName), (location, err) -> {
+ RegionInfo regionInfo = location.getRegion();
+ 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."));
+ return;
+ }
+ ServerName serverName = location.getServerName();
+ if (serverName == null) {
+ future
+ .completeExceptionally(new NoServerForRegionException(Bytes.toStringBinary(regionName)));
+ return;
+ }
+ addListener(split(regionInfo, null), (ret, err2) -> {
+ if (err2 != null) {
+ future.completeExceptionally(err2);
+ } else {
+ future.complete(ret);
}
- split(regionInfo, null).whenComplete((ret, err2) -> {
- if (err2 != null) {
- future.completeExceptionally(err2);
- } else {
- future.complete(ret);
- }
- });
});
+ });
return future;
}
@@ -1333,35 +1327,34 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
Preconditions.checkNotNull(splitPoint,
"splitPoint is null. If you don't specify a splitPoint, use splitRegion(byte[]) instead");
CompletableFuture<Void> future = new CompletableFuture<>();
- getRegionLocation(regionName).whenComplete(
- (location, err) -> {
- RegionInfo regionInfo = location.getRegion();
- 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."));
- return;
- }
- ServerName serverName = location.getServerName();
- if (serverName == null) {
- future.completeExceptionally(new NoServerForRegionException(Bytes
- .toStringBinary(regionName)));
- return;
- }
- if (regionInfo.getStartKey() != null
- && Bytes.compareTo(regionInfo.getStartKey(), splitPoint) == 0) {
- future.completeExceptionally(new IllegalArgumentException(
- "should not give a splitkey which equals to startkey!"));
- return;
+ addListener(getRegionLocation(regionName), (location, err) -> {
+ RegionInfo regionInfo = location.getRegion();
+ 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."));
+ return;
+ }
+ ServerName serverName = location.getServerName();
+ if (serverName == null) {
+ future
+ .completeExceptionally(new NoServerForRegionException(Bytes.toStringBinary(regionName)));
+ return;
+ }
+ if (regionInfo.getStartKey() != null &&
+ Bytes.compareTo(regionInfo.getStartKey(), splitPoint) == 0) {
+ future.completeExceptionally(
+ new IllegalArgumentException("should not give a splitkey which equals to startkey!"));
+ return;
+ }
+ addListener(split(regionInfo, splitPoint), (ret, err2) -> {
+ if (err2 != null) {
+ future.completeExceptionally(err2);
+ } else {
+ future.complete(ret);
}
- split(regionInfo, splitPoint).whenComplete((ret, err2) -> {
- if (err2 != null) {
- future.completeExceptionally(err2);
- } else {
- future.complete(ret);
- }
- });
});
+ });
return future;
}
@@ -1370,121 +1363,119 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
TableName tableName = hri.getTable();
SplitTableRegionRequest request = null;
try {
- request = RequestConverter
- .buildSplitTableRegionRequest(hri, splitPoint,
- ng.getNonceGroup(), ng.newNonce());
+ request = RequestConverter.buildSplitTableRegionRequest(hri, splitPoint, ng.getNonceGroup(),
+ ng.newNonce());
} catch (DeserializationException e) {
future.completeExceptionally(e);
return future;
}
- this.<SplitTableRegionRequest, SplitTableRegionResponse>procedureCall(request,
- (s, c, req, done) -> s.splitRegion(c, req, done), (resp) -> resp.getProcId(),
- new SplitTableRegionProcedureBiConsumer(tableName)).whenComplete((ret, err2) -> {
- if (err2 != null) {
- future.completeExceptionally(err2);
- } else {
- future.complete(ret);
- }
- });
+ addListener(this.<SplitTableRegionRequest, SplitTableRegionResponse> procedureCall(request,
+ (s, c, req, done) -> s.splitRegion(c, req, done), (resp) -> resp.getProcId(),
+ new SplitTableRegionProcedureBiConsumer(tableName)), (ret, err2) -> {
+ if (err2 != null) {
+ future.completeExceptionally(err2);
+ } else {
+ future.complete(ret);
+ }
+ });
return future;
}
@Override
public CompletableFuture<Void> assign(byte[] regionName) {
CompletableFuture<Void> future = new CompletableFuture<>();
- getRegionInfo(regionName).whenComplete(
- (regionInfo, err) -> {
- if (err != null) {
- future.completeExceptionally(err);
- return;
- }
- this.<Void> newMasterCaller()
- .action(
- ((controller, stub) -> this.<AssignRegionRequest, AssignRegionResponse, Void> call(
- controller, stub, RequestConverter.buildAssignRegionRequest(regionInfo
- .getRegionName()), (s, c, req, done) -> s.assignRegion(c, req, done),
- resp -> null))).call().whenComplete((ret, err2) -> {
- if (err2 != null) {
- future.completeExceptionally(err2);
- } else {
- future.complete(ret);
- }
- });
- });
+ addListener(getRegionInfo(regionName), (regionInfo, err) -> {
+ if (err != null) {
+ future.completeExceptionally(err);
+ return;
+ }
+ addListener(this.<Void> newMasterCaller()
+ .action(((controller, stub) -> this.<AssignRegionRequest, AssignRegionResponse, Void> call(
+ controller, stub, RequestConverter.buildAssignRegionRequest(regionInfo.getRegionName()),
+ (s, c, req, done) -> s.assignRegion(c, req, done), resp -> null)))
+ .call(), (ret, err2) -> {
+ if (err2 != null) {
+ future.completeExceptionally(err2);
+ } else {
+ future.complete(ret);
+ }
+ });
+ });
return future;
}
@Override
public CompletableFuture<Void> unassign(byte[] regionName, boolean forcible) {
CompletableFuture<Void> future = new CompletableFuture<>();
- getRegionInfo(regionName).whenComplete(
- (regionInfo, err) -> {
- if (err != null) {
- future.completeExceptionally(err);
- return;
- }
+ addListener(getRegionInfo(regionName), (regionInfo, err) -> {
+ if (err != null) {
+ future.completeExceptionally(err);
+ return;
+ }
+ addListener(
this.<Void> newMasterCaller()
- .action(
- ((controller, stub) -> this
- .<UnassignRegionRequest, UnassignRegionResponse, Void> call(controller, stub,
- RequestConverter.buildUnassignRegionRequest(regionInfo.getRegionName(), forcible),
- (s, c, req, done) -> s.unassignRegion(c, req, done), resp -> null))).call()
- .whenComplete((ret, err2) -> {
- if (err2 != null) {
- future.completeExceptionally(err2);
- } else {
- future.complete(ret);
- }
- });
- });
+ .action(((controller, stub) -> this
+ .<UnassignRegionRequest, UnassignRegionResponse, Void> call(controller, stub,
+ RequestConverter.buildUnassignRegionRequest(regionInfo.getRegionName(), forcible),
+ (s, c, req, done) -> s.unassignRegion(c, req, done), resp -> null)))
+ .call(),
+ (ret, err2) -> {
+ if (err2 != null) {
+ future.completeExceptionally(err2);
+ } else {
+ future.complete(ret);
+ }
+ });
+ });
return future;
}
@Override
public CompletableFuture<Void> offline(byte[] regionName) {
CompletableFuture<Void> future = new CompletableFuture<>();
- getRegionInfo(regionName).whenComplete(
- (regionInfo, err) -> {
- if (err != null) {
- future.completeExceptionally(err);
- return;
- }
+ addListener(getRegionInfo(regionName), (regionInfo, err) -> {
+ if (err != null) {
+ future.completeExceptionally(err);
+ return;
+ }
+ addListener(
this.<Void> newMasterCaller()
- .action(
- ((controller, stub) -> this.<OfflineRegionRequest, OfflineRegionResponse, Void> call(
- controller, stub, RequestConverter.buildOfflineRegionRequest(regionInfo
- .getRegionName()), (s, c, req, done) -> s.offlineRegion(c, req, done),
- resp -> null))).call().whenComplete((ret, err2) -> {
- if (err2 != null) {
- future.completeExceptionally(err2);
- } else {
- future.complete(ret);
- }
- });
- });
+ .action(((controller, stub) -> this
+ .<OfflineRegionRequest, OfflineRegionResponse, Void> call(controller, stub,
+ RequestConverter.buildOfflineRegionRequest(regionInfo.getRegionName()),
+ (s, c, req, done) -> s.offlineRegion(c, req, done), resp -> null)))
+ .call(),
+ (ret, err2) -> {
+ if (err2 != null) {
+ future.completeExceptionally(err2);
+ } else {
+ future.complete(ret);
+ }
+ });
+ });
return future;
}
@Override
public CompletableFuture<Void> move(byte[] regionName) {
CompletableFuture<Void> future = new CompletableFuture<>();
- getRegionInfo(regionName).whenComplete(
- (regionInfo, err) -> {
- if (err != null) {
- future.completeExceptionally(err);
- return;
- }
+ addListener(getRegionInfo(regionName), (regionInfo, err) -> {
+ if (err != null) {
+ future.completeExceptionally(err);
+ return;
+ }
+ addListener(
moveRegion(
- RequestConverter.buildMoveRegionRequest(regionInfo.getEncodedNameAsBytes(), null))
- .whenComplete((ret, err2) -> {
- if (err2 != null) {
- future.completeExceptionally(err2);
- } else {
- future.complete(ret);
- }
- });
- });
+ RequestConverter.buildMoveRegionRequest(regionInfo.getEncodedNameAsBytes(), null)),
+ (ret, err2) -> {
+ if (err2 != null) {
+ future.completeExceptionally(err2);
+ } else {
+ future.complete(ret);
+ }
+ });
+ });
return future;
}
@@ -1493,20 +1484,20 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
Preconditions.checkNotNull(destServerName,
"destServerName is null. If you don't specify a destServerName, use move(byte[]) instead");
CompletableFuture<Void> future = new CompletableFuture<>();
- getRegionInfo(regionName).whenComplete((regionInfo, err) -> {
+ addListener(getRegionInfo(regionName), (regionInfo, err) -> {
if (err != null) {
future.completeExceptionally(err);
return;
}
- moveRegion(
- RequestConverter.buildMoveRegionRequest(regionInfo.getEncodedNameAsBytes(), destServerName))
- .whenComplete((ret, err2) -> {
- if (err2 != null) {
- future.completeExceptionally(err2);
- } else {
- future.complete(ret);
- }
- });
+ addListener(moveRegion(RequestConverter
+ .buildMoveRegionRequest(regionInfo.getEncodedNameAsBytes(), destServerName)),
+ (ret, err2) -> {
+ if (err2 != null) {
+ future.completeExceptionally(err2);
+ } else {
+ future.complete(ret);
+ }
+ });
});
return future;
}
@@ -1636,11 +1627,11 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
}
CompletableFuture<Void> future = new CompletableFuture<Void>();
- getReplicationPeerConfig(id).whenComplete((peerConfig, error) -> {
+ addListener(getReplicationPeerConfig(id), (peerConfig, error) -> {
if (!completeExceptionally(future, error)) {
ReplicationPeerConfig newPeerConfig =
- ReplicationPeerConfigUtil.appendTableCFsToReplicationPeerConfig(tableCfs, peerConfig);
- updateReplicationPeerConfig(id, newPeerConfig).whenComplete((result, err) -> {
+ ReplicationPeerConfigUtil.appendTableCFsToReplicationPeerConfig(tableCfs, peerConfig);
+ addListener(updateReplicationPeerConfig(id, newPeerConfig), (result, err) -> {
if (!completeExceptionally(future, error)) {
future.complete(result);
}
@@ -1658,24 +1649,23 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
}
CompletableFuture<Void> future = new CompletableFuture<Void>();
- getReplicationPeerConfig(id).whenComplete(
- (peerConfig, error) -> {
- if (!completeExceptionally(future, error)) {
- ReplicationPeerConfig newPeerConfig = null;
- try {
- newPeerConfig = ReplicationPeerConfigUtil
- .removeTableCFsFromReplicationPeerConfig(tableCfs, peerConfig, id);
- } catch (ReplicationException e) {
- future.completeExceptionally(e);
- return;
- }
- updateReplicationPeerConfig(id, newPeerConfig).whenComplete((result, err) -> {
- if (!completeExceptionally(future, error)) {
- future.complete(result);
- }
- });
+ addListener(getReplicationPeerConfig(id), (peerConfig, error) -> {
+ if (!completeExceptionally(future, error)) {
+ ReplicationPeerConfig newPeerConfig = null;
+ try {
+ newPeerConfig = ReplicationPeerConfigUtil
+ .removeTableCFsFromReplicationPeerConfig(tableCfs, peerConfig, id);
+ } catch (ReplicationException e) {
+ future.completeExceptionally(e);
+ return;
}
- });
+ addListener(updateReplicationPeerConfig(id, newPeerConfig), (result, err) -> {
+ if (!completeExceptionally(future, error)) {
+ future.complete(result);
+ }
+ });
+ }
+ });
return future;
}
@@ -1710,31 +1700,30 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
@Override
public CompletableFuture<List<TableCFs>> listReplicatedTableCFs() {
CompletableFuture<List<TableCFs>> future = new CompletableFuture<List<TableCFs>>();
- listTableDescriptors().whenComplete(
- (tables, error) -> {
- if (!completeExceptionally(future, error)) {
- List<TableCFs> replicatedTableCFs = new ArrayList<>();
- tables.forEach(table -> {
- Map<String, Integer> cfs = new HashMap<>();
- Stream.of(table.getColumnFamilies())
- .filter(column -> column.getScope() != HConstants.REPLICATION_SCOPE_LOCAL)
- .forEach(column -> {
- cfs.put(column.getNameAsString(), column.getScope());
- });
- if (!cfs.isEmpty()) {
- replicatedTableCFs.add(new TableCFs(table.getTableName(), cfs));
- }
- });
- future.complete(replicatedTableCFs);
- }
- });
+ addListener(listTableDescriptors(), (tables, error) -> {
+ if (!completeExceptionally(future, error)) {
+ List<TableCFs> replicatedTableCFs = new ArrayList<>();
+ tables.forEach(table -> {
+ Map<String, Integer> cfs = new HashMap<>();
+ Stream.of(table.getColumnFamilies())
+ .filter(column -> column.getScope() != HConstants.REPLICATION_SCOPE_LOCAL)
+ .forEach(column -> {
+ cfs.put(column.getNameAsString(), column.getScope());
+ });
+ if (!cfs.isEmpty()) {
+ replicatedTableCFs.add(new TableCFs(table.getTableName(), cfs));
+ }
+ });
+ future.complete(replicatedTableCFs);
+ }
+ });
return future;
}
@Override
public CompletableFuture<Void> snapshot(SnapshotDescription snapshotDesc) {
- SnapshotProtos.SnapshotDescription snapshot = ProtobufUtil
- .createHBaseProtosSnapshotDesc(snapshotDesc);
+ SnapshotProtos.SnapshotDescription snapshot =
+ ProtobufUtil.createHBaseProtosSnapshotDesc(snapshotDesc);
try {
ClientSnapshotDescriptionUtils.assertSnapshotRequestIsValid(snapshot);
} catch (IllegalArgumentException e) {
@@ -1742,47 +1731,47 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
}
CompletableFuture<Void> future = new CompletableFuture<>();
final SnapshotRequest request = SnapshotRequest.newBuilder().setSnapshot(snapshot).build();
- this.<Long> newMasterCaller()
- .action(
- (controller, stub) -> this.<SnapshotRequest, SnapshotResponse, Long> call(controller,
- stub, request, (s, c, req, done) -> s.snapshot(c, req, done),
- resp -> resp.getExpectedTimeout())).call().whenComplete((expectedTimeout, err) -> {
- if (err != null) {
- future.completeExceptionally(err);
- return;
- }
- TimerTask pollingTask = new TimerTask() {
- int tries = 0;
- long startTime = EnvironmentEdgeManager.currentTime();
- long endTime = startTime + expectedTimeout;
- long maxPauseTime = expectedTimeout / maxAttempts;
-
- @Override
- public void run(Timeout timeout) throws Exception {
- if (EnvironmentEdgeManager.currentTime() < endTime) {
- isSnapshotFinished(snapshotDesc).whenComplete((done, err2) -> {
- if (err2 != null) {
- future.completeExceptionally(err2);
- } else if (done) {
- future.complete(null);
- } else {
- // retry again after pauseTime.
- long pauseTime = ConnectionUtils.getPauseTime(
- TimeUnit.NANOSECONDS.toMillis(pauseNs), ++tries);
+ addListener(this.<Long> newMasterCaller()
+ .action((controller, stub) -> this.<SnapshotRequest, SnapshotResponse, Long> call(controller,
+ stub, request, (s, c, req, done) -> s.snapshot(c, req, done),
+ resp -> resp.getExpectedTimeout()))
+ .call(), (expectedTimeout, err) -> {
+ if (err != null) {
+ future.completeExceptionally(err);
+ return;
+ }
+ TimerTask pollingTask = new TimerTask() {
+ int tries = 0;
+ long startTime = EnvironmentEdgeManager.currentTime();
+ long endTime = startTime + expectedTimeout;
+ long maxPauseTime = expectedTimeout / maxAttempts;
+
+ @Override
+ public void run(Timeout timeout) throws Exception {
+ if (EnvironmentEdgeManager.currentTime() < endTime) {
+ addListener(isSnapshotFinished(snapshotDesc), (done, err2) -> {
+ if (err2 != null) {
+ future.completeExceptionally(err2);
+ } else if (done) {
+ future.complete(null);
+ } else {
+ // retry again after pauseTime.
+ long pauseTime =
+ ConnectionUtils.getPauseTime(TimeUnit.NANOSECONDS.toMillis(pauseNs), ++tries);
pauseTime = Math.min(pauseTime, maxPauseTime);
- AsyncConnectionImpl.RETRY_TIMER
- .newTimeout(this, pauseTime, TimeUnit.MILLISECONDS);
+ AsyncConnectionImpl.RETRY_TIMER.newTimeout(this, pauseTime,
+ TimeUnit.MILLISECONDS);
}
- } );
- } else {
- future.completeExceptionally(new SnapshotCreationException("Snapshot '"
- + snapshot.getName() + "' wasn't completed in expectedTime:" + expectedTimeout
- + " ms", snapshotDesc));
- }
+ });
+ } else {
+ future.completeExceptionally(
+ new SnapshotCreationException("Snapshot '" + snapshot.getName() +
+ "' wasn't completed in expectedTime:" + expectedTimeout + " ms", snapshotDesc));
}
- };
- AsyncConnectionImpl.RETRY_TIMER.newTimeout(pollingTask, 1, TimeUnit.MILLISECONDS);
- });
+ }
+ };
+ AsyncConnectionImpl.RETRY_TIMER.newTimeout(pollingTask, 1, TimeUnit.MILLISECONDS);
+ });
return future;
}
@@ -1808,52 +1797,50 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
}
@Override
- public CompletableFuture<Void> restoreSnapshot(String snapshotName, boolean takeFailSafeSnapshot) {
+ public CompletableFuture<Void> restoreSnapshot(String snapshotName,
+ boolean takeFailSafeSnapshot) {
CompletableFuture<Void> future = new CompletableFuture<>();
- listSnapshots(Pattern.compile(snapshotName)).whenComplete(
- (snapshotDescriptions, err) -> {
- if (err != null) {
- future.completeExceptionally(err);
- return;
- }
- TableName tableName = null;
- if (snapshotDescriptions != null && !snapshotDescriptions.isEmpty()) {
- for (SnapshotDescription snap : snapshotDescriptions) {
- if (snap.getName().equals(snapshotName)) {
- tableName = snap.getTableName();
- break;
- }
+ addListener(listSnapshots(Pattern.compile(snapshotName)), (snapshotDescriptions, err) -> {
+ if (err != null) {
+ future.completeExceptionally(err);
+ return;
+ }
+ TableName tableName = null;
+ if (snapshotDescriptions != null && !snapshotDescriptions.isEmpty()) {
+ for (SnapshotDescription snap : snapshotDescriptions) {
+ if (snap.getName().equals(snapshotName)) {
+ tableName = snap.getTableName();
+ break;
}
}
- if (tableName == null) {
- future.completeExceptionally(new RestoreSnapshotException(
- "Unable to find the table name for snapshot=" + snapshotName));
- return;
- }
- final TableName finalTableName = tableName;
- tableExists(finalTableName)
- .whenComplete((exists, err2) -> {
- if (err2 != null) {
- future.completeExceptionally(err2);
- } else if (!exists) {
- // if table does not exist, then just clone snapshot into new table.
- completeConditionalOnFuture(future,
- internalRestoreSnapshot(snapshotName, finalTableName));
+ }
+ if (tableName == null) {
+ future.completeExceptionally(new RestoreSnapshotException(
+ "Unable to find the table name for snapshot=" + snapshotName));
+ return;
+ }
+ final TableName finalTableName = tableName;
+ addListener(tableExists(finalTableName), (exists, err2) -> {
+ if (err2 != null) {
+ future.completeExceptionally(err2);
+ } else if (!exists) {
+ // if table does not exist, then just clone snapshot into new table.
+ completeConditionalOnFuture(future,
+ internalRestoreSnapshot(snapshotName, finalTableName));
+ } else {
+ addListener(isTableDisabled(finalTableName), (disabled, err4) -> {
+ if (err4 != null) {
+ future.completeExceptionally(err4);
+ } else if (!disabled) {
+ future.completeExceptionally(new TableNotDisabledException(finalTableName));
} else {
- isTableDisabled(finalTableName).whenComplete(
- (disabled, err4) -> {
- if (err4 != null) {
- future.completeExceptionally(err4);
- } else if (!disabled) {
- future.completeExceptionally(new TableNotDisabledException(finalTableName));
- } else {
- completeConditionalOnFuture(future,
- restoreSnapshot(snapshotName, finalTableName, takeFailSafeSnapshot));
- }
- });
+ completeConditionalOnFuture(future,
+ restoreSnapshot(snapshotName, finalTableName, takeFailSafeSnapshot));
}
- } );
+ });
+ }
});
+ });
return future;
}
@@ -1862,49 +1849,50 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
if (takeFailSafeSnapshot) {
CompletableFuture<Void> future = new CompletableFuture<>();
// Step.1 Take a snapshot of the current state
- String failSafeSnapshotSnapshotNameFormat = this.connection.getConfiguration().get(
- HConstants.SNAPSHOT_RESTORE_FAILSAFE_NAME,
- HConstants.DEFAULT_SNAPSHOT_RESTORE_FAILSAFE_NAME);
- final String failSafeSnapshotSnapshotName = failSafeSnapshotSnapshotNameFormat
- .replace("{snapshot.name}", snapshotName)
+ String failSafeSnapshotSnapshotNameFormat =
+ this.connection.getConfiguration().get(HConstants.SNAPSHOT_RESTORE_FAILSAFE_NAME,
+ HConstants.DEFAULT_SNAPSHOT_RESTORE_FAILSAFE_NAME);
+ final String failSafeSnapshotSnapshotName =
+ failSafeSnapshotSnapshotNameFormat.replace("{snapshot.name}", snapshotName)
.replace("{table.name}", tableName.toString().replace(TableName.NAMESPACE_DELIM, '.'))
.replace("{restore.timestamp}", String.valueOf(EnvironmentEdgeManager.currentTime()));
LOG.info("Taking restore-failsafe snapshot: " + failSafeSnapshotSnapshotName);
- snapshot(failSafeSnapshotSnapshotName, tableName).whenComplete((ret, err) -> {
+ addListener(snapshot(failSafeSnapshotSnapshotName, tableName), (ret, err) -> {
if (err != null) {
future.completeExceptionally(err);
} else {
// Step.2 Restore snapshot
- internalRestoreSnapshot(snapshotName, tableName).whenComplete((void2, err2) -> {
- if (err2 != null) {
- // Step.3.a Something went wrong during the restore and try to rollback.
- internalRestoreSnapshot(failSafeSnapshotSnapshotName, tableName).whenComplete(
- (void3, err3) -> {
- if (err3 != null) {
- future.completeExceptionally(err3);
- } else {
- String msg = "Restore snapshot=" + snapshotName + " failed. Rollback to snapshot="
- + failSafeSnapshotSnapshotName + " succeeded.";
- future.completeExceptionally(new RestoreSnapshotException(msg));
- }
- });
- } else {
- // Step.3.b If the restore is succeeded, delete the pre-restore snapshot.
- LOG.info("Deleting restore-failsafe snapshot: " + failSafeSnapshotSnapshotName);
- deleteSnapshot(failSafeSnapshotSnapshotName).whenComplete(
- (ret3, err3) -> {
- if (err3 != null) {
- LOG.error(
- "Unable to remove the failsafe snapshot: " + failSafeSnapshotSnapshotName, err3);
- future.completeExceptionally(err3);
- } else {
- future.complete(ret3);
- }
- });
+ addListener(internalRestoreSnapshot(snapshotName, tableName), (void2, err2) -> {
+ if (err2 != null) {
+ // Step.3.a Something went wrong during the restore and try to rollback.
+ addListener(internalRestoreSnapshot(failSafeSnapshotSnapshotName, tableName),
+ (void3, err3) -> {
+ if (err3 != null) {
+ future.completeExceptionally(err3);
+ } else {
+ String msg =
+ "Restore snapshot=" + snapshotName + " failed. Rollback to snapshot=" +
+ failSafeSnapshotSnapshotName + " succeeded.";
+ future.completeExceptionally(new RestoreSnapshotException(msg));
+ }
+ });
+ } else {
+ // Step.3.b If the restore is succeeded, delete the pre-restore snapshot.
+ LOG.info("Deleting restore-failsafe snapshot: " + failSafeSnapshotSnapshotName);
+ addListener(deleteSnapshot(failSafeSnapshotSnapshotName), (ret3, err3) -> {
+ if (err3 != null) {
+ LOG.error(
+ "Unable to remove the failsafe snapshot: " + failSafeSnapshotSnapshotName,
+ err3);
+ future.completeExceptionally(err3);
+ } else {
+ future.complete(ret3);
+ }
+ });
+ }
+ });
}
- } );
- }
- } );
+ });
return future;
} else {
return internalRestoreSnapshot(snapshotName, tableName);
@@ -1913,7 +1901,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
private <T> void completeConditionalOnFuture(CompletableFuture<T> dependentFuture,
CompletableFuture<T> parentFuture) {
- parentFuture.whenComplete((res, err) -> {
+ addListener(parentFuture, (res, err) -> {
if (err != null) {
dependentFuture.completeExceptionally(err);
} else {
@@ -1925,7 +1913,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
@Override
public CompletableFuture<Void> cloneSnapshot(String snapshotName, TableName tableName) {
CompletableFuture<Void> future = new CompletableFuture<>();
- tableExists(tableName).whenComplete((exists, err) -> {
+ addListener(tableExists(tableName), (exists, err) -> {
if (err != null) {
future.completeExceptionally(err);
} else if (exists) {
@@ -1995,31 +1983,29 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
private CompletableFuture<List<SnapshotDescription>> getCompletedSnapshots(
Pattern tableNamePattern, Pattern snapshotNamePattern) {
CompletableFuture<List<SnapshotDescription>> future = new CompletableFuture<>();
- listTableNames(tableNamePattern, false).whenComplete(
- (tableNames, err) -> {
- if (err != null) {
- future.completeExceptionally(err);
+ addListener(listTableNames(tableNamePattern, false), (tableNames, err) -> {
+ if (err != null) {
+ future.completeExceptionally(err);
+ return;
+ }
+ if (tableNames == null || tableNames.size() <= 0) {
+ future.complete(Collections.emptyList());
+ return;
+ }
+ addListener(getCompletedSnapshots(snapshotNamePattern), (snapshotDescList, err2) -> {
+ if (err2 != null) {
+ future.completeExceptionally(err2);
return;
}
- if (tableNames == null || tableNames.size() <= 0) {
+ if (snapshotDescList == null || snapshotDescList.isEmpty()) {
future.complete(Collections.emptyList());
return;
}
- getCompletedSnapshots(snapshotNamePattern).whenComplete(
- (snapshotDescList, err2) -> {
- if (err2 != null) {
- future.completeExceptionally(err2);
- return;
- }
- if (snapshotDescList == null || snapshotDescList.isEmpty()) {
- future.complete(Collections.emptyList());
- return;
- }
- future.complete(snapshotDescList.stream()
- .filter(snap -> (snap != null && tableNames.contains(snap.getTableName())))
- .collect(Collectors.toList()));
- });
+ future.complete(snapshotDescList.stream()
+ .filter(snap -> (snap != null && tableNames.contains(snap.getTableName())))
+ .collect(Collectors.toList()));
});
+ });
return future;
}
@@ -2066,7 +2052,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
listSnapshotsFuture = getCompletedSnapshots(tableNamePattern, snapshotNamePattern);
}
CompletableFuture<Void> future = new CompletableFuture<>();
- listSnapshotsFuture.whenComplete(((snapshotDescriptions, err) -> {
+ addListener(listSnapshotsFuture, ((snapshotDescriptions, err) -> {
if (err != null) {
future.completeExceptionally(err);
return;
@@ -2075,12 +2061,14 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
future.complete(null);
return;
}
- List<CompletableFuture<Void>> deleteSnapshotFutures = new ArrayList<>();
- snapshotDescriptions.forEach(snapDesc -> deleteSnapshotFutures
- .add(internalDeleteSnapshot(snapDesc)));
- CompletableFuture.allOf(
- deleteSnapshotFutures.toArray(new CompletableFuture<?>[deleteSnapshotFutures.size()]))
- .thenAccept(v -> future.complete(v));
+ addListener(CompletableFuture.allOf(snapshotDescriptions.stream()
+ .map(this::internalDeleteSnapshot).toArray(CompletableFuture[]::new)), (v, e) -> {
+ if (e != null) {
+ future.completeExceptionally(e);
+ } else {
+ future.complete(v);
+ }
+ });
}));
return future;
}
@@ -2102,50 +2090,50 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
Map<String, String> props) {
CompletableFuture<Void> future = new CompletableFuture<>();
ProcedureDescription procDesc =
- ProtobufUtil.buildProcedureDescription(signature, instance, props);
- this.<Long> newMasterCaller()
- .action((controller, stub) -> this.<ExecProcedureRequest, ExecProcedureResponse, Long> call(
- controller, stub, ExecProcedureRequest.newBuilder().setProcedure(procDesc).build(),
- (s, c, req, done) -> s.execProcedure(c, req, done), resp -> resp.getExpectedTimeout()))
- .call().whenComplete((expectedTimeout, err) -> {
- if (err != null) {
- future.completeExceptionally(err);
- return;
- }
- TimerTask pollingTask = new TimerTask() {
- int tries = 0;
- long startTime = EnvironmentEdgeManager.currentTime();
- long endTime = startTime + expectedTimeout;
- long maxPauseTime = expectedTimeout / maxAttempts;
-
- @Override
- public void run(Timeout timeout) throws Exception {
- if (EnvironmentEdgeManager.currentTime() < endTime) {
- isProcedureFinished(signature, instance, props).whenComplete((done, err2) -> {
- if (err2 != null) {
- future.completeExceptionally(err2);
- return;
- }
- if (done) {
- future.complete(null);
- } else {
- // retry again after pauseTime.
- long pauseTime = ConnectionUtils
- .getPauseTime(TimeUnit.NANOSECONDS.toMillis(pauseNs), ++tries);
- pauseTime = Math.min(pauseTime, maxPauseTime);
- AsyncConnectionImpl.RETRY_TIMER.newTimeout(this, pauseTime,
- TimeUnit.MICROSECONDS);
- }
- });
- } else {
- future.completeExceptionally(new IOException("Procedure '" + signature + " : "
- + instance + "' wasn't completed in expectedTime:" + expectedTimeout + " ms"));
- }
+ ProtobufUtil.buildProcedureDescription(signature, instance, props);
+ addListener(this.<Long> newMasterCaller()
+ .action((controller, stub) -> this.<ExecProcedureRequest, ExecProcedureResponse, Long> call(
+ controller, stub, ExecProcedureRequest.newBuilder().setProcedure(procDesc).build(),
+ (s, c, req, done) -> s.execProcedure(c, req, done), resp -> resp.getExpectedTimeout()))
+ .call(), (expectedTimeout, err) -> {
+ if (err != null) {
+ future.completeExceptionally(err);
+ return;
+ }
+ TimerTask pollingTask = new TimerTask() {
+ int tries = 0;
+ long startTime = EnvironmentEdgeManager.currentTime();
+ long endTime = startTime + expectedTimeout;
+ long maxPauseTime = expectedTimeout / maxAttempts;
+
+ @Override
+ public void run(Timeout timeout) throws Exception {
+ if (EnvironmentEdgeManager.currentTime() < endTime) {
+ addListener(isProcedureFinished(signature, instance, props), (done, err2) -> {
+ if (err2 != null) {
+ future.completeExceptionally(err2);
+ return;
+ }
+ if (done) {
+ future.complete(null);
+ } else {
+ // retry again after pauseTime.
+ long pauseTime =
+ ConnectionUtils.getPauseTime(TimeUnit.NANOSECONDS.toMillis(pauseNs), ++tries);
+ pauseTime = Math.min(pauseTime, maxPauseTime);
+ AsyncConnectionImpl.RETRY_TIMER.newTimeout(this, pauseTime,
+ TimeUnit.MICROSECONDS);
+ }
+ });
+ } else {
+ future.completeExceptionally(new IOException("Procedure '" + signature + " : " +
+ instance + "' wasn't completed in expectedTime:" + expectedTimeout + " ms"));
}
- };
- // Queue the polling task into RETRY_TIMER to poll procedure state asynchronously.
- AsyncConnectionImpl.RETRY_TIMER.newTimeout(pollingTask, 1, TimeUnit.MILLISECONDS);
- });
+ }
+ };
+ // Queue the polling task into RETRY_TIMER to poll procedure state asynchronously.
+ AsyncConnectionImpl.RETRY_TIMER.newTimeout(pollingTask, 1, TimeUnit.MILLISECONDS);
+ });
return future;
}
@@ -2264,15 +2252,15 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
}
CompletableFuture<HRegionLocation> returnedFuture = new CompletableFuture<>();
- future.whenComplete((location, err) -> {
+ addListener(future, (location, err) -> {
if (err != null) {
returnedFuture.completeExceptionally(err);
return;
}
if (!location.isPresent() || location.get().getRegion() == null) {
- returnedFuture.completeExceptionally(new UnknownRegionException(
- "Invalid region name or encoded region name: "
- + Bytes.toStringBinary(regionNameOrEncodedRegionName)));
+ returnedFuture.completeExceptionally(
+ new UnknownRegionException("Invalid region name or encoded region name: " +
+ Bytes.toStringBinary(regionNameOrEncodedRegionName)));
} else {
returnedFuture.complete(location.get());
}
@@ -2296,14 +2284,14 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
}
if (Bytes.equals(regionNameOrEncodedRegionName,
- RegionInfoBuilder.FIRST_META_REGIONINFO.getRegionName())
- || Bytes.equals(regionNameOrEncodedRegionName,
+ RegionInfoBuilder.FIRST_META_REGIONINFO.getRegionName()) ||
+ Bytes.equals(regionNameOrEncodedRegionName,
RegionInfoBuilder.FIRST_META_REGIONINFO.getEncodedNameAsBytes())) {
return CompletableFuture.completedFuture(RegionInfoBuilder.FIRST_META_REGIONINFO);
}
CompletableFuture<RegionInfo> future = new CompletableFuture<>();
- getRegionLocation(regionNameOrEncodedRegionName).whenComplete((location, err) -> {
+ addListener(getRegionLocation(regionNameOrEncodedRegionName), (location, err) -> {
if (err != null) {
future.completeExceptionally(err);
} else {
@@ -2345,7 +2333,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
}
}
- private abstract class ProcedureBiConsumer implements BiConsumer<Void, Throwable> {
+ private static abstract class ProcedureBiConsumer implements BiConsumer<Void, Throwable> {
abstract void onFinished();
@@ -2361,7 +2349,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
}
}
- private abstract class TableProcedureBiConsumer extends ProcedureBiConsumer {
+ private static abstract class TableProcedureBiConsumer extends ProcedureBiConsumer {
protected final TableName tableName;
TableProcedureBiConsumer(TableName tableName) {
@@ -2386,7 +2374,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
}
}
- private abstract class NamespaceProcedureBiConsumer extends ProcedureBiConsumer {
+ private static abstract class NamespaceProcedureBiConsumer extends ProcedureBiConsumer {
protected final String namespaceName;
NamespaceProcedureBiConsumer(String namespaceName) {
@@ -2410,7 +2398,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
}
}
- private class CreateTableProcedureBiConsumer extends TableProcedureBiConsumer {
+ private static class CreateTableProcedureBiConsumer extends TableProcedureBiConsumer {
CreateTableProcedureBiConsumer(TableName tableName) {
super(tableName);
@@ -2422,7 +2410,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
}
}
- private class ModifyTableProcedureBiConsumer extends TableProcedureBiConsumer {
+ private static class ModifyTableProcedureBiConsumer extends TableProcedureBiConsumer {
ModifyTableProcedureBiConsumer(AsyncAdmin admin, TableName tableName) {
super(tableName);
@@ -2452,7 +2440,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
}
}
- private class TruncateTableProcedureBiConsumer extends TableProcedureBiConsumer {
+ private static class TruncateTableProcedureBiConsumer extends TableProcedureBiConsumer {
TruncateTableProcedureBiConsumer(TableName tableName) {
super(tableName);
@@ -2464,7 +2452,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
}
}
- private class EnableTableProcedureBiConsumer extends TableProcedureBiConsumer {
+ private static class EnableTableProcedureBiConsumer extends TableProcedureBiConsumer {
EnableTableProcedureBiConsumer(TableName tableName) {
super(tableName);
@@ -2476,7 +2464,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
}
}
- private class DisableTableProcedureBiConsumer extends TableProcedureBiConsumer {
+ private static class DisableTableProcedureBiConsumer extends TableProcedureBiConsumer {
DisableTableProcedureBiConsumer(TableName tableName) {
super(tableName);
@@ -2488,7 +2476,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
}
}
- private class AddColumnFamilyProcedureBiConsumer extends TableProcedureBiConsumer {
+ private static class AddColumnFamilyProcedureBiConsumer extends TableProcedureBiConsumer {
AddColumnFamilyProcedureBiConsumer(TableName tableName) {
super(tableName);
@@ -2500,7 +2488,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
}
}
- private class DeleteColumnFamilyProcedureBiConsumer extends TableProcedureBiConsumer {
+ private static class DeleteColumnFamilyProcedureBiConsumer extends TableProcedureBiConsumer {
DeleteColumnFamilyProcedureBiConsumer(TableName tableName) {
super(tableName);
@@ -2512,7 +2500,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
}
}
- private class ModifyColumnFamilyProcedureBiConsumer extends TableProcedureBiConsumer {
+ private static class ModifyColumnFamilyProcedureBiConsumer extends TableProcedureBiConsumer {
ModifyColumnFamilyProcedureBiConsumer(TableName tableName) {
super(tableName);
@@ -2524,7 +2512,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
}
}
- private class CreateNamespaceProcedureBiConsumer extends NamespaceProcedureBiConsumer {
+ private static class CreateNamespaceProcedureBiConsumer extends NamespaceProcedureBiConsumer {
CreateNamespaceProcedureBiConsumer(String namespaceName) {
super(namespaceName);
@@ -2536,7 +2524,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
}
}
- private class DeleteNamespaceProcedureBiConsumer extends NamespaceProcedureBiConsumer {
+ private static class DeleteNamespaceProcedureBiConsumer extends NamespaceProcedureBiConsumer {
DeleteNamespaceProcedureBiConsumer(String namespaceName) {
super(namespaceName);
@@ -2548,7 +2536,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
}
}
- private class ModifyNamespaceProcedureBiConsumer extends NamespaceProcedureBiConsumer {
+ private static class ModifyNamespaceProcedureBiConsumer extends NamespaceProcedureBiConsumer {
ModifyNamespaceProcedureBiConsumer(String namespaceName) {
super(namespaceName);
@@ -2560,7 +2548,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
}
}
- private class MergeTableRegionProcedureBiConsumer extends TableProcedureBiConsumer {
+ private static class MergeTableRegionProcedureBiConsumer extends TableProcedureBiConsumer {
MergeTableRegionProcedureBiConsumer(TableName tableName) {
super(tableName);
@@ -2572,7 +2560,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
}
}
- private class SplitTableRegionProcedureBiConsumer extends TableProcedureBiConsumer {
+ private static class SplitTableRegionProcedureBiConsumer extends TableProcedureBiConsumer {
SplitTableRegionProcedureBiConsumer(TableName tableName) {
super(tableName);
@@ -2584,7 +2572,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
}
}
- private class ReplicationProcedureBiConsumer extends ProcedureBiConsumer {
+ private static class ReplicationProcedureBiConsumer extends ProcedureBiConsumer {
private final String peerId;
private final Supplier<String> getOperation;
@@ -2610,7 +2598,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
private CompletableFuture<Void> waitProcedureResult(CompletableFuture<Long> procFuture) {
CompletableFuture<Void> future = new CompletableFuture<>();
- procFuture.whenComplete((procId, error) -> {
+ addListener(procFuture, (procId, error) -> {
if (error != null) {
future.completeExceptionally(error);
return;
@@ -2621,30 +2609,33 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
}
private void getProcedureResult(long procId, CompletableFuture<Void> future, int retries) {
- this.<GetProcedureResultResponse> newMasterCaller().action((controller, stub) -> this
- .<GetProcedureResultRequest, GetProcedureResultResponse, GetProcedureResultResponse> call(
- controller, stub, GetProcedureResultRequest.newBuilder().setProcId(procId).build(),
- (s, c, req, done) -> s.getProcedureResult(c, req, done), (resp) -> resp))
- .call().whenComplete((response, error) -> {
- if (error != null) {
- LOG.warn("failed to get the procedure result procId={}", procId,
- ConnectionUtils.translateException(error));
- retryTimer.newTimeout(t -> getProcedureResult(procId, future, retries + 1),
- ConnectionUtils.getPauseTime(pauseNs, retries), TimeUnit.NANOSECONDS);
- return;
- }
- if (response.getState() == GetProcedureResultResponse.State.RUNNING) {
- retryTimer.newTimeout(t -> getProcedureResult(procId, future, retries + 1),
- ConnectionUtils.getPauseTime(pauseNs, retries), TimeUnit.NANOSECONDS);
- return;
- }
- if (response.hasException()) {
- IOException ioe = ForeignExceptionUtil.toIOException(response.getException());
- future.completeExceptionally(ioe);
- } else {
- future.complete(null);
- }
- });
+ addListener(
+ this.<GetProcedureResultResponse> newMasterCaller()
+ .action((controller, stub) -> this
+ .<GetProcedureResultRequest, GetProcedureResultResponse, GetProcedureResultResponse> call(
+ controller, stub, GetProcedureResultRequest.newBuilder().setProcId(procId).build(),
+ (s, c, req, done) -> s.getProcedureResult(c, req, done), (resp) -> resp))
+ .call(),
+ (response, error) -> {
+ if (error != null) {
+ LOG.warn("failed to get the procedure result procId={}", procId,
+ ConnectionUtils.translateException(error));
+ retryTimer.newTimeout(t -> getProcedureResult(procId, future, retries + 1),
+ ConnectionUtils.getPauseTime(pauseNs, retries), TimeUnit.NANOSECONDS);
+ return;
+ }
+ if (response.getState() == GetProcedureResultResponse.State.RUNNING) {
+ retryTimer.newTimeout(t -> getProcedureResult(procId, future, retries + 1),
+ ConnectionUtils.getPauseTime(pauseNs, retries), TimeUnit.NANOSECONDS);
+ return;
+ }
+ if (response.hasException()) {
+ IOException ioe = ForeignExceptionUtil.toIOException(response.getException());
+ future.completeExceptionally(ioe);
+ } else {
+ future.complete(null);
+ }
+ });
}
private <T> CompletableFuture<T> failedFuture(Throwable error) {
@@ -2726,24 +2717,26 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
@Override
public CompletableFuture<Void> updateConfiguration() {
CompletableFuture<Void> future = new CompletableFuture<Void>();
- getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS, Option.MASTER, Option.BACKUP_MASTERS))
- .whenComplete((status, err) -> {
+ addListener(
+ getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS, Option.MASTER, Option.BACKUP_MASTERS)),
+ (status, err) -> {
if (err != null) {
future.completeExceptionally(err);
} else {
List<CompletableFuture<Void>> futures = new ArrayList<>();
status.getLiveServerMetrics().keySet()
- .forEach(server -> futures.add(updateConfiguration(server)));
+ .forEach(server -> futures.add(updateConfiguration(server)));
futures.add(updateConfiguration(status.getMasterName()));
status.getBackupMasterNames().forEach(master -> futures.add(updateConfiguration(master)));
- CompletableFuture.allOf(futures.toArray(new CompletableFuture<?>[futures.size()]))
- .whenComplete((result, err2) -> {
- if (err2 != null) {
- future.completeExceptionally(err2);
- } else {
- future.complete(result);
- }
- });
+ addListener(
+ CompletableFuture.allOf(futures.toArray(new CompletableFuture<?>[futures.size()])),
+ (result, err2) -> {
+ if (err2 != null) {
+ future.completeExceptionally(err2);
+ } else {
+ future.complete(result);
+ }
+ });
}
});
return future;
@@ -2826,88 +2819,87 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
switch (compactType) {
case MOB:
- connection.registry.getMasterAddress().whenComplete((serverName, err) -> {
+ addListener(connection.registry.getMasterAddress(), (serverName, err) -> {
if (err != null) {
future.completeExceptionally(err);
return;
}
RegionInfo regionInfo = RegionInfo.createMobRegionInfo(tableName);
- this.<GetRegionInfoResponse> newAdminCaller().serverName(serverName).action(
- (controller, stub) -> this
- .<GetRegionInfoRequest, GetRegionInfoResponse, GetRegionInfoResponse> adminCall(
+ addListener(this.<GetRegionInfoResponse> newAdminCaller().serverName(serverName)
+ .action((controller, stub) -> this
+ .<GetRegionInfoRequest, GetRegionInfoResponse, GetRegionInfoResponse> adminCall(
controller, stub,
RequestConverter.buildGetRegionInfoRequest(regionInfo.getRegionName(), true),
- (s, c, req, done) -> s.getRegionInfo(controller, req, done), resp -> resp)
- ).call().whenComplete((resp2, err2) -> {
- if (err2 != null) {
- future.completeExceptionally(err2);
- } else {
- if (resp2.hasCompactionState()) {
- future.complete(ProtobufUtil.createCompactionState(resp2.getCompactionState()));
+ (s, c, req, done) -> s.getRegionInfo(controller, req, done), resp -> resp))
+ .call(), (resp2, err2) -> {
+ if (err2 != null) {
+ future.completeExceptionally(err2);
} else {
- future.complete(CompactionState.NONE);
+ if (resp2.hasCompactionState()) {
+ future.complete(ProtobufUtil.createCompactionState(resp2.getCompactionState()));
+ } else {
+ future.complete(CompactionState.NONE);
+ }
}
- }
- });
+ });
});
break;
case NORMAL:
- getTableHRegionLocations(tableName).whenComplete(
- (locations, err) -> {
- if (err != null) {
- future.completeExceptionally(err);
- return;
- }
- List<CompactionState> regionStates = new ArrayList<>();
- List<CompletableFuture<CompactionState>> futures = new ArrayList<>();
- locations.stream().filter(loc -> loc.getServerName() != null)
- .filter(loc -> loc.getRegion() != null)
- .filter(loc -> !loc.getRegion().isOffline())
- .map(loc -> loc.getRegion().getRegionName()).forEach(region -> {
- futures.add(getCompactionStateForRegion(region).whenComplete((regionState, err2) -> {
- // If any region compaction state is MAJOR_AND_MINOR
- // the table compaction state is MAJOR_AND_MINOR, too.
- if (err2 != null) {
- future.completeExceptionally(err2);
- } else if (regionState == CompactionState.MAJOR_AND_MINOR) {
- future.complete(regionState);
- } else {
- regionStates.add(regionState);
- }
- }));
- });
- CompletableFuture.allOf(futures.toArray(new CompletableFuture<?>[futures.size()]))
- .whenComplete((ret, err3) -> {
- // If future not completed, check all regions's compaction state
- if (!future.isCompletedExceptionally() && !future.isDone()) {
- CompactionState state = CompactionState.NONE;
- for (CompactionState regionState : regionStates) {
- switch (regionState) {
- case MAJOR:
- if (state == CompactionState.MINOR) {
- future.complete(CompactionState.MAJOR_AND_MINOR);
- } else {
- state = CompactionState.MAJOR;
- }
- break;
- case MINOR:
- if (state == CompactionState.MAJOR) {
- future.complete(CompactionState.MAJOR_AND_MINOR);
- } else {
- state = CompactionState.MINOR;
- }
- break;
- case NONE:
- default:
+ addListener(getTableHRegionLocations(tableName), (locations, err) -> {
+ if (err != null) {
+ future.completeExceptionally(err);
+ return;
+ }
+ List<CompactionState> regionStates = new ArrayList<>();
+ List<CompletableFuture<CompactionState>> futures = new ArrayList<>();
+ locations.stream().filter(loc -> loc.getServerName() != null)
+ .filter(loc -> loc.getRegion() != null).filter(loc -> !loc.getRegion().isOffline())
+ .map(loc -> loc.getRegion().getRegionName()).forEach(region -> {
+ futures.add(getCompactionStateForRegion(region).whenComplete((regionState, err2) -> {
+ // If any region compaction state is MAJOR_AND_MINOR
+ // the table compaction state is MAJOR_AND_MINOR, too.
+ if (err2 != null) {
+ future.completeExceptionally(err2);
+ } else if (regionState == CompactionState.MAJOR_AND_MINOR) {
+ future.complete(regionState);
+ } else {
+ regionStates.add(regionState);
+ }
+ }));
+ });
+ addListener(
+ CompletableFuture.allOf(futures.toArray(new CompletableFuture<?>[futures.size()])),
+ (ret, err3) -> {
+ // If future not completed, check all regions's compaction state
+ if (!future.isCompletedExceptionally() && !future.isDone()) {
+ CompactionState state = CompactionState.NONE;
+ for (CompactionState regionState : regionStates) {
+ switch (regionState) {
+ case MAJOR:
+ if (state == CompactionState.MINOR) {
+ future.complete(CompactionState.MAJOR_AND_MINOR);
+ } else {
+ state = CompactionState.MAJOR;
}
- if (!future.isDone()) {
- future.complete(state);
+ break;
+ case MINOR:
+ if (state == CompactionState.MAJOR) {
+ future.complete(CompactionState.MAJOR_AND_MINOR);
+ } else {
+ state = CompactionState.MINOR;
}
- }
+ break;
+ case NONE:
+ default:
}
- });
- });
+ if (!future.isDone()) {
+ future.complete(state);
+ }
+ }
+ }
+ });
+ });
break;
default:
throw new IllegalArgumentException("Unknown compactType: " + compactType);
@@ -2919,37 +2911,38 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
@Override
public CompletableFuture<CompactionState> getCompactionStateForRegion(byte[] regionName) {
CompletableFuture<CompactionState> future = new CompletableFuture<>();
- getRegionLocation(regionName).whenComplete(
- (location, err) -> {
- if (err != null) {
- future.completeExceptionally(err);
- return;
- }
- ServerName serverName = location.getServerName();
- if (serverName == null) {
- future.completeExceptionally(new NoServerForRegionException(Bytes
- .toStringBinary(regionName)));
- return;
- }
+ addListener(getRegionLocation(regionName), (location, err) -> {
+ if (err != null) {
+ future.completeExceptionally(err);
+ return;
+ }
+ ServerName serverName = location.getServerName();
+ if (serverName == null) {
+ future
+ .completeExceptionally(new NoServerForRegionException(Bytes.toStringBinary(regionName)));
+ return;
+ }
+ addListener(
this.<GetRegionInfoResponse> newAdminCaller()
- .action(
- (controller, stub) -> this
- .<GetRegionInfoRequest, GetRegionInfoResponse, GetRegionInfoResponse> adminCall(
- controller, stub, RequestConverter.buildGetRegionInfoRequest(location
- .getRegion().getRegionName(), true), (s, c, req, done) -> s
-
<TRUNCATED>
[15/15] hbase git commit: HBASE-21579 Use AsyncClusterConnection for
HBaseInterClusterReplicationEndpoint
Posted by zh...@apache.org.
HBASE-21579 Use AsyncClusterConnection for HBaseInterClusterReplicationEndpoint
Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/04e6909a
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/04e6909a
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/04e6909a
Branch: refs/heads/HBASE-21512
Commit: 04e6909adfdf2e3a5b92fadb13dd8fc06e15c5e3
Parents: 0d051ab
Author: zhangduo <zh...@apache.org>
Authored: Tue Jan 1 21:27:14 2019 +0800
Committer: Duo Zhang <zh...@apache.org>
Committed: Fri Jan 4 14:34:02 2019 +0800
----------------------------------------------------------------------
.../hbase/client/AsyncRegionServerAdmin.java | 14 +++++--
.../hbase/protobuf/ReplicationProtbufUtil.java | 35 +++++++++--------
.../HBaseInterClusterReplicationEndpoint.java | 31 +++++++--------
.../regionserver/ReplicationSinkManager.java | 40 +++++++-------------
.../replication/SyncReplicationTestBase.java | 12 +++---
.../TestReplicationSinkManager.java | 21 +++++-----
6 files changed, 74 insertions(+), 79 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/hbase/blob/04e6909a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionServerAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionServerAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionServerAdmin.java
index 9accd89..b9141a9 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionServerAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionServerAdmin.java
@@ -19,6 +19,7 @@ package org.apache.hadoop.hbase.client;
import java.io.IOException;
import java.util.concurrent.CompletableFuture;
+import org.apache.hadoop.hbase.CellScanner;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.ipc.HBaseRpcController;
import org.apache.yetus.audience.InterfaceAudience;
@@ -94,9 +95,9 @@ public class AsyncRegionServerAdmin {
void call(AdminService.Interface stub, HBaseRpcController controller, RpcCallback<RESP> done);
}
- private <RESP> CompletableFuture<RESP> call(RpcCall<RESP> rpcCall) {
+ private <RESP> CompletableFuture<RESP> call(RpcCall<RESP> rpcCall, CellScanner cellScanner) {
CompletableFuture<RESP> future = new CompletableFuture<>();
- HBaseRpcController controller = conn.rpcControllerFactory.newController();
+ HBaseRpcController controller = conn.rpcControllerFactory.newController(cellScanner);
try {
rpcCall.call(conn.getAdminStub(server), controller, new RpcCallback<RESP>() {
@@ -115,6 +116,10 @@ public class AsyncRegionServerAdmin {
return future;
}
+ private <RESP> CompletableFuture<RESP> call(RpcCall<RESP> rpcCall) {
+ return call(rpcCall, null);
+ }
+
public CompletableFuture<GetRegionInfoResponse> getRegionInfo(GetRegionInfoRequest request) {
return call((stub, controller, done) -> stub.getRegionInfo(controller, request, done));
}
@@ -154,8 +159,9 @@ public class AsyncRegionServerAdmin {
}
public CompletableFuture<ReplicateWALEntryResponse> replicateWALEntry(
- ReplicateWALEntryRequest request) {
- return call((stub, controller, done) -> stub.replicateWALEntry(controller, request, done));
+ ReplicateWALEntryRequest request, CellScanner cellScanner) {
+ return call((stub, controller, done) -> stub.replicateWALEntry(controller, request, done),
+ cellScanner);
}
public CompletableFuture<ReplicateWALEntryResponse> replay(ReplicateWALEntryRequest request) {
http://git-wip-us.apache.org/repos/asf/hbase/blob/04e6909a/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ReplicationProtbufUtil.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ReplicationProtbufUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ReplicationProtbufUtil.java
index c1b3911..74fad26 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ReplicationProtbufUtil.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ReplicationProtbufUtil.java
@@ -20,51 +20,54 @@ package org.apache.hadoop.hbase.protobuf;
import java.io.IOException;
+import java.io.InterruptedIOException;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
+import java.util.concurrent.ExecutionException;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellScanner;
import org.apache.hadoop.hbase.PrivateCellUtil;
+import org.apache.hadoop.hbase.client.AsyncRegionServerAdmin;
import org.apache.hadoop.hbase.io.SizedCellScanner;
-import org.apache.hadoop.hbase.ipc.HBaseRpcController;
-import org.apache.hadoop.hbase.ipc.HBaseRpcControllerImpl;
import org.apache.hadoop.hbase.regionserver.wal.WALCellCodec;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.wal.WAL.Entry;
import org.apache.hadoop.hbase.wal.WALEdit;
import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hbase.thirdparty.com.google.common.base.Throwables;
import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations;
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.WALProtos;
@InterfaceAudience.Private
public class ReplicationProtbufUtil {
+
/**
- * A helper to replicate a list of WAL entries using admin protocol.
- * @param admin Admin service
+ * A helper to replicate a list of WAL entries using region server admin
+ * @param admin the region server admin
* @param entries Array of WAL entries to be replicated
* @param replicationClusterId Id which will uniquely identify source cluster FS client
* configurations in the replication configuration directory
* @param sourceBaseNamespaceDir Path to source cluster base namespace directory
* @param sourceHFileArchiveDir Path to the source cluster hfile archive directory
- * @throws java.io.IOException
*/
- public static void replicateWALEntry(final AdminService.BlockingInterface admin,
- final Entry[] entries, String replicationClusterId, Path sourceBaseNamespaceDir,
- Path sourceHFileArchiveDir) throws IOException {
- Pair<AdminProtos.ReplicateWALEntryRequest, CellScanner> p =
- buildReplicateWALEntryRequest(entries, null, replicationClusterId, sourceBaseNamespaceDir,
- sourceHFileArchiveDir);
- HBaseRpcController controller = new HBaseRpcControllerImpl(p.getSecond());
+ public static void replicateWALEntry(AsyncRegionServerAdmin admin, Entry[] entries,
+ String replicationClusterId, Path sourceBaseNamespaceDir, Path sourceHFileArchiveDir)
+ throws IOException {
+ Pair<AdminProtos.ReplicateWALEntryRequest, CellScanner> p = buildReplicateWALEntryRequest(
+ entries, null, replicationClusterId, sourceBaseNamespaceDir, sourceHFileArchiveDir);
try {
- admin.replicateWALEntry(controller, p.getFirst());
- } catch (org.apache.hbase.thirdparty.com.google.protobuf.ServiceException e) {
- throw ProtobufUtil.getServiceException(e);
+ admin.replicateWALEntry(p.getFirst(), p.getSecond()).get();
+ } catch (InterruptedException e) {
+ throw (IOException) new InterruptedIOException().initCause(e);
+ } catch (ExecutionException e) {
+ Throwable cause = e.getCause();
+ Throwables.propagateIfPossible(cause, IOException.class);
+ throw new IOException(e);
}
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/04e6909a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/HBaseInterClusterReplicationEndpoint.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/HBaseInterClusterReplicationEndpoint.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/HBaseInterClusterReplicationEndpoint.java
index 7db53aa..0359096 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/HBaseInterClusterReplicationEndpoint.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/HBaseInterClusterReplicationEndpoint.java
@@ -39,7 +39,6 @@ import java.util.regex.Matcher;
import java.util.regex.Pattern;
import java.util.stream.Collectors;
import java.util.stream.Stream;
-
import org.apache.commons.lang3.StringUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
@@ -48,13 +47,16 @@ import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableNotFoundException;
-import org.apache.hadoop.hbase.client.ClusterConnection;
+import org.apache.hadoop.hbase.client.AsyncClusterConnection;
+import org.apache.hadoop.hbase.client.AsyncRegionServerAdmin;
+import org.apache.hadoop.hbase.client.ClusterConnectionFactory;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.ipc.RpcServer;
import org.apache.hadoop.hbase.protobuf.ReplicationProtbufUtil;
import org.apache.hadoop.hbase.replication.HBaseReplicationEndpoint;
import org.apache.hadoop.hbase.replication.regionserver.ReplicationSinkManager.SinkPeer;
+import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.wal.WAL.Entry;
@@ -65,8 +67,6 @@ import org.slf4j.LoggerFactory;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface;
-
/**
* A {@link org.apache.hadoop.hbase.replication.ReplicationEndpoint}
* implementation for replicating to another HBase cluster.
@@ -85,8 +85,7 @@ public class HBaseInterClusterReplicationEndpoint extends HBaseReplicationEndpoi
private static final long DEFAULT_MAX_TERMINATION_WAIT_MULTIPLIER = 2;
- private ClusterConnection conn;
- private Configuration localConf;
+ private AsyncClusterConnection conn;
private Configuration conf;
// How long should we sleep for each retry
private long sleepForRetries;
@@ -117,7 +116,6 @@ public class HBaseInterClusterReplicationEndpoint extends HBaseReplicationEndpoi
public void init(Context context) throws IOException {
super.init(context);
this.conf = HBaseConfiguration.create(ctx.getConfiguration());
- this.localConf = HBaseConfiguration.create(ctx.getLocalConfiguration());
decorateConf();
this.maxRetriesMultiplier = this.conf.getInt("replication.source.maxretriesmultiplier", 300);
this.socketTimeoutMultiplier = this.conf.getInt("replication.source.socketTimeoutMultiplier",
@@ -132,12 +130,13 @@ public class HBaseInterClusterReplicationEndpoint extends HBaseReplicationEndpoi
// TODO: This connection is replication specific or we should make it particular to
// replication and make replication specific settings such as compression or codec to use
// passing Cells.
- this.conn = (ClusterConnection) ConnectionFactory.createConnection(this.conf);
+ this.conn =
+ ClusterConnectionFactory.createAsyncClusterConnection(conf, null, User.getCurrent());
this.sleepForRetries =
this.conf.getLong("replication.source.sleepforretries", 1000);
this.metrics = context.getMetrics();
// ReplicationQueueInfo parses the peerId out of the znode for us
- this.replicationSinkMgr = new ReplicationSinkManager(conn, ctx.getPeerId(), this, this.conf);
+ this.replicationSinkMgr = new ReplicationSinkManager(conn, this, this.conf);
// per sink thread pool
this.maxThreads = this.conf.getInt(HConstants.REPLICATION_SOURCE_MAXTHREADS_KEY,
HConstants.REPLICATION_SOURCE_MAXTHREADS_DEFAULT);
@@ -284,9 +283,10 @@ public class HBaseInterClusterReplicationEndpoint extends HBaseReplicationEndpoi
}
private void reconnectToPeerCluster() {
- ClusterConnection connection = null;
+ AsyncClusterConnection connection = null;
try {
- connection = (ClusterConnection) ConnectionFactory.createConnection(this.conf);
+ connection =
+ ClusterConnectionFactory.createAsyncClusterConnection(conf, null, User.getCurrent());
} catch (IOException ioe) {
LOG.warn("Failed to create connection for peer cluster", ioe);
}
@@ -367,7 +367,7 @@ public class HBaseInterClusterReplicationEndpoint extends HBaseReplicationEndpoi
}
continue;
}
- if (this.conn == null || this.conn.isClosed()) {
+ if (this.conn == null) {
reconnectToPeerCluster();
}
try {
@@ -480,10 +480,11 @@ public class HBaseInterClusterReplicationEndpoint extends HBaseReplicationEndpoi
entriesHashCode, entries.size(), size, replicationClusterId);
}
sinkPeer = replicationSinkMgr.getReplicationSink();
- BlockingInterface rrs = sinkPeer.getRegionServer();
+ AsyncRegionServerAdmin rsAdmin = sinkPeer.getRegionServer();
try {
- ReplicationProtbufUtil.replicateWALEntry(rrs, entries.toArray(new Entry[entries.size()]),
- replicationClusterId, baseNamespaceDir, hfileArchiveDir);
+ ReplicationProtbufUtil.replicateWALEntry(rsAdmin,
+ entries.toArray(new Entry[entries.size()]), replicationClusterId, baseNamespaceDir,
+ hfileArchiveDir);
LOG.trace("Completed replicating batch {}", entriesHashCode);
} catch (IOException e) {
LOG.trace("Failed replicating batch {}", entriesHashCode, e);
http://git-wip-us.apache.org/repos/asf/hbase/blob/04e6909a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSinkManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSinkManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSinkManager.java
index 3cd7884..21b07ac 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSinkManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSinkManager.java
@@ -21,11 +21,11 @@ import java.io.IOException;
import java.util.Collections;
import java.util.List;
import java.util.Map;
-import java.util.Random;
+import java.util.concurrent.ThreadLocalRandom;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.client.ClusterConnection;
-import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.AsyncClusterConnection;
+import org.apache.hadoop.hbase.client.AsyncRegionServerAdmin;
import org.apache.hadoop.hbase.replication.HBaseReplicationEndpoint;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
@@ -35,8 +35,6 @@ import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesti
import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
import org.apache.hbase.thirdparty.com.google.common.collect.Maps;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService;
-
/**
* Maintains a collection of peers to replicate to, and randomly selects a
* single peer to replicate to per set of data to replicate. Also handles
@@ -61,9 +59,7 @@ public class ReplicationSinkManager {
static final float DEFAULT_REPLICATION_SOURCE_RATIO = 0.5f;
- private final Connection conn;
-
- private final String peerClusterId;
+ private final AsyncClusterConnection conn;
private final HBaseReplicationEndpoint endpoint;
@@ -77,8 +73,6 @@ public class ReplicationSinkManager {
// replication sinks is refreshed
private final int badSinkThreshold;
- private final Random random;
-
// A timestamp of the last time the list of replication peers changed
private long lastUpdateToPeers;
@@ -88,26 +82,22 @@ public class ReplicationSinkManager {
/**
* Instantiate for a single replication peer cluster.
* @param conn connection to the peer cluster
- * @param peerClusterId identifier of the peer cluster
* @param endpoint replication endpoint for inter cluster replication
* @param conf HBase configuration, used for determining replication source ratio and bad peer
* threshold
*/
- public ReplicationSinkManager(ClusterConnection conn, String peerClusterId,
- HBaseReplicationEndpoint endpoint, Configuration conf) {
+ public ReplicationSinkManager(AsyncClusterConnection conn, HBaseReplicationEndpoint endpoint,
+ Configuration conf) {
this.conn = conn;
- this.peerClusterId = peerClusterId;
this.endpoint = endpoint;
this.badReportCounts = Maps.newHashMap();
this.ratio = conf.getFloat("replication.source.ratio", DEFAULT_REPLICATION_SOURCE_RATIO);
- this.badSinkThreshold = conf.getInt("replication.bad.sink.threshold",
- DEFAULT_BAD_SINK_THRESHOLD);
- this.random = new Random();
+ this.badSinkThreshold =
+ conf.getInt("replication.bad.sink.threshold", DEFAULT_BAD_SINK_THRESHOLD);
}
/**
* Get a randomly-chosen replication sink to replicate to.
- *
* @return a replication sink to replicate to
*/
public synchronized SinkPeer getReplicationSink() throws IOException {
@@ -119,8 +109,8 @@ public class ReplicationSinkManager {
if (sinks.isEmpty()) {
throw new IOException("No replication sinks are available");
}
- ServerName serverName = sinks.get(random.nextInt(sinks.size()));
- return new SinkPeer(serverName, ((ClusterConnection) conn).getAdmin(serverName));
+ ServerName serverName = sinks.get(ThreadLocalRandom.current().nextInt(sinks.size()));
+ return new SinkPeer(serverName, conn.getRegionServerAdmin(serverName));
}
/**
@@ -160,7 +150,7 @@ public class ReplicationSinkManager {
*/
public synchronized void chooseSinks() {
List<ServerName> slaveAddresses = endpoint.getRegionServers();
- Collections.shuffle(slaveAddresses, random);
+ Collections.shuffle(slaveAddresses, ThreadLocalRandom.current());
int numSinks = (int) Math.ceil(slaveAddresses.size() * ratio);
sinks = slaveAddresses.subList(0, numSinks);
lastUpdateToPeers = System.currentTimeMillis();
@@ -182,9 +172,9 @@ public class ReplicationSinkManager {
*/
public static class SinkPeer {
private ServerName serverName;
- private AdminService.BlockingInterface regionServer;
+ private AsyncRegionServerAdmin regionServer;
- public SinkPeer(ServerName serverName, AdminService.BlockingInterface regionServer) {
+ public SinkPeer(ServerName serverName, AsyncRegionServerAdmin regionServer) {
this.serverName = serverName;
this.regionServer = regionServer;
}
@@ -193,10 +183,8 @@ public class ReplicationSinkManager {
return serverName;
}
- public AdminService.BlockingInterface getRegionServer() {
+ public AsyncRegionServerAdmin getRegionServer() {
return regionServer;
}
-
}
-
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/04e6909a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/SyncReplicationTestBase.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/SyncReplicationTestBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/SyncReplicationTestBase.java
index f373590..e0d112d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/SyncReplicationTestBase.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/SyncReplicationTestBase.java
@@ -36,7 +36,7 @@ import org.apache.hadoop.hbase.StartMiniClusterOption;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.Waiter.ExplainingPredicate;
import org.apache.hadoop.hbase.client.Admin;
-import org.apache.hadoop.hbase.client.ClusterConnection;
+import org.apache.hadoop.hbase.client.AsyncClusterConnection;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.Put;
@@ -250,19 +250,19 @@ public class SyncReplicationTestBase {
protected final void verifyReplicationRequestRejection(HBaseTestingUtility utility,
boolean expectedRejection) throws Exception {
HRegionServer regionServer = utility.getRSForFirstRegionInTable(TABLE_NAME);
- ClusterConnection connection = regionServer.getClusterConnection();
+ AsyncClusterConnection connection = regionServer.getAsyncClusterConnection();
Entry[] entries = new Entry[10];
for (int i = 0; i < entries.length; i++) {
entries[i] =
new Entry(new WALKeyImpl(HConstants.EMPTY_BYTE_ARRAY, TABLE_NAME, 0), new WALEdit());
}
if (!expectedRejection) {
- ReplicationProtbufUtil.replicateWALEntry(connection.getAdmin(regionServer.getServerName()),
- entries, null, null, null);
+ ReplicationProtbufUtil.replicateWALEntry(
+ connection.getRegionServerAdmin(regionServer.getServerName()), entries, null, null, null);
} else {
try {
- ReplicationProtbufUtil.replicateWALEntry(connection.getAdmin(regionServer.getServerName()),
- entries, null, null, null);
+ ReplicationProtbufUtil.replicateWALEntry(
+ connection.getRegionServerAdmin(regionServer.getServerName()), entries, null, null, null);
fail("Should throw IOException when sync-replication state is in A or DA");
} catch (DoNotRetryIOException e) {
assertTrue(e.getMessage().contains("Reject to apply to sink cluster"));
http://git-wip-us.apache.org/repos/asf/hbase/blob/04e6909a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSinkManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSinkManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSinkManager.java
index 39dabb4..60afd40 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSinkManager.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSinkManager.java
@@ -25,7 +25,8 @@ import java.util.List;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseClassTestRule;
import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.client.ClusterConnection;
+import org.apache.hadoop.hbase.client.AsyncClusterConnection;
+import org.apache.hadoop.hbase.client.AsyncRegionServerAdmin;
import org.apache.hadoop.hbase.replication.HBaseReplicationEndpoint;
import org.apache.hadoop.hbase.replication.regionserver.ReplicationSinkManager.SinkPeer;
import org.apache.hadoop.hbase.testclassification.ReplicationTests;
@@ -37,8 +38,6 @@ import org.junit.experimental.categories.Category;
import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService;
-
@Category({ReplicationTests.class, SmallTests.class})
public class TestReplicationSinkManager {
@@ -46,16 +45,14 @@ public class TestReplicationSinkManager {
public static final HBaseClassTestRule CLASS_RULE =
HBaseClassTestRule.forClass(TestReplicationSinkManager.class);
- private static final String PEER_CLUSTER_ID = "PEER_CLUSTER_ID";
-
private HBaseReplicationEndpoint replicationEndpoint;
private ReplicationSinkManager sinkManager;
@Before
public void setUp() {
replicationEndpoint = mock(HBaseReplicationEndpoint.class);
- sinkManager = new ReplicationSinkManager(mock(ClusterConnection.class),
- PEER_CLUSTER_ID, replicationEndpoint, new Configuration());
+ sinkManager = new ReplicationSinkManager(mock(AsyncClusterConnection.class),
+ replicationEndpoint, new Configuration());
}
@Test
@@ -100,7 +97,7 @@ public class TestReplicationSinkManager {
// Sanity check
assertEquals(1, sinkManager.getNumSinks());
- SinkPeer sinkPeer = new SinkPeer(serverNameA, mock(AdminService.BlockingInterface.class));
+ SinkPeer sinkPeer = new SinkPeer(serverNameA, mock(AsyncRegionServerAdmin.class));
sinkManager.reportBadSink(sinkPeer);
@@ -131,7 +128,7 @@ public class TestReplicationSinkManager {
ServerName serverName = sinkManager.getSinksForTesting().get(0);
- SinkPeer sinkPeer = new SinkPeer(serverName, mock(AdminService.BlockingInterface.class));
+ SinkPeer sinkPeer = new SinkPeer(serverName, mock(AsyncRegionServerAdmin.class));
sinkManager.reportSinkSuccess(sinkPeer); // has no effect, counter does not go negative
for (int i = 0; i <= ReplicationSinkManager.DEFAULT_BAD_SINK_THRESHOLD; i++) {
@@ -147,7 +144,7 @@ public class TestReplicationSinkManager {
//
serverName = sinkManager.getSinksForTesting().get(0);
- sinkPeer = new SinkPeer(serverName, mock(AdminService.BlockingInterface.class));
+ sinkPeer = new SinkPeer(serverName, mock(AsyncRegionServerAdmin.class));
for (int i = 0; i <= ReplicationSinkManager.DEFAULT_BAD_SINK_THRESHOLD-1; i++) {
sinkManager.reportBadSink(sinkPeer);
}
@@ -188,8 +185,8 @@ public class TestReplicationSinkManager {
ServerName serverNameA = sinkList.get(0);
ServerName serverNameB = sinkList.get(1);
- SinkPeer sinkPeerA = new SinkPeer(serverNameA, mock(AdminService.BlockingInterface.class));
- SinkPeer sinkPeerB = new SinkPeer(serverNameB, mock(AdminService.BlockingInterface.class));
+ SinkPeer sinkPeerA = new SinkPeer(serverNameA, mock(AsyncRegionServerAdmin.class));
+ SinkPeer sinkPeerB = new SinkPeer(serverNameB, mock(AsyncRegionServerAdmin.class));
for (int i = 0; i <= ReplicationSinkManager.DEFAULT_BAD_SINK_THRESHOLD; i++) {
sinkManager.reportBadSink(sinkPeerA);