You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by bu...@apache.org on 2017/08/20 21:30:36 UTC
[39/50] [abbrv] hbase git commit: HBASE-18608 AsyncConnection should
return AsyncAdmin interface instead of the implemenation
HBASE-18608 AsyncConnection should return AsyncAdmin interface instead of the implemenation
Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/092dc6de
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/092dc6de
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/092dc6de
Branch: refs/heads/HBASE-18467
Commit: 092dc6de8483eea1b4e0d960cf22e65359379da1
Parents: a17ed03
Author: Guanghao Zhang <zg...@apache.org>
Authored: Wed Aug 16 18:00:53 2017 +0800
Committer: Guanghao Zhang <zg...@apache.org>
Committed: Thu Aug 17 09:47:39 2017 +0800
----------------------------------------------------------------------
.../hadoop/hbase/client/AsyncAdminBuilder.java | 16 ++++++++--------
.../hadoop/hbase/client/AsyncAdminBuilderBase.java | 12 ++++++------
.../apache/hadoop/hbase/client/AsyncConnection.java | 4 ++--
.../hadoop/hbase/client/AsyncConnectionImpl.java | 12 ++++++------
.../hadoop/hbase/client/RawAsyncHBaseAdmin.java | 2 +-
.../hadoop/hbase/client/TestAsyncAdminBuilder.java | 6 +++---
6 files changed, 26 insertions(+), 26 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/hbase/blob/092dc6de/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdminBuilder.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdminBuilder.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdminBuilder.java
index d706949..fb0aefd 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdminBuilder.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdminBuilder.java
@@ -29,7 +29,7 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
* create a new AsyncAdmin instance.
*/
@InterfaceAudience.Public
-public interface AsyncAdminBuilder<T extends AsyncAdmin> {
+public interface AsyncAdminBuilder {
/**
* Set timeout for a whole admin operation. Operation timeout and max attempt times(or max retry
@@ -39,7 +39,7 @@ public interface AsyncAdminBuilder<T extends AsyncAdmin> {
* @param unit
* @return this for invocation chaining
*/
- AsyncAdminBuilder<T> setOperationTimeout(long timeout, TimeUnit unit);
+ AsyncAdminBuilder setOperationTimeout(long timeout, TimeUnit unit);
/**
* Set timeout for each rpc request.
@@ -47,7 +47,7 @@ public interface AsyncAdminBuilder<T extends AsyncAdmin> {
* @param unit
* @return this for invocation chaining
*/
- AsyncAdminBuilder<T> setRpcTimeout(long timeout, TimeUnit unit);
+ AsyncAdminBuilder setRpcTimeout(long timeout, TimeUnit unit);
/**
* Set the base pause time for retrying. We use an exponential policy to generate sleep time when
@@ -56,7 +56,7 @@ public interface AsyncAdminBuilder<T extends AsyncAdmin> {
* @param unit
* @return this for invocation chaining
*/
- AsyncAdminBuilder<T> setRetryPause(long timeout, TimeUnit unit);
+ AsyncAdminBuilder setRetryPause(long timeout, TimeUnit unit);
/**
* Set the max retry times for an admin operation. Usually it is the max attempt times minus 1.
@@ -65,7 +65,7 @@ public interface AsyncAdminBuilder<T extends AsyncAdmin> {
* @param maxRetries
* @return this for invocation chaining
*/
- default AsyncAdminBuilder<T> setMaxRetries(int maxRetries) {
+ default AsyncAdminBuilder setMaxRetries(int maxRetries) {
return setMaxAttempts(retries2Attempts(maxRetries));
}
@@ -76,18 +76,18 @@ public interface AsyncAdminBuilder<T extends AsyncAdmin> {
* @param maxAttempts
* @return this for invocation chaining
*/
- AsyncAdminBuilder<T> setMaxAttempts(int maxAttempts);
+ AsyncAdminBuilder setMaxAttempts(int maxAttempts);
/**
* Set the number of retries that are allowed before we start to log.
* @param startLogErrorsCnt
* @return this for invocation chaining
*/
- AsyncAdminBuilder<T> setStartLogErrorsCnt(int startLogErrorsCnt);
+ AsyncAdminBuilder setStartLogErrorsCnt(int startLogErrorsCnt);
/**
* Create a {@link AsyncAdmin} instance.
* @return a {@link AsyncAdmin} instance
*/
- T build();
+ AsyncAdmin build();
}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/hbase/blob/092dc6de/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdminBuilderBase.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdminBuilderBase.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdminBuilderBase.java
index 013e8d7..77ff88d 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdminBuilderBase.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdminBuilderBase.java
@@ -25,7 +25,7 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
* Base class for all asynchronous admin builders.
*/
@InterfaceAudience.Private
-abstract class AsyncAdminBuilderBase<T extends AsyncAdmin> implements AsyncAdminBuilder<T> {
+abstract class AsyncAdminBuilderBase implements AsyncAdminBuilder {
protected long rpcTimeoutNs;
@@ -46,31 +46,31 @@ abstract class AsyncAdminBuilderBase<T extends AsyncAdmin> implements AsyncAdmin
}
@Override
- public AsyncAdminBuilder<T> setOperationTimeout(long timeout, TimeUnit unit) {
+ public AsyncAdminBuilder setOperationTimeout(long timeout, TimeUnit unit) {
this.operationTimeoutNs = unit.toNanos(timeout);
return this;
}
@Override
- public AsyncAdminBuilder<T> setRpcTimeout(long timeout, TimeUnit unit) {
+ public AsyncAdminBuilder setRpcTimeout(long timeout, TimeUnit unit) {
this.rpcTimeoutNs = unit.toNanos(timeout);
return this;
}
@Override
- public AsyncAdminBuilder<T> setRetryPause(long timeout, TimeUnit unit) {
+ public AsyncAdminBuilder setRetryPause(long timeout, TimeUnit unit) {
this.pauseNs = unit.toNanos(timeout);
return this;
}
@Override
- public AsyncAdminBuilder<T> setMaxAttempts(int maxAttempts) {
+ public AsyncAdminBuilder setMaxAttempts(int maxAttempts) {
this.maxAttempts = maxAttempts;
return this;
}
@Override
- public AsyncAdminBuilder<T> setStartLogErrorsCnt(int startLogErrorsCnt) {
+ public AsyncAdminBuilder setStartLogErrorsCnt(int startLogErrorsCnt) {
this.startLogErrorsCnt = startLogErrorsCnt;
return this;
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/092dc6de/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnection.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnection.java
index 04ef78e..24907ba 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnection.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnection.java
@@ -117,7 +117,7 @@ public interface AsyncConnection extends Closeable {
* framework's callback thread, so typically you should not do any time consuming work inside
* these methods.
*/
- AsyncAdminBuilder<RawAsyncHBaseAdmin> getAdminBuilder();
+ AsyncAdminBuilder getAdminBuilder();
/**
* Retrieve an {@link AsyncAdmin} implementation to administer an HBase cluster.
@@ -135,5 +135,5 @@ public interface AsyncConnection extends Closeable {
* Returns an {@link AsyncAdminBuilder} for creating {@link AsyncAdmin}.
* @param pool the thread pool to use for executing callback
*/
- AsyncAdminBuilder<AsyncHBaseAdmin> getAdminBuilder(ExecutorService pool);
+ AsyncAdminBuilder getAdminBuilder(ExecutorService pool);
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/092dc6de/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 9913af9..d8f051f 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
@@ -278,20 +278,20 @@ class AsyncConnectionImpl implements AsyncConnection {
}
@Override
- public AsyncAdminBuilder<RawAsyncHBaseAdmin> getAdminBuilder() {
- return new AsyncAdminBuilderBase<RawAsyncHBaseAdmin>(connConf) {
+ public AsyncAdminBuilder getAdminBuilder() {
+ return new AsyncAdminBuilderBase(connConf) {
@Override
- public RawAsyncHBaseAdmin build() {
+ public AsyncAdmin build() {
return new RawAsyncHBaseAdmin(AsyncConnectionImpl.this, this);
}
};
}
@Override
- public AsyncAdminBuilder<AsyncHBaseAdmin> getAdminBuilder(ExecutorService pool) {
- return new AsyncAdminBuilderBase<AsyncHBaseAdmin>(connConf) {
+ public AsyncAdminBuilder getAdminBuilder(ExecutorService pool) {
+ return new AsyncAdminBuilderBase(connConf) {
@Override
- public AsyncHBaseAdmin build() {
+ public AsyncAdmin build() {
RawAsyncHBaseAdmin rawAdmin = new RawAsyncHBaseAdmin(AsyncConnectionImpl.this, this);
return new AsyncHBaseAdmin(rawAdmin, pool);
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/092dc6de/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 0070e0e..ba68a96 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
@@ -278,7 +278,7 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
private final NonceGenerator ng;
- RawAsyncHBaseAdmin(AsyncConnectionImpl connection, AsyncAdminBuilderBase<?> builder) {
+ RawAsyncHBaseAdmin(AsyncConnectionImpl connection, AsyncAdminBuilderBase builder) {
this.connection = connection;
this.metaTable = connection.getRawTable(META_TABLE_NAME);
this.rpcTimeoutNs = builder.rpcTimeoutNs;
http://git-wip-us.apache.org/repos/asf/hbase/blob/092dc6de/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminBuilder.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminBuilder.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminBuilder.java
index ea25ee4..1f7346b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminBuilder.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminBuilder.java
@@ -59,13 +59,13 @@ public class TestAsyncAdminBuilder {
private static AsyncConnection ASYNC_CONN;
@Parameter
- public Supplier<AsyncAdminBuilder<?>> getAdminBuilder;
+ public Supplier<AsyncAdminBuilder> getAdminBuilder;
- private static AsyncAdminBuilder<RawAsyncHBaseAdmin> getRawAsyncAdminBuilder() {
+ private static AsyncAdminBuilder getRawAsyncAdminBuilder() {
return ASYNC_CONN.getAdminBuilder();
}
- private static AsyncAdminBuilder<AsyncHBaseAdmin> getAsyncAdminBuilder() {
+ private static AsyncAdminBuilder getAsyncAdminBuilder() {
return ASYNC_CONN.getAdminBuilder(ForkJoinPool.commonPool());
}