HBASE-18608 AsyncConnection should return AsyncAdmin interface instead of the implemenation

This commit is contained in:
Guanghao Zhang 2017-08-16 18:00:53 +08:00
parent a17ed0356f
commit 092dc6de84
6 changed files with 26 additions and 26 deletions

View File

@ -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();
}

View File

@ -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;
}

View File

@ -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);
}

View File

@ -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);
}

View File

@ -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;

View File

@ -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());
}