From 2bb5e6c60cc13c0cb88e77fc2ba023d1e519178d Mon Sep 17 00:00:00 2001 From: Jurriaan Mous Date: Wed, 27 May 2015 13:18:28 +0200 Subject: [PATCH] HBASE-13784 --- .../hbase/client/AbstractRegionServerCallable.java | 157 ++++ .../hadoop/hbase/client/AsyncClientScanner.java | 822 +++++++++++++++++++++ .../client/AsyncClientSmallReversedScanner.java | 243 ++++++ .../hbase/client/AsyncClientSmallScanner.java | 290 ++++++++ .../hbase/client/AsyncRegionServerCallable.java | 72 ++ .../hadoop/hbase/client/AsyncResultScanner.java | 34 + .../hadoop/hbase/client/AsyncRetryingCallable.java | 45 ++ .../hbase/client/AsyncReversedClientScanner.java | 170 +++++ .../org/apache/hadoop/hbase/client/AsyncTable.java | 252 +++++++ .../apache/hadoop/hbase/client/AsyncTableImpl.java | 770 +++++++++++++++++++ .../hadoop/hbase/client/BufferedMutatorImpl.java | 2 +- .../hbase/client/ClientAsyncPrefetchScanner.java | 6 +- .../apache/hadoop/hbase/client/ClientScanner.java | 26 +- .../hadoop/hbase/client/ClientSimpleScanner.java | 5 +- .../hbase/client/ClientSmallReversedScanner.java | 26 +- .../hadoop/hbase/client/ClientSmallScanner.java | 98 +-- .../hadoop/hbase/client/ClusterConnection.java | 20 +- .../org/apache/hadoop/hbase/client/Connection.java | 19 + .../hadoop/hbase/client/ConnectionAdapter.java | 18 + .../hbase/client/ConnectionImplementation.java | 49 +- .../hadoop/hbase/client/ConnectionUtils.java | 1 + .../apache/hadoop/hbase/client/DelayedPromise.java | 63 ++ .../apache/hadoop/hbase/client/FailedFuture.java | 74 ++ .../hbase/client/FastFailInterceptorContext.java | 2 +- .../org/apache/hadoop/hbase/client/Future.java | 29 + .../org/apache/hadoop/hbase/client/HTable.java | 27 +- .../hadoop/hbase/client/KeyedPromiseKeeper.java | 105 +++ .../client/NoOpRetryingInterceptorContext.java | 2 +- .../apache/hadoop/hbase/client/PromiseKeeper.java | 108 +++ .../hadoop/hbase/client/ProtoRetryingCallable.java | 62 ++ .../hadoop/hbase/client/RegionServerCallable.java | 116 +-- .../hbase/client/ResponseFutureListener.java | 30 + .../hadoop/hbase/client/RetryingCallable.java | 34 +- .../client/RetryingCallerInterceptorContext.java | 2 +- .../hadoop/hbase/client/RetryingPromise.java | 112 +++ .../hadoop/hbase/client/ReversedClientScanner.java | 32 +- .../hbase/client/ReversedScannerCallable.java | 39 +- .../hadoop/hbase/client/RpcRetryingCaller.java | 22 +- .../hadoop/hbase/client/RpcRetryingCallerImpl.java | 105 ++- .../client/RpcRetryingCallerWithReadReplicas.java | 247 ++++--- .../hadoop/hbase/client/ScannerCallable.java | 367 +++++---- .../hbase/client/ScannerCallableWithReplicas.java | 296 ++------ .../hbase/client/ScannerReplicaPromiseKeeper.java | 137 ++++ .../hadoop/hbase/client/SinglePromiseKeeper.java | 112 +++ .../client/StatsTrackingRpcRetryingCaller.java | 32 +- .../hadoop/hbase/client/SuccessfulFuture.java | 68 ++ .../org/apache/hadoop/hbase/client/TableImpl.java | 585 +++++++++++++++ .../hadoop/hbase/client/VoidPromiseKeeper.java | 104 +++ .../hbase/client/coprocessor/AsyncBatch.java | 76 ++ .../apache/hadoop/hbase/ipc/AbstractRpcClient.java | 34 +- .../org/apache/hadoop/hbase/ipc/AsyncCall.java | 80 +- .../hbase/ipc/AsyncCoprocessorRpcChannel.java | 88 +++ .../apache/hadoop/hbase/ipc/AsyncRpcChannel.java | 728 +----------------- .../hadoop/hbase/ipc/AsyncRpcChannelImpl.java | 758 +++++++++++++++++++ .../apache/hadoop/hbase/ipc/AsyncRpcClient.java | 134 ++-- .../hbase/ipc/AsyncServerResponseHandler.java | 4 +- .../hadoop/hbase/ipc/CoprocessorRpcChannel.java | 42 +- .../hadoop/hbase/ipc/IOExceptionConverter.java | 15 + .../hbase/ipc/MasterCoprocessorRpcChannel.java | 2 +- .../apache/hadoop/hbase/ipc/MessageConverter.java | 49 ++ .../java/org/apache/hadoop/hbase/ipc/Promise.java | 39 + .../hbase/ipc/RegionCoprocessorRpcChannel.java | 95 ++- .../ipc/RegionServerCoprocessorRpcChannel.java | 2 +- .../org/apache/hadoop/hbase/ipc/RpcClient.java | 67 +- .../org/apache/hadoop/hbase/ipc/RpcClientImpl.java | 203 ++++- .../hbase/ipc/SyncCoprocessorRpcChannel.java | 85 +++ .../apache/hadoop/hbase/protobuf/ProtobufUtil.java | 14 - .../hadoop/hbase/client/TestClientScanner.java | 106 ++- .../client/TestClientSmallReversedScanner.java | 38 +- .../hbase/client/TestClientSmallScanner.java | 27 +- .../hadoop/hbase/client/TestFromClientSide.java | 8 +- .../hbase/client/TestFromClientSideAsync.java | 593 +++++++++++++++ .../hadoop/hbase/client/TestReplicasClient.java | 14 +- 73 files changed, 7585 insertions(+), 1753 deletions(-) create mode 100644 hbase-client/src/main/java/org/apache/hadoop/hbase/client/AbstractRegionServerCallable.java create mode 100644 hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncClientScanner.java create mode 100644 hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncClientSmallReversedScanner.java create mode 100644 hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncClientSmallScanner.java create mode 100644 hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionServerCallable.java create mode 100644 hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncResultScanner.java create mode 100644 hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRetryingCallable.java create mode 100644 hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncReversedClientScanner.java create mode 100644 hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTable.java create mode 100644 hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableImpl.java create mode 100644 hbase-client/src/main/java/org/apache/hadoop/hbase/client/DelayedPromise.java create mode 100644 hbase-client/src/main/java/org/apache/hadoop/hbase/client/FailedFuture.java create mode 100644 hbase-client/src/main/java/org/apache/hadoop/hbase/client/Future.java create mode 100644 hbase-client/src/main/java/org/apache/hadoop/hbase/client/KeyedPromiseKeeper.java create mode 100644 hbase-client/src/main/java/org/apache/hadoop/hbase/client/PromiseKeeper.java create mode 100644 hbase-client/src/main/java/org/apache/hadoop/hbase/client/ProtoRetryingCallable.java create mode 100644 hbase-client/src/main/java/org/apache/hadoop/hbase/client/ResponseFutureListener.java create mode 100644 hbase-client/src/main/java/org/apache/hadoop/hbase/client/RetryingPromise.java create mode 100644 hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerReplicaPromiseKeeper.java create mode 100644 hbase-client/src/main/java/org/apache/hadoop/hbase/client/SinglePromiseKeeper.java create mode 100644 hbase-client/src/main/java/org/apache/hadoop/hbase/client/SuccessfulFuture.java create mode 100644 hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableImpl.java create mode 100644 hbase-client/src/main/java/org/apache/hadoop/hbase/client/VoidPromiseKeeper.java create mode 100644 hbase-client/src/main/java/org/apache/hadoop/hbase/client/coprocessor/AsyncBatch.java create mode 100644 hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AsyncCoprocessorRpcChannel.java create mode 100644 hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AsyncRpcChannelImpl.java create mode 100644 hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/IOExceptionConverter.java create mode 100644 hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/MessageConverter.java create mode 100644 hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/Promise.java create mode 100644 hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/SyncCoprocessorRpcChannel.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSideAsync.java diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AbstractRegionServerCallable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AbstractRegionServerCallable.java new file mode 100644 index 0000000..34970f9 --- /dev/null +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AbstractRegionServerCallable.java @@ -0,0 +1,157 @@ +/** + * + * 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.HRegionInfo; +import org.apache.hadoop.hbase.HRegionLocation; +import org.apache.hadoop.hbase.NotServingRegionException; +import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.exceptions.RegionMovedException; +import org.apache.hadoop.hbase.util.Bytes; + +import java.io.IOException; +import java.net.ConnectException; +import java.net.SocketTimeoutException; + +/** + * Implementations call a RegionServer. + * Passed to a {@link RpcRetryingCaller} so we retry on fail. + * TODO: this class is actually tied to one region, because most of the paths make use of + * the regioninfo part of location when building requests. The only reason it works for + * multi-region requests (e.g. batch) is that they happen to not use the region parts. + * This could be done cleaner (e.g. having a generic parameter and 2 derived classes, + * RegionCallable and actual RegionServerCallable with ServerName. + * @param the class that the ServerCallable handles + */ +@InterfaceAudience.Private +abstract class AbstractRegionServerCallable implements ProtoRetryingCallable { + protected final Connection connection; + protected final TableName tableName; + protected final byte[] row; + protected HRegionLocation location; + + protected final static int MIN_WAIT_DEAD_SERVER = 10000; + + /** + * @param connection Connection to use. + * @param tableName Table name to which row belongs. + * @param row The row we want in tableName. + */ + public AbstractRegionServerCallable(Connection connection, TableName tableName, byte[] row) { + this.connection = connection; + this.tableName = tableName; + this.row = row; + } + + /** + * @return {@link ClusterConnection} instance used by this Callable. + */ + ClusterConnection getConnection() { + return (ClusterConnection) this.connection; + } + + protected HRegionLocation getLocation() { + return this.location; + } + + protected void setLocation(final HRegionLocation location) { + this.location = location; + } + + public TableName getTableName() { + return this.tableName; + } + + public byte [] getRow() { + return this.row; + } + + @Override + public void throwable(Throwable t, boolean retrying) { + if (t instanceof SocketTimeoutException || + t instanceof ConnectException || + t instanceof RetriesExhaustedException || + (location != null && getConnection().isDeadServer(location.getServerName()))) { + // if thrown these exceptions, we clear all the cache entries that + // map to that slow/dead server; otherwise, let cache miss and ask + // hbase:meta again to find the new location + if (this.location != null) getConnection().clearCaches(location.getServerName()); + } else if (t instanceof RegionMovedException) { + getConnection().updateCachedLocations(tableName, row, t, location); + } else if (t instanceof NotServingRegionException && !retrying) { + // Purge cache entries for this specific region from hbase:meta cache + // since we don't call connect(true) when number of retries is 1. + getConnection().deleteCachedRegionLocation(location); + } + } + + @Override + public String getExceptionMessageAdditionalDetail() { + return "row '" + Bytes.toString(row) + "' on table '" + tableName + "' at " + location; + } + + @Override + public long sleep(long pause, int tries) { + // Tries hasn't been bumped up yet so we use "tries + 1" to get right pause time + long sleep = ConnectionUtils.getPauseTime(pause, tries + 1); + if (sleep < MIN_WAIT_DEAD_SERVER + && (location == null || getConnection().isDeadServer(location.getServerName()))) { + sleep = ConnectionUtils.addJitter(MIN_WAIT_DEAD_SERVER, 0.10f); + } + return sleep; + } + + /** + * @return the HRegionInfo for the current region + */ + public HRegionInfo getHRegionInfo() { + if (this.location == null) { + return null; + } + return this.location.getRegionInfo(); + } + + /** + * Prepare for connection to the server hosting region with row from tablename. Does lookup + * to find region location and hosting server. + * @param reload Set this to true if connection should re-find the region + * @throws IOException e + */ + @Override + public void prepare(final boolean reload) throws IOException { + try (RegionLocator regionLocator = connection.getRegionLocator(tableName)) { + this.location = regionLocator.getRegionLocation(row, reload); + } + if (this.location == null) { + throw new IOException("Failed to find location, tableName=" + tableName + + ", row=" + Bytes.toString(row) + ", reload=" + reload); + } + setClientByServiceName(this.location.getServerName()); + } + + /** + * Set the Rpc client for Client services + * @param serviceName to get client for + * @throws IOException When client could not be created + */ + abstract void setClientByServiceName(ServerName serviceName) throws IOException; +} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncClientScanner.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncClientScanner.java new file mode 100644 index 0000000..08b18e2 --- /dev/null +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncClientScanner.java @@ -0,0 +1,822 @@ +/** + * 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 com.google.common.annotations.VisibleForTesting; +import io.netty.util.concurrent.GenericFutureListener; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.DoNotRetryIOException; +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.HRegionInfo; +import org.apache.hadoop.hbase.NotServingRegionException; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.UnknownScannerException; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.client.metrics.ScanMetrics; +import org.apache.hadoop.hbase.exceptions.OutOfOrderScannerNextException; +import org.apache.hadoop.hbase.ipc.Promise; +import org.apache.hadoop.hbase.protobuf.ProtobufUtil; +import org.apache.hadoop.hbase.protobuf.generated.MapReduceProtos; +import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException; +import org.apache.hadoop.hbase.util.Bytes; + +import java.io.IOException; +import java.io.InterruptedIOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.LinkedList; +import java.util.List; +import java.util.concurrent.ExecutionException; + +/** + * Implements the scanner interface for the HBase client. + * If there are multiple regions in a table, this scanner will iterate + * through them all. + */ +@InterfaceAudience.Private +public class AsyncClientScanner implements AsyncResultScanner { + private static final Log LOG = LogFactory.getLog(AsyncClientScanner.class); + + private final ClusterConnection connection; + private final TableName tableName; + + // A byte array in which all elements are the max byte, and it is used to + // construct closest front row + static byte[] MAX_BYTE_ARRAY = Bytes.createMaxByteArray(9); + protected Scan scan; + protected boolean closed = false; + // Current region scanner is against. Gets cleared if current region goes + // wonky: e.g. if it splits on us. + protected HRegionInfo currentRegion = null; + protected ScannerCallableWithReplicas callable = null; + + /** + * A list of partial results that have been returned from the server. This list should only + * contain results if this scanner does not have enough partial results to form the complete + * result. + */ + protected final LinkedList partialResults = new LinkedList(); + + protected long lastNext; + // Keep lastResult returned successfully in case we have to reset scanner. + protected Result lastResult = null; + protected final long maxScannerResultSize; + + protected final int scannerTimeout; + protected boolean scanMetricsPublished = false; + protected RpcRetryingCaller caller; + protected Configuration conf; + //The timeout on the primary. Applicable if there are multiple replicas for a region + //In that case, we will only wait for this much timeout on the primary before going + //to the replicas and trying the same scan. Note that the retries will still happen + //on each replica and the first successful results will be taken. A timeout of 0 is + //disallowed. + protected final int primaryOperationTimeout; + private final int retries; + private byte[] partialResultsRow; + + protected final int batchSize; + + protected boolean shouldContinue = false; + private boolean retryAfterOutOfOrderException; + + protected ScanMetrics scanMetrics; + + /** + * Create a new ClientScanner for the specified table Note that the passed {@link Scan}'s start + * row maybe changed changed. + * @param conf The {@link Configuration} to use. + * @param scan {@link Scan} to use in this scanner + * @param tableName The table that we wish to scan + * @param connection Connection identifying the cluster + * @throws IOException + */ + public AsyncClientScanner(final Configuration conf, final Scan scan, final TableName tableName, + ClusterConnection connection, RpcRetryingCallerFactory rpcFactory, + int primaryOperationTimeout) throws IOException { + if (LOG.isTraceEnabled()) { + LOG.trace("Scan table=" + tableName + + ", startRow=" + Bytes.toStringBinary(scan.getStartRow())); + } + this.scan = scan; + this.tableName = tableName; + this.lastNext = System.currentTimeMillis(); + this.connection = connection; + this.primaryOperationTimeout = primaryOperationTimeout; + this.retries = conf.getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, + HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER); + if (scan.getMaxResultSize() > 0) { + this.maxScannerResultSize = scan.getMaxResultSize(); + } else { + this.maxScannerResultSize = conf.getLong( + HConstants.HBASE_CLIENT_SCANNER_MAX_RESULT_SIZE_KEY, + HConstants.DEFAULT_HBASE_CLIENT_SCANNER_MAX_RESULT_SIZE); + } + this.scannerTimeout = HBaseConfiguration.getInt(conf, + HConstants.HBASE_CLIENT_SCANNER_TIMEOUT_PERIOD, + HConstants.HBASE_REGIONSERVER_LEASE_PERIOD_KEY, + HConstants.DEFAULT_HBASE_CLIENT_SCANNER_TIMEOUT_PERIOD); + + // Use the caching from the Scan. If not set, use the default cache setting for this table. + if (this.scan.getCaching() > 0) { + this.batchSize = this.scan.getCaching(); + } else { + this.batchSize = conf.getInt(HConstants.HBASE_CLIENT_SCANNER_CACHING, + HConstants.DEFAULT_HBASE_CLIENT_SCANNER_CACHING); + } + + // check if application wants to collect scan metrics + initScanMetrics(scan); + + this.caller = rpcFactory. newCaller(); + + this.conf = conf; + initializeScannerInConstruction(); + } + + /** + * Check and initialize if application wants to collect scan metrics + */ + protected void initScanMetrics(Scan scan) { + // check if application wants to collect scan metrics + if (scan.isScanMetricsEnabled()) { + scanMetrics = new ScanMetrics(); + } + } + + /** + * Used internally accumulating metrics on scan. To + * enable collection of metrics on a Scanner, call {@link Scan#setScanMetricsEnabled(boolean)}. + * These metrics are cleared at key transition points. Metrics are accumulated in the + * {@link Scan} object itself. + * @see Scan#getScanMetrics() + * @return Returns the running {@link ScanMetrics} instance or null if scan metrics not enabled. + */ + public ScanMetrics getScanMetrics() { + return scanMetrics; + } + + protected void initializeScannerInConstruction() throws IOException{ + // initialize the scanner + nextScanner(this.batchSize, false); + } + + protected ClusterConnection getConnection() { + return this.connection; + } + + protected TableName getTable() { + return this.tableName; + } + + protected int getRetries() { + return this.retries; + } + + protected int getScannerTimeout() { + return this.scannerTimeout; + } + + protected Configuration getConf() { + return this.conf; + } + + protected Scan getScan() { + return scan; + } + + protected int getPrimaryOperationTimeout() { + return primaryOperationTimeout; + } + + protected long getTimestamp() { + return lastNext; + } + + @VisibleForTesting + protected long getMaxResultSize() { + return maxScannerResultSize; + } + + // returns true if the passed region endKey + protected boolean checkScanStopRow(final byte [] endKey) { + if (this.scan.getStopRow().length > 0) { + // there is a stop row, check to see if we are past it. + byte [] stopRow = scan.getStopRow(); + int cmp = Bytes.compareTo(stopRow, 0, stopRow.length, + endKey, 0, endKey.length); + if (cmp <= 0) { + // stopRow <= endKey (endKey is equals to or larger than stopRow) + // This is a stop. + return true; + } + } + return false; //unlikely. + } + + private boolean possiblyNextScanner(int nbRows, final boolean done) throws IOException { + // If we have just switched replica, don't go to the next scanner yet. Rather, try + // the scanner operations on the new replica, from the right point in the scan + // Note that when we switched to a different replica we left it at a point + // where we just did the "openScanner" with the appropriate startrow + if (callable != null && callable.switchedToADifferentReplica()) return true; + return nextScanner(nbRows, done); + } + + /** + * Gets a scanner for the next region. If this.currentRegion != null, then + * we will move to the endrow of this.currentRegion. Else we will get + * scanner at the scan.getStartRow(). We will go no further, just tidy + * up outstanding scanners, if currentRegion != null and + * done is true. + * @param nbRows + * @param done Server-side says we're done scanning. + */ + protected boolean nextScanner(int nbRows, final boolean done) + throws IOException { + // Close the previous scanner if it's open + if (this.callable != null) { + this.callable.setClose(); + call(callable, caller, scannerTimeout); + this.callable = null; + } + + // Where to start the next scanner + byte [] localStartKey; + + // if we're at end of table, close and return false to stop iterating + if (this.currentRegion != null) { + byte [] endKey = this.currentRegion.getEndKey(); + if (endKey == null || + Bytes.equals(endKey, HConstants.EMPTY_BYTE_ARRAY) || + checkScanStopRow(endKey) || + done) { + close(); + if (LOG.isTraceEnabled()) { + LOG.trace("Finished " + this.currentRegion); + } + return false; + } + localStartKey = endKey; + if (LOG.isTraceEnabled()) { + LOG.trace("Finished " + this.currentRegion); + } + } else { + localStartKey = this.scan.getStartRow(); + } + + if (LOG.isDebugEnabled() && this.currentRegion != null) { + // Only worth logging if NOT first region in scan. + LOG.debug("Advancing internal scanner to startKey at '" + + Bytes.toStringBinary(localStartKey) + "'"); + } + try { + callable = getScannerCallable(localStartKey, nbRows); + // Open a scanner on the region server starting at the + // beginning of the region + call(callable, caller, scannerTimeout).get(); + this.currentRegion = callable.getHRegionInfo(); + if (this.scanMetrics != null) { + this.scanMetrics.countOfRegions.incrementAndGet(); + } + } catch (InterruptedException e) { + close(); + throw new InterruptedIOException(e.getMessage()); + } catch (ExecutionException e) { + close(); + throw (IOException) e.getCause(); + } + return true; + } + + @VisibleForTesting + boolean isAnyRPCcancelled() { + return callable.isAnyRPCcancelled(); + } + + /** + * Call the scanner for results + * @param callable to call + * @param caller to call with + * @param scannerTimeout for timeout + * @return Promise with results + * @throws IOException + * @throws RuntimeException + */ + Future call(ScannerCallableWithReplicas callable, + RpcRetryingCaller caller, int scannerTimeout) { + return caller.callAsyncWithRetries(callable, scannerTimeout); + } + + @InterfaceAudience.Private + protected ScannerCallableWithReplicas getScannerCallable(byte [] localStartKey, + int nbRows) { + scan.setStartRow(localStartKey); + ScannerCallable s = + new ScannerCallable(getConnection(), getTable(), scan, this.scanMetrics); + s.setCaching(nbRows); + return new ScannerCallableWithReplicas(tableName, getConnection(), + s, primaryOperationTimeout, scan, + scannerTimeout, caller); + } + + /** + * Publish the scan metrics. For now, we use scan.setAttribute to pass the metrics back to the + * application or TableInputFormat.Later, we could push it to other systems. We don't use + * metrics framework because it doesn't support multi-instances of the same metrics on the same + * machine; for scan/map reduce scenarios, we will have multiple scans running at the same time. + * + * By default, scan metrics are disabled; if the application wants to collect them, this + * behavior can be turned on by calling calling {@link Scan#setScanMetricsEnabled(boolean)} + * + *

This invocation clears the scan metrics. Metrics are aggregated in the Scan instance. + */ + protected void writeScanMetrics() { + if (this.scanMetrics == null || scanMetricsPublished) { + return; + } + MapReduceProtos.ScanMetrics pScanMetrics = ProtobufUtil.toScanMetrics(scanMetrics); + scan.setAttribute(Scan.SCAN_ATTRIBUTES_METRICS_DATA, pScanMetrics.toByteArray()); + scanMetricsPublished = true; + } + + /** + * Contact the servers to load more {@link Result}s in the cache. + */ + @Override + public Future nextBatch() { + // check if scanner was closed during previous prefetch + // Also return null if scan is done + if (closed || !shouldContinue){ + return new SuccessfulFuture<>(callable.getEventExecutor(),null); + } + + final Promise responsePromise = new Promise<>(callable + .getEventExecutor()); + + call(callable, caller, scannerTimeout).addListener( + new GenericFutureListener>() { + @Override + public void operationComplete(Future future) throws Exception { + long remainingResultSize = maxScannerResultSize; + int countdown = batchSize; + + // We need to reset it if it's a new callable that was created + // with a countdown in nextScanner + callable.setCaching(batchSize); + // This flag is set when we want to skip the result returned. We do + // this when we reset scanner because it split under us. + retryAfterOutOfOrderException = true; + + // We don't expect that the server will have more results for us if + // it doesn't tell us otherwise. We rely on the size or count of results + boolean serverHasMoreResults = false; + + Result[] values; + + if(future.isSuccess()){ + // Server returns a null values if scanning is to stop. Else, + // returns an empty array if scanning is to go on and we've just + // exhausted current region. + values = future.getNow(); + + // When the replica switch happens, we need to do certain operations + // again. The callable will openScanner with the right startkey + // but we need to pick up from there. Bypass the rest of the loop + // and let the catch-up happen in the beginning of the loop as it + // happens for the cases where we see exceptions. Since only openScanner + // would have happened, values would be null + if (values == null && callable.switchedToADifferentReplica()) { + // Any accumulated partial results are no longer valid since the callable will + // openScanner with the correct startkey and we must pick up from there + clearPartialResults(); + currentRegion = callable.getHRegionInfo(); + possiblyNextScanner(batchSize, true); + return; + } + + long currentTime = System.currentTimeMillis(); + if (scanMetrics != null) { + scanMetrics.sumOfMillisSecBetweenNexts.addAndGet(currentTime - lastNext); + } + lastNext = currentTime; + // Groom the array of Results that we received back from the server before adding that + // Results to the scanner's cache. If partial results are not allowed to be seen by the + // caller, all book keeping will be performed within this method. + List resultsForResponse = + getResultsForResponse(values, callable.isHeartbeatMessage()); + if (!resultsForResponse.isEmpty()) { + for (Result rs : resultsForResponse) { + long estimatedHeapSizeOfResult = calcEstimatedSize(rs); + countdown--; + remainingResultSize -= estimatedHeapSizeOfResult; + addEstimatedSize(estimatedHeapSizeOfResult); + lastResult = rs; + } + } + + // Caller of this method just wants a Result. If we see a heartbeat message, it means + // processing of the scan is taking a long time server side. Rather than continue to + // loop until a limit (e.g. size or caching) is reached, break out early to avoid causing + // unnecesary delays to the caller + if (callable.isHeartbeatMessage() && !resultsForResponse.isEmpty()) { + if (LOG.isTraceEnabled()) { + LOG.trace("Heartbeat message received and cache contains Results." + + " Breaking out of scan loop"); + } + responsePromise.setSuccess(resultsForResponse.toArray( + new Result[resultsForResponse.size()])); + return; + } + + // We expect that the server won't have more results for us when we exhaust + // the size (bytes or count) of the results returned. If the server *does* inform us that + // there are more results, we want to avoid possiblyNextScanner(...). Only when we + // actually get results is the moreResults context valid. + if (null != values && values.length > 0 && callable.hasMoreResultsContext()) { + // Only adhere to more server results when we don't have any partialResults + // as it keeps the outer loop logic the same. + serverHasMoreResults = callable.getServerHasMoreResults() & partialResults.isEmpty(); + } + retryAfterOutOfOrderException = true; + + responsePromise.setSuccess( + resultsForResponse.toArray(new Result[resultsForResponse.size()]) + ); + }else{ + values = null; + IOException e = (IOException) future.cause(); + if(e instanceof DoNotRetryIOException) { + // An exception was thrown which makes any partial results that we were collecting + // invalid. The scanner will need to be reset to the beginning of a row. + clearPartialResults(); + + // DNRIOEs are thrown to make us break out of retries. Some types of DNRIOEs want us + // to reset the scanner and come back in again. + if (e instanceof UnknownScannerException) { + long timeout = lastNext + scannerTimeout; + // If we are over the timeout, throw this exception to the client wrapped in + // a ScannerTimeoutException. Else, it's because the region moved and we used the old + // id against the new region server; reset the scanner. + if (timeout < System.currentTimeMillis()) { + long elapsed = System.currentTimeMillis() - lastNext; + ScannerTimeoutException ex = new ScannerTimeoutException( + elapsed + "ms passed since the last invocation, " + + "timeout is currently set to " + scannerTimeout); + ex.initCause(e); + throw ex; + } + } else { + // If exception is any but the list below throw it back to the client; else setup + // the scanner and retry. + Throwable cause = e.getCause(); + if ((cause != null && cause instanceof NotServingRegionException) || + (cause != null && cause instanceof RegionServerStoppedException) || + e instanceof OutOfOrderScannerNextException) { + // Pass + // It is easier writing the if loop test as list of what is allowed rather than + // as a list of what is not allowed... so if in here, it means we do not throw. + } else { + responsePromise.setFailure(e); + return; + } + } + // Else, its signal from depths of ScannerCallable that we need to reset the scanner. + if (lastResult != null) { + // The region has moved. We need to open a brand new scanner at + // the new location. + // Reset the startRow to the row we've seen last so that the new + // scanner starts at the correct row. Otherwise we may see previously + // returned rows again. + // (ScannerCallable by now has "relocated" the correct region) + if (scan.isReversed()) { + scan.setStartRow(createClosestRowBefore(lastResult.getRow())); + } else { + scan.setStartRow(Bytes.add(lastResult.getRow(), new byte[1])); + } + } + if (e instanceof OutOfOrderScannerNextException) { + if (retryAfterOutOfOrderException) { + retryAfterOutOfOrderException = false; + } else { + // TODO: Why wrap this in a DNRIOE when it already is a DNRIOE? + throw new DoNotRetryIOException("Failed after retry of " + + "OutOfOrderScannerNextException: was there a rpc timeout?", e); + } + } + // Clear region. + currentRegion = null; + // Set this to zero so we don't try and do an rpc and close on remote server when + // the exception we got was UnknownScanner or the Server is going down. + callable = null; + possiblyNextScanner(batchSize, true); + + responsePromise.setFailure(e); + } + } + + // Values == null means server-side filter has determined we must STOP + // !partialResults.isEmpty() means that we are still accumulating partial Results for a + // row. We should not change scanners before we receive all the partial Results for that + // row. + shouldContinue = doneWithRegion(remainingResultSize, countdown, serverHasMoreResults) + && (!partialResults.isEmpty() || possiblyNextScanner(countdown, values == null)); + } + }); + + return responsePromise; + } + + /** + * @param remainingResultSize + * @param remainingRows + * @param regionHasMoreResults + * @return true when the current region has been exhausted. When the current region has been + * exhausted, the region must be changed before scanning can continue + */ + private boolean doneWithRegion(long remainingResultSize, int remainingRows, + boolean regionHasMoreResults) { + return remainingResultSize > 0 && remainingRows > 0 && !regionHasMoreResults; + } + + /** + * Calculate the estimated heap size of the result + * @param rs Results to calculate size of + * @return heap size of result + */ + protected long calcEstimatedSize(Result rs) { + long estimatedHeapSizeOfResult = 0; + // We don't make Iterator here + for (Cell cell : rs.rawCells()) { + estimatedHeapSizeOfResult += CellUtil.estimatedHeapSizeOf(cell); + } + return estimatedHeapSizeOfResult; + } + + protected void addEstimatedSize(long estimatedHeapSizeOfResult) { + return; + } + + /** + * This method ensures all of our book keeping regarding partial results is kept up to date. This + * method should be called once we know that the results we received back from the RPC request do + * not contain errors. We return a list of results that should be added to the cache. In general, + * this list will contain all NON-partial results from the input array (unless the client has + * specified that they are okay with receiving partial results) + * @param resultsFromServer The array of {@link Result}s returned from the server + * @param heartbeatMessage Flag indicating whether or not the response received from the server + * represented a complete response, or a heartbeat message that was sent to keep the + * client-server connection alive + * @return the list of results that should be added to the cache. + * @throws IOException + */ + protected List getResultsForResponse(Result[] resultsFromServer, boolean heartbeatMessage) + throws IOException { + int resultSize = resultsFromServer != null ? resultsFromServer.length : 0; + List resultsToAddToCache = new ArrayList(resultSize); + + final boolean isBatchSet = scan != null && scan.getBatch() > 0; + final boolean allowPartials = scan != null && scan.getAllowPartialResults(); + + // If the caller has indicated in their scan that they are okay with seeing partial results, + // then simply add all results to the list. Note that since scan batching also returns results + // for a row in pieces we treat batch being set as equivalent to allowing partials. The + // implication of treating batching as equivalent to partial results is that it is possible + // the caller will receive a result back where the number of cells in the result is less than + // the batch size even though it may not be the last group of cells for that row. + if (allowPartials || isBatchSet) { + addResultsToList(resultsToAddToCache, resultsFromServer, 0, resultsFromServer.length); + return resultsToAddToCache; + } + + // If no results were returned it indicates that either we have the all the partial results + // necessary to construct the complete result or the server had to send a heartbeat message + // to the client to keep the client-server connection alive + if (resultsFromServer == null || resultsFromServer.length == 0) { + // If this response was an empty heartbeat message, then we have not exhausted the region + // and thus there may be more partials server side that still need to be added to the partial + // list before we form the complete Result + if (!partialResults.isEmpty() && !heartbeatMessage) { + resultsToAddToCache.add(Result.createCompleteResult(partialResults)); + clearPartialResults(); + } + + return resultsToAddToCache; + } + + // In every RPC response there should be at most a single partial result. Furthermore, if + // there is a partial result, it is guaranteed to be in the last position of the array. + Result last = resultsFromServer[resultsFromServer.length - 1]; + Result partial = last.isPartial() ? last : null; + + if (LOG.isTraceEnabled()) { + LOG.trace( + "number results from RPC: " + resultsFromServer.length + "," + "partial != null: " + ( + partial != null) + "," + "number of partials so far: " + partialResults.size()); + } + + // There are three possibilities cases that can occur while handling partial results + // + // 1. (partial != null && partialResults.isEmpty()) + // This is the first partial result that we have received. It should be added to + // the list of partialResults and await the next RPC request at which point another + // portion of the complete result will be received + // + // 2. !partialResults.isEmpty() + // Since our partialResults list is not empty it means that we have been accumulating partial + // Results for a particular row. We cannot form the complete/whole Result for that row until + // all partials for the row have been received. Thus we loop through all of the Results + // returned from the server and determine whether or not all partial Results for the row have + // been received. We know that we have received all of the partial Results for the row when: + // i) We notice a row change in the Results + // ii) We see a Result for the partial row that is NOT marked as a partial Result + // + // 3. (partial == null && partialResults.isEmpty()) + // Business as usual. We are not accumulating partial results and there wasn't a partial result + // in the RPC response. This means that all of the results we received from the server are + // complete and can be added directly to the cache + if (partial != null && partialResults.isEmpty()) { + addToPartialResults(partial); + + // Exclude the last result, it's a partial + addResultsToList(resultsToAddToCache, resultsFromServer, 0, resultsFromServer.length - 1); + } else if (!partialResults.isEmpty()) { + for (Result result : resultsFromServer) { + // This result is from the same row as the partial Results. Add it to the list of partials + // and check if it was the last partial Result for that row + if (Bytes.equals(partialResultsRow, result.getRow())) { + addToPartialResults(result); + + // If the result is not a partial, it is a signal to us that it is the last Result we + // need to form the complete Result client-side + if (!result.isPartial()) { + resultsToAddToCache.add(Result.createCompleteResult(partialResults)); + clearPartialResults(); + } + } else { + // The row of this result differs from the row of the partial results we have received so + // far. If our list of partials isn't empty, this is a signal to form the complete Result + // since the row has now changed + if (!partialResults.isEmpty()) { + resultsToAddToCache.add(Result.createCompleteResult(partialResults)); + clearPartialResults(); + } + + // It's possible that in one response from the server we receive the final partial for + // one row and receive a partial for a different row. Thus, make sure that all Results + // are added to the proper list + if (result.isPartial()) { + addToPartialResults(result); + } else { + resultsToAddToCache.add(result); + } + } + } + } else { // partial == null && partialResults.isEmpty() -- business as usual + addResultsToList(resultsToAddToCache, resultsFromServer, 0, resultsFromServer.length); + } + + return resultsToAddToCache; + } + + /** + * A convenience method for adding a Result to our list of partials. This method ensure that only + * Results that belong to the same row as the other partials can be added to the list. + * @param result The result that we want to add to our list of partial Results + * @throws IOException if partial result does not belong to current found results + */ + private void addToPartialResults(final Result result) throws IOException { + final byte[] row = result.getRow(); + if (partialResultsRow != null && !Bytes.equals(row, partialResultsRow)) { + throw new IOException("Partial result row does not match. All partial results must come " + + "from the same row. partialResultsRow: " + Bytes.toString(partialResultsRow) + "row: " + + Bytes.toString(row)); + } + partialResultsRow = row; + partialResults.add(result); + } + + /** + * Convenience method for clearing the list of partials and resetting the partialResultsRow. + */ + private void clearPartialResults() { + partialResults.clear(); + partialResultsRow = null; + } + + /** + * Helper method for adding results between the indices [start, end) to the outputList + * @param outputList the list that results will be added to + * @param inputArray the array that results are taken from + * @param start beginning index (inclusive) + * @param end ending index (exclusive) + */ + private void addResultsToList(List outputList, Result[] inputArray, int start, int end) { + if (inputArray == null || start < 0 || end > inputArray.length) return; + + outputList.addAll(Arrays.asList(inputArray).subList(start, end)); + } + + @Override + public void close() { + if (!scanMetricsPublished) writeScanMetrics(); + if (callable != null) { + callable.setClose(); + try { + call(callable, caller, scannerTimeout).get(); + } catch (ExecutionException e) { + if(e.getCause() instanceof UnknownScannerException) { + // We used to catch this error, interpret, and rethrow. However, we + // have since decided that it's not nice for a scanner's close to + // throw exceptions. Chances are it was just due to lease time out. + if (LOG.isDebugEnabled()) { + LOG.debug("scanner failed to close", e); + } + }else{ + /* An exception other than UnknownScanner is unexpected. */ + LOG.warn("scanner failed to close.", e); + } + } catch (InterruptedException e) { + /* An exception other than UnknownScanner is unexpected. */ + LOG.warn("scanner failed to close.", e); + } + callable = null; + } + closed = true; + } + + /** + * Create the closest row before the specified row + * @param row to get closest row before + * @return a new byte array which is the closest front row of the specified one + */ + protected static byte[] createClosestRowBefore(byte[] row) { + if (row == null) { + throw new IllegalArgumentException("The passed row is empty"); + } + if (Bytes.equals(row, HConstants.EMPTY_BYTE_ARRAY)) { + return MAX_BYTE_ARRAY; + } + if (row[row.length - 1] == 0) { + return Arrays.copyOf(row, row.length - 1); + } else { + byte[] closestFrontRow = Arrays.copyOf(row, row.length); + closestFrontRow[row.length - 1] = (byte) ((closestFrontRow[row.length - 1] & 0xff) - 1); + closestFrontRow = Bytes.add(closestFrontRow, MAX_BYTE_ARRAY); + return closestFrontRow; + } + } + + @Override + public Future renewLease() { + if (callable != null) { + // do not return any rows, do not advance the scanner + callable.setCaching(0); + + final Promise dp = new Promise<>(callable.getEventExecutor()); + + Future promise = + this.caller.callAsyncWithoutRetries(callable, this.scannerTimeout); + promise.addListener(new GenericFutureListener>() { + @Override + public void operationComplete(Future future) throws Exception { + if (future.isSuccess()) { + dp.setSuccess(true); + } else { + dp.setSuccess(false); + } + callable.setCaching(scan.getCaching()); + } + }); + + return dp; + }else{ + return new SuccessfulFuture<>(connection.getEventExecutor(),false); + } + } + + @Override + public boolean isClosed() { + return closed; + } +} \ No newline at end of file diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncClientSmallReversedScanner.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncClientSmallReversedScanner.java new file mode 100644 index 0000000..b3f507f --- /dev/null +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncClientSmallReversedScanner.java @@ -0,0 +1,243 @@ +/** + * Copyright The Apache Software Foundation + * + * 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 com.google.common.annotations.VisibleForTesting; +import io.netty.util.concurrent.GenericFutureListener; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.client.ClientSmallScanner.SmallScannerCallable; +import org.apache.hadoop.hbase.client.ClientSmallScanner.SmallScannerCallableFactory; +import org.apache.hadoop.hbase.util.Bytes; + +import java.io.IOException; + +/** + * Client scanner for small reversed scan. Generally, only one RPC is called to fetch the + * scan results, unless the results cross multiple regions or the row count of + * results exceed the caching. + *

+ * For small scan, it will get better performance than {@link ReversedClientScanner} + */ +@InterfaceAudience.Private +public class AsyncClientSmallReversedScanner extends AsyncReversedClientScanner { + private static final Log LOG = LogFactory.getLog(AsyncClientSmallReversedScanner.class); + private ScannerCallableWithReplicas smallScanCallable = null; + private SmallScannerCallableFactory callableFactory; + private boolean currentRegionDone = false; + + /** + * Create a new ReversibleClientScanner for the specified table. Take note that the passed + * {@link Scan} 's start row maybe changed changed. + * + * @param conf + * The {@link Configuration} to use. + * @param scan + * {@link Scan} to use in this scanner + * @param tableName + * The table that we wish to rangeGet + * @param connection + * Connection identifying the cluster + * @param rpcFactory + * Factory used to create the {@link RpcRetryingCaller} + * @param primaryOperationTimeout + * Call timeout + * @throws IOException + * If the remote call fails + */ + public AsyncClientSmallReversedScanner(final Configuration conf, final Scan scan, + final TableName tableName, ClusterConnection connection, RpcRetryingCallerFactory rpcFactory, + int primaryOperationTimeout) + throws IOException { + this(conf, scan, tableName, connection, rpcFactory, primaryOperationTimeout, + new SmallScannerCallableFactory()); + } + + /** + * Create a new ReversibleClientScanner for the specified table. Take note that the passed + * {@link Scan}'s start row may be changed. + * + * @param conf + * The {@link Configuration} to use. + * @param scan + * {@link Scan} to use in this scanner + * @param tableName + * The table that we wish to rangeGet + * @param connection + * Connection identifying the cluster + * @param rpcFactory + * Factory used to create the {@link RpcRetryingCaller} + * @param primaryOperationTimeout + * Call timeout + * @param callableFactory + * Factory used to create the {@link SmallScannerCallable} + * @throws IOException + * If the remote call fails + */ + @VisibleForTesting + AsyncClientSmallReversedScanner(final Configuration conf, final Scan scan, + final TableName tableName, ClusterConnection connection, RpcRetryingCallerFactory rpcFactory, + int primaryOperationTimeout, SmallScannerCallableFactory callableFactory) throws IOException { + super(conf, scan, tableName, connection, rpcFactory, + primaryOperationTimeout); + this.callableFactory = callableFactory; + } + + /** + * Gets a scanner for following scan. Move to next region or continue from the last result or + * start from the start row. + * + * @param nbRows + * @param done + * true if Server-side says we're done scanning. + * @param currentRegionDone + * true if scan is over on current region + * @return true if has next scanner + * @throws IOException + */ + private boolean nextScanner(int nbRows, final boolean done, + boolean currentRegionDone) throws IOException { + // Where to start the next getter + byte[] localStartKey; + int cacheNum = nbRows; + boolean regionChanged = true; + // if we're at end of table, close and return false to stop iterating + if (this.currentRegion != null && currentRegionDone) { + byte[] startKey = this.currentRegion.getStartKey(); + if (startKey == null + || Bytes.equals(startKey, HConstants.EMPTY_BYTE_ARRAY) + || checkScanStopRow(startKey) || done) { + close(); + if (LOG.isDebugEnabled()) { + LOG.debug("Finished with small scan at " + this.currentRegion); + } + return false; + } + // We take the row just under to get to the previous region. + localStartKey = createClosestRowBefore(startKey); + if (LOG.isDebugEnabled()) { + LOG.debug("Finished with region " + this.currentRegion); + } + } else if (this.lastResult != null) { + regionChanged = false; + localStartKey = createClosestRowBefore(lastResult.getRow()); + } else { + localStartKey = this.scan.getStartRow(); + } + + if (LOG.isTraceEnabled()) { + LOG.trace("Advancing internal small scanner to startKey at '" + + Bytes.toStringBinary(localStartKey) + "'"); + } + + smallScanCallable = callableFactory.getCallable(getConnection(), getTable(), scan, + getScanMetrics(), localStartKey, cacheNum, getPrimaryOperationTimeout(), + getScannerTimeout(), caller); + + if (this.scanMetrics != null && regionChanged) { + this.scanMetrics.countOfRegions.incrementAndGet(); + } + return true; + } + + @Override + public Future nextBatch() { + this.currentRegionDone = false; + + try { + shouldContinue = maxScannerResultSize > 0 && batchSize > 0 && nextScanner(batchSize, true, + currentRegionDone); + } catch (IOException e) { + return new FailedFuture<>(getConnection().getEventExecutor(),e); + } + + if(!shouldContinue){ + return new SuccessfulFuture<>(getConnection().getEventExecutor(),null); + } + + Future responsePromise = + this.caller.callAsyncWithoutRetries(smallScanCallable, scannerTimeout); + + responsePromise.addListener(new GenericFutureListener>() { + @Override + public void operationComplete(Future future) throws Exception { + long remainingResultSize = maxScannerResultSize; + int countdown = batchSize; + + // Server returns a null values if scanning is to stop. Else, + // returns an empty array if scanning is to go on and we've just + // exhausted current region. + // callWithoutRetries is at this layer. Within the ScannerCallableWithReplicas, + // we do a callWithRetries + Result[] values = future.getNow(); + + currentRegion = smallScanCallable.getHRegionInfo(); + long currentTime = System.currentTimeMillis(); + if (scanMetrics != null) { + scanMetrics.sumOfMillisSecBetweenNexts.addAndGet(currentTime - lastNext); + } + lastNext = currentTime; + if (values != null && values.length > 0) { + for (Result rs : values) { + // We don't make Iterator here + for (Cell cell : rs.rawCells()) { + remainingResultSize -= CellUtil.estimatedHeapSizeOf(cell); + } + countdown--; + lastResult = rs; + } + } + if (smallScanCallable.hasMoreResultsContext()) { + currentRegionDone = !smallScanCallable.getServerHasMoreResults(); + } else { + currentRegionDone = countdown > 0; + } + + shouldContinue = remainingResultSize > 0 && countdown > 0 && nextScanner(countdown, + values == null, currentRegionDone); + } + }); + + return responsePromise; + } + + @Override + protected void initializeScannerInConstruction() throws IOException { + // No need to initialize the scanner when constructing instance, do it when + // calling next(). Do nothing here. + } + + @Override + public void close() { + if (!scanMetricsPublished) writeScanMetrics(); + closed = true; + } + + @VisibleForTesting + protected void setScannerCallableFactory(SmallScannerCallableFactory callableFactory) { + this.callableFactory = callableFactory; + } +} \ No newline at end of file diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncClientSmallScanner.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncClientSmallScanner.java new file mode 100644 index 0000000..a7b2eb9 --- /dev/null +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncClientSmallScanner.java @@ -0,0 +1,290 @@ +/** + * Copyright The Apache Software Foundation + * + * 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 com.google.common.annotations.VisibleForTesting; +import io.netty.util.concurrent.GenericFutureListener; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.CellScanner; +import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.client.metrics.ScanMetrics; +import org.apache.hadoop.hbase.ipc.IOExceptionConverter; +import org.apache.hadoop.hbase.ipc.MessageConverter; +import org.apache.hadoop.hbase.protobuf.RequestConverter; +import org.apache.hadoop.hbase.protobuf.ResponseConverter; +import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanRequest; +import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanResponse; +import org.apache.hadoop.hbase.util.Bytes; + +import java.io.IOException; + +/** + * Client scanner for small scan. Generally, only one RPC is called to fetch the + * scan results, unless the results cross multiple regions or the row count of + * results excess the caching. + * + * For small scan, it will get better performance than {@link ClientScanner} + */ +@InterfaceAudience.Private +public class AsyncClientSmallScanner extends AsyncClientScanner { + private static final Log LOG = LogFactory.getLog(AsyncClientSmallScanner.class); + private ScannerCallableWithReplicas smallScanCallable = null; + private ClientSmallScanner.SmallScannerCallableFactory callableFactory; + private boolean currentRegionDone = false; + + /** + * Create a new ShortClientScanner for the specified table. Take note that the passed {@link Scan} + * 's start row maybe changed changed. + * + * @param conf + * The {@link Configuration} to use. + * @param scan + * {@link Scan} to use in this scanner + * @param tableName + * The table that we wish to rangeGet + * @param connection + * Connection identifying the cluster + * @param rpcFactory + * Factory used to create the {@link RpcRetryingCaller} + * @param primaryOperationTimeout + * Call timeout + * @throws IOException + * If the remote call fails + */ + public AsyncClientSmallScanner(final Configuration conf, final Scan scan, + final TableName tableName, ClusterConnection connection, RpcRetryingCallerFactory rpcFactory, + int primaryOperationTimeout) + throws IOException { + this(conf, scan, tableName, connection, rpcFactory, + primaryOperationTimeout, new ClientSmallScanner.SmallScannerCallableFactory()); + } + + /** + * Create a new ShortClientScanner for the specified table. Take note that the passed {@link Scan} + * 's start row maybe changed changed. Intended for unit tests to provide their own + * {@link ClientSmallScanner.SmallScannerCallableFactory} implementation/mock. + * + * @param conf + * The {@link Configuration} to use. + * @param scan + * {@link Scan} to use in this scanner + * @param tableName + * The table that we wish to rangeGet + * @param connection + * Connection identifying the cluster + * @param rpcFactory + * Factory used to create the {@link RpcRetryingCaller} + * @param primaryOperationTimeout + * Call timeout + * @param callableFactory + * Factory used to create the {@link SmallScannerCallable} + * @throws IOException + */ + @VisibleForTesting + AsyncClientSmallScanner(final Configuration conf, final Scan scan, final TableName tableName, + ClusterConnection connection, RpcRetryingCallerFactory rpcFactory, + int primaryOperationTimeout, ClientSmallScanner.SmallScannerCallableFactory callableFactory) + throws IOException { + super(conf, scan, tableName, connection, rpcFactory, primaryOperationTimeout); + this.callableFactory = callableFactory; + } + + @Override + protected void initializeScannerInConstruction() throws IOException { + // No need to initialize the scanner when constructing instance, do it when + // calling next(). Do nothing here. + } + + /** + * Gets a scanner for following scan. Move to next region or continue from the + * last result or start from the start row. + * @param nbRows + * @param done true if Server-side says we're done scanning. + * @param currentRegionDone true if scan is over on current region + * @return true if has next scanner + * @throws IOException + */ + private boolean nextScanner(int nbRows, final boolean done, + boolean currentRegionDone) throws IOException { + // Where to start the next getter + byte[] localStartKey; + boolean regionChanged = true; + // if we're at end of table, close and return false to stop iterating + if (this.currentRegion != null && currentRegionDone) { + byte[] endKey = this.currentRegion.getEndKey(); + if (endKey == null || Bytes.equals(endKey, HConstants.EMPTY_BYTE_ARRAY) + || checkScanStopRow(endKey) || done) { + close(); + if (LOG.isTraceEnabled()) { + LOG.trace("Finished with small scan at " + this.currentRegion); + } + return false; + } + localStartKey = endKey; + if (LOG.isTraceEnabled()) { + LOG.trace("Finished with region " + this.currentRegion); + } + } else if (this.lastResult != null) { + regionChanged = false; + localStartKey = Bytes.add(lastResult.getRow(), new byte[1]); + } else { + localStartKey = this.scan.getStartRow(); + } + + if (LOG.isTraceEnabled()) { + LOG.trace("Advancing internal small scanner to startKey at '" + + Bytes.toStringBinary(localStartKey) + "'"); + } + smallScanCallable = callableFactory.getCallable(getConnection(), getTable(), scan, + getScanMetrics(), localStartKey, nbRows, + getPrimaryOperationTimeout(), getScannerTimeout(), caller); + if (this.scanMetrics != null && regionChanged) { + this.scanMetrics.countOfRegions.incrementAndGet(); + } + return true; + } + + @Override + public Future nextBatch() { + try { + shouldContinue = maxScannerResultSize > 0 && batchSize > 0 + && nextScanner(batchSize, true, currentRegionDone); + } catch (IOException e) { + return new FailedFuture<>(getConnection().getEventExecutor(),e); + } + + if(!shouldContinue){ + return new SuccessfulFuture<>(getConnection().getEventExecutor(),null); + } + + Future responsePromise = + this.caller.callAsyncWithoutRetries(smallScanCallable, scannerTimeout); + + responsePromise.addListener(new GenericFutureListener>() { + @Override + public void operationComplete(Future future) throws Exception { + long remainingResultSize = maxScannerResultSize; + int countdown = batchSize; + + // Server returns a null values if scanning is to stop. Else, + // returns an empty array if scanning is to go on and we've just + // exhausted current region. + // callWithoutRetries is at this layer. Within the ScannerCallableWithReplicas, + // we do a callWithRetries + Result[] values = future.getNow(); + currentRegion = smallScanCallable.getHRegionInfo(); + long currentTime = System.currentTimeMillis(); + if (scanMetrics != null) { + scanMetrics.sumOfMillisSecBetweenNexts.addAndGet(currentTime - lastNext); + } + lastNext = currentTime; + if (values != null && values.length > 0) { + for (Result rs : values) { + // We don't make Iterator here + for (Cell cell : rs.rawCells()) { + remainingResultSize -= CellUtil.estimatedHeapSizeOf(cell); + } + countdown--; + lastResult = rs; + } + } + if (smallScanCallable.hasMoreResultsContext()) { + // If the server has more results, the current region is not done + currentRegionDone = !smallScanCallable.getServerHasMoreResults(); + } else { + // not guaranteed to get the context in older versions, fall back to checking countdown + currentRegionDone = countdown > 0; + } + + shouldContinue = remainingResultSize > 0 && countdown > 0 + && nextScanner(countdown, values == null, currentRegionDone); + } + }); + + return responsePromise; + } + + public void close() { + if (!scanMetricsPublished) writeScanMetrics(); + closed = true; + } + + @VisibleForTesting + protected void setScannerCallableFactory( + ClientSmallScanner.SmallScannerCallableFactory callableFactory) { + this.callableFactory = callableFactory; + } + + static class SmallScannerCallable extends ScannerCallable { + public SmallScannerCallable( + ClusterConnection connection, TableName table, Scan scan, + ScanMetrics scanMetrics, int caching, int id) { + super(connection, table, scan, scanMetrics, id); + this.setCaching(caching); + } + + @Override + public Future call(int timeout) { + if (this.closed) { + return null; + } + ScanRequest request; + try { + request = RequestConverter + .buildScanRequest(getLocation().getRegionInfo().getRegionName(), getScan(), + getCaching(), true); + } catch (IOException e) { + return new FailedFuture<>(getEventExecutor(),e); + } + + return sendRequest(request, timeout, new MessageConverter() { + @Override + public Result[] convert(ScanResponse response, CellScanner cellScanner) throws IOException { + Result[] results = ResponseConverter.getResults(cellScanner, response); + if (response.hasMoreResultsInRegion()) { + setHasMoreResultsContext(true); + setServerHasMoreResults(response.getMoreResultsInRegion()); + } else { + setHasMoreResultsContext(false); + } + // We need to update result metrics since we are overriding call() + updateResultsMetrics(results); + return results; + } + }, new IOExceptionConverter() { + @Override + public IOException convert(IOException e) { + return null; + } + }); + } + + @Override + public ScannerCallable getScannerCallableForReplica(int id) { + return new SmallScannerCallable((ClusterConnection)connection, tableName, getScan(), + scanMetrics, getCaching(), id); + } + } +} \ No newline at end of file diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionServerCallable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionServerCallable.java new file mode 100644 index 0000000..9841ed0 --- /dev/null +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionServerCallable.java @@ -0,0 +1,72 @@ +/** + * + * 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 io.netty.util.concurrent.EventExecutor; +import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.ipc.AsyncRpcChannel; + +import java.io.IOException; + +/** + * Implementations call a RegionServer and implement {@link #call(int)}. + * Passed to a {@link RpcRetryingCaller} so we retry on fail. + * TODO: this class is actually tied to one region, because most of the paths make use of + * the regioninfo part of location when building requests. The only reason it works for + * multi-region requests (e.g. batch) is that they happen to not use the region parts. + * This could be done cleaner (e.g. having a generic parameter and 2 derived classes, + * RegionCallable and actual RegionServerCallable with ServerName. + * @param the class that the ServerCallable handles + */ +@InterfaceAudience.Private +public abstract class AsyncRegionServerCallable extends AbstractRegionServerCallable + implements AsyncRetryingCallable { + private AsyncRpcChannel channel; + // Public because used outside of this package over in ipc. + + /** + * @param connection Connection to use. + * @param tableName Table name to which row belongs. + * @param row The row we want in tableName. + */ + public AsyncRegionServerCallable(Connection connection, TableName tableName, byte[] row) { + super(connection, tableName, row); + } + + @Override + void setClientByServiceName(ServerName service) throws IOException { + this.channel = getConnection().getAsyncClientChannel(service); + } + + /** + * Get the Async RPC channel for this Callable + * @return AsyncRpcChannel + */ + public AsyncRpcChannel getChannel() { + return channel; + } + + @Override + public EventExecutor getEventExecutor() { + return ((ClusterConnection)this.connection).getEventExecutor(); + } +} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncResultScanner.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncResultScanner.java new file mode 100644 index 0000000..f08028f --- /dev/null +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncResultScanner.java @@ -0,0 +1,34 @@ +package org.apache.hadoop.hbase.client; + +import java.io.Closeable; + +/** + * Async Interface for client-side scanning. + * Go to {@link AsyncTable} to obtain instances. + */ +public interface AsyncResultScanner extends Closeable { + /** + * Grab the next row's worth of values. The scanner will return a Result. + * @return Promise with result objects if there is another row, null if the scanner is + * exhausted. + */ + Future nextBatch(); + + /** + * Allow the client to renew the scanner's lease on the server. + * @return true if the lease was successfully renewed, false otherwise. + */ + Future renewLease(); + + /** + * Closes the scanner and releases any resources it has allocated + */ + @Override + void close(); + + /** + * Returns true if scanner is closed + * @return true if scanner was closed + */ + boolean isClosed(); +} \ No newline at end of file diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRetryingCallable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRetryingCallable.java new file mode 100644 index 0000000..488b420 --- /dev/null +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRetryingCallable.java @@ -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. + */ + +package org.apache.hadoop.hbase.client; + +import io.netty.channel.EventLoop; +import io.netty.util.concurrent.EventExecutor; +import org.apache.hadoop.hbase.classification.InterfaceAudience; + +/** + * A Callable that will be retried async. + * @param + */ +@InterfaceAudience.Private +public interface AsyncRetryingCallable extends ProtoRetryingCallable { + /** + * Computes a result, or throws an exception if unable to do so. + * + * @param callTimeout - the time available for this call. 0 for infinite. + * @return Future which handles the Result + */ + Future call(int callTimeout); + + /** + * Get EventLoop to operate async operations on + * @return AsyncRpcChannel + */ + EventExecutor getEventExecutor(); +} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncReversedClientScanner.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncReversedClientScanner.java new file mode 100644 index 0000000..219f35a --- /dev/null +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncReversedClientScanner.java @@ -0,0 +1,170 @@ +/** + * Copyright The Apache Software Foundation + * + * 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.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.ExceptionUtil; + +import java.io.IOException; +import java.io.InterruptedIOException; +import java.util.concurrent.ExecutionException; + +/** + * A reversed client scanner which support backward scanning + */ +@InterfaceAudience.Private +public class AsyncReversedClientScanner extends AsyncClientScanner { + private static final Log LOG = LogFactory.getLog(AsyncReversedClientScanner.class); + + /** + * Create a new ReversibleClientScanner for the specified table Note that the + * passed {@link Scan}'s start row maybe changed. + * @param conf + * @param scan + * @param tableName + * @param connection + * @param primaryOperationTimeout + * @throws IOException + */ + public AsyncReversedClientScanner(Configuration conf, Scan scan, TableName tableName, + ClusterConnection connection, RpcRetryingCallerFactory rpcFactory, + int primaryOperationTimeout) + throws IOException { + super(conf, scan, tableName, connection, rpcFactory, primaryOperationTimeout); + } + + @Override + protected boolean nextScanner(int nbRows, final boolean done) + throws IOException { + // Close the previous scanner if it's open + if (this.callable != null) { + this.callable.setClose(); + // callWithoutRetries is at this layer. Within the ScannerCallableWithReplicas, + // we do a callWithRetries + try { + this.caller.callAsyncWithoutRetries(callable, scannerTimeout).get(); + } catch (InterruptedException e) { + throw new InterruptedIOException(e.getMessage()); + } catch (ExecutionException e) { + throw (IOException) e.getCause(); + } + this.callable = null; + } + + // Where to start the next scanner + byte[] localStartKey; + boolean locateTheClosestFrontRow = true; + // if we're at start of table, close and return false to stop iterating + if (this.currentRegion != null) { + byte[] startKey = this.currentRegion.getStartKey(); + if (startKey == null + || Bytes.equals(startKey, HConstants.EMPTY_BYTE_ARRAY) + || checkScanStopRow(startKey) || done) { + close(); + if (LOG.isDebugEnabled()) { + LOG.debug("Finished " + this.currentRegion); + } + return false; + } + localStartKey = startKey; + if (LOG.isDebugEnabled()) { + LOG.debug("Finished " + this.currentRegion); + } + } else { + localStartKey = this.scan.getStartRow(); + if (!Bytes.equals(localStartKey, HConstants.EMPTY_BYTE_ARRAY)) { + locateTheClosestFrontRow = false; + } + } + + if (LOG.isDebugEnabled() && this.currentRegion != null) { + // Only worth logging if NOT first region in scan. + LOG.debug("Advancing internal scanner to startKey at '" + + Bytes.toStringBinary(localStartKey) + "'"); + } + try { + // In reversed scan, we want to locate the previous region through current + // region's start key. In order to get that previous region, first we + // create a closest row before the start key of current region, then + // locate all the regions from the created closest row to start key of + // current region, thus the last one of located regions should be the + // previous region of current region. The related logic of locating + // regions is implemented in ReversedScannerCallable + byte[] locateStartRow = locateTheClosestFrontRow ? createClosestRowBefore(localStartKey) + : null; + callable = getScannerCallable(localStartKey, nbRows, locateStartRow); + // Open a scanner on the region server starting at the + // beginning of the region + // callWithoutRetries is at this layer. Within the ScannerCallableWithReplicas, + // we do a callWithRetries + try { + this.caller.callAsyncWithoutRetries(callable, scannerTimeout).get(); + } catch (InterruptedException e) { + throw new InterruptedIOException(e.getMessage()); + } catch (ExecutionException e) { + throw (IOException) e.getCause(); + } + this.currentRegion = callable.getHRegionInfo(); + if (this.scanMetrics != null) { + this.scanMetrics.countOfRegions.incrementAndGet(); + } + } catch (IOException e) { + ExceptionUtil.rethrowIfInterrupt(e); + close(); + throw e; + } + return true; + } + + protected ScannerCallableWithReplicas getScannerCallable(byte[] localStartKey, + int nbRows, byte[] locateStartRow) { + scan.setStartRow(localStartKey); + ScannerCallable s = + new ReversedScannerCallable(getConnection(), getTable(), scan, this.scanMetrics, + locateStartRow); + s.setCaching(nbRows); + ScannerCallableWithReplicas sr = + new ScannerCallableWithReplicas(getTable(), getConnection(), s, + primaryOperationTimeout, scan, getScannerTimeout(), caller); + return sr; + } + + @Override + // returns true if stopRow >= passed region startKey + protected boolean checkScanStopRow(final byte[] startKey) { + if (this.scan.getStopRow().length > 0) { + // there is a stop row, check to see if we are past it. + byte[] stopRow = scan.getStopRow(); + int cmp = Bytes.compareTo(stopRow, 0, stopRow.length, startKey, 0, + startKey.length); + if (cmp >= 0) { + // stopRow >= startKey (stopRow is equals to or larger than endKey) + // This is a stop. + return true; + } + } + return false; // unlikely. + } +} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTable.java new file mode 100644 index 0000000..9d2bfe3 --- /dev/null +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTable.java @@ -0,0 +1,252 @@ +/** + * + * 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 com.google.protobuf.Descriptors; +import com.google.protobuf.Message; +import com.google.protobuf.Service; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.classification.InterfaceStability; +import org.apache.hadoop.hbase.client.coprocessor.AsyncBatch; +import org.apache.hadoop.hbase.client.coprocessor.Batch; +import org.apache.hadoop.hbase.filter.CompareFilter; +import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel; + +import java.io.Closeable; +import java.io.IOException; +import java.util.List; +import java.util.Map; + +/** + * Used to communicate with a single HBase table. + * Obtain an instance from a {@link Connection} and call {@link #close()} afterwards. + * + *

AsyncTable can be used to get, put, delete or scan data from a table. + * @since 2.0.0 + */ +@InterfaceAudience.Public +@InterfaceStability.Evolving +public interface AsyncTable extends Closeable { + /** + * Gets the fully qualified table name instance of this table. + */ + TableName getName(); + + /** + * Returns the {@link Configuration} object used by this instance. + *

+ * The reference returned is not a copy, so any change made to it will + * affect this instance. + */ + Configuration getConfiguration(); + + /** + * Extracts certain cells from a given row. + * @param get The object that specifies what data to fetch and from which row. + * @return Future with the data coming from the specified row, if it exists. If the row + * specified doesn't exist, the {@link Result} instance returned won't + * contain any {@link org.apache.hadoop.hbase.KeyValue}, as indicated by {@link Result#isEmpty()}. + */ + Future get(Get get); + + /** + * Extracts certain cells from the given rows, in batch. + * + * @param gets The objects that specify what data to fetch and from which rows. + * @return Future with the data coming from the specified rows, if it exists. If the row + * specified doesn't exist, the {@link Result} instance returned won't contain any {@link + * org.apache.hadoop.hbase.KeyValue}, as indicated by {@link Result#isEmpty()}. If there are any + * failures even after retries, there will be a null in the results array for those Gets, AND an + * exception will be thrown. + */ + Future get(List gets); + + /** + * Returns an async scanner on the current table as specified by the {@link Scan} + * object. + * Note that the passed {@link Scan}'s start row and caching properties + * maybe changed. + * + * @param scan A configured {@link Scan} object. + * @return A scanner. + */ + AsyncResultScanner getScanner(Scan scan) throws IOException; + + /** + * Mutates a Row with a Mutation. Can be a Put, Append, Delete or Increment + * @param mutation to be committed + * @return Future with Result if Append or Increment + */ + Future mutate(Mutation mutation); + + /** + * Mutates a Row with multiple mutations on one row. + * @param mutation to be committed + * @return Future with Result if Append or Increment + */ + Future mutate(RowMutations mutation); + + /** + * Atomically checks if a row/family/qualifier value matches the expected value. + * If it does, it performs the row mutations. If the passed value is null, the check + * is for the lack of column (ie: non-existence) + * + * It accepts a single Mutation like Put/Delete/Append/Increment + * + * @param family column family to check + * @param qualifier column qualifier to check + * @param compareOp the comparison operator + * @param value the expected value + * @param mutation mutations to perform if check succeeds + * @return Future with true if the new put was executed, false otherwise + */ + Future checkAndMutate(byte[] family, byte[] qualifier, + CompareFilter.CompareOp compareOp, byte[] value, Mutation mutation); + + /** + * Atomically checks if a row/family/qualifier value matches the expected value. + * If it does, it performs the row mutations. If the passed value is null, the check + * is for the lack of column (ie: non-existence) + * + * It accepts a RowMutations object which contains multiple mutations on a single row. + * + * @param family column family to check + * @param qualifier column qualifier to check + * @param compareOp the comparison operator + * @param value the expected value + * @param mutations mutations to perform if check succeeds + * @return Future with true if the new put was executed, false otherwise + */ + Future checkAndMutate(byte[] family, byte[] qualifier, + CompareFilter.CompareOp compareOp, byte[] value, RowMutations mutations); + + /** + * Creates and returns a {@link com.google.protobuf.RpcChannel} instance connected to the + * table region containing the specified row. The row given does not actually have + * to exist. Whichever region would contain the row based on start and end keys will + * be used. Note that the {@code row} parameter is also not passed to the + * coprocessor handler registered for this protocol, unless the {@code row} + * is separately passed as an argument in the service request. The parameter + * here is only used to locate the region used to handle the call. + * + *

+ * The obtained {@link com.google.protobuf.RpcChannel} instance can be used to access a published + * coprocessor {@link com.google.protobuf.Service} using standard protobuf service invocations: + *

+ * + *
+ *
+   * CoprocessorRpcChannel channel = myTable.coprocessorService(rowkey);
+   * MyService.BlockingInterface service = MyService.newBlockingStub(channel);
+   * MyCallRequest request = MyCallRequest.newBuilder()
+   *     ...
+   *     .build();
+   * MyCallResponse response = service.myCall(null, request);
+   * 
+ * + * @param row The row key used to identify the remote region location + * @return A CoprocessorRpcChannel instance + */ + CoprocessorRpcChannel coprocessorService(byte[] row); + + /** + * Creates an instance of the given {@link com.google.protobuf.Service} subclass for each table + * region spanning the range from the {@code startKey} row to {@code endKey} row (inclusive), and + * invokes the passed {@link org.apache.hadoop.hbase.client.coprocessor.Batch.Call#call} method + * with each {@link com.google.protobuf.Service} instance. + * + * @param service the protocol buffer {@code Service} implementation to call + * @param startKey start region selection with region containing this row. If {@code null}, the + * selection will start with the first table region. + * @param endKey select regions up to and including the region containing this row. If {@code + * null}, selection will continue through the last table region. + * @param callable this instance's {@link org.apache.hadoop.hbase.client.coprocessor.Batch + * .Call#call} + * method will be invoked once per table region, using the {@link com.google.protobuf.Service} + * instance connected to that region. + * @param the {@link com.google.protobuf.Service} subclass to connect to + * @param Return type for the {@code callable} parameter's {@link + * org.apache.hadoop.hbase.client.coprocessor.Batch.Call#call} method + * @return Future with a map of result values keyed by region name + */ + Future> coprocessorService(final Class service, + byte[] startKey, byte[] endKey, final AsyncBatch.Call callable); + + /** + * Creates an instance of the given {@link com.google.protobuf.Service} subclass for each table + * region spanning the range from the {@code startKey} row to {@code endKey} row (inclusive), all + * the invocations to the same region server will be batched into one call. The coprocessor + * service is invoked according to the service instance, method name and parameters. + * + * @param methodDescriptor + * the descriptor for the protobuf service method to call. + * @param request + * the method call parameters + * @param startKey + * start region selection with region containing this row. If {@code null}, the + * selection will start with the first table region. + * @param endKey + * select regions up to and including the region containing this row. If {@code null}, + * selection will continue through the last table region. + * @param responsePrototype + * the proto type of the response of the method in Service. + * @param + * the response type for the coprocessor Service method + * @return Future with a map of result values keyed by region name + */ + Future> batchCoprocessorService( + Descriptors.MethodDescriptor methodDescriptor, Message request, + byte[] startKey, byte[] endKey, R responsePrototype); + + /** + * Creates an instance of the given {@link com.google.protobuf.Service} subclass for each table + * region spanning the range from the {@code startKey} row to {@code endKey} row (inclusive), all + * the invocations to the same region server will be batched into one call. The coprocessor + * service is invoked according to the service instance, method name and parameters. + * + *

+ * The given + * {@link org.apache.hadoop.hbase.client.coprocessor.Batch.Callback#update(byte[],byte[],Object)} + * method will be called with the return value from each region's invocation. + *

+ * + * @param methodDescriptor + * the descriptor for the protobuf service method to call. + * @param request + * the method call parameters + * @param startKey + * start region selection with region containing this row. If {@code null}, the + * selection will start with the first table region. + * @param endKey + * select regions up to and including the region containing this row. If {@code null}, + * selection will continue through the last table region. + * @param responsePrototype + * the proto type of the response of the method in Service. + * @param callback + * callback to invoke with the response for each region + * @param + * the response type for the coprocessor Service method + * @return Future with a map of result values keyed by region name + */ + Future batchCoprocessorService( + Descriptors.MethodDescriptor methodDescriptor, Message request, + byte[] startKey, byte[] endKey, R responsePrototype, Batch.Callback callback); +} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableImpl.java new file mode 100644 index 0000000..61481b1 --- /dev/null +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableImpl.java @@ -0,0 +1,770 @@ +/** + * + * 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 com.google.protobuf.Descriptors; +import com.google.protobuf.Message; +import com.google.protobuf.Service; +import io.netty.util.concurrent.GenericFutureListener; +import org.apache.commons.lang.NotImplementedException; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.CellScanner; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.HRegionLocation; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.classification.InterfaceStability; +import org.apache.hadoop.hbase.client.coprocessor.AsyncBatch; +import org.apache.hadoop.hbase.client.coprocessor.Batch; +import org.apache.hadoop.hbase.filter.BinaryComparator; +import org.apache.hadoop.hbase.filter.CompareFilter; +import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel; +import org.apache.hadoop.hbase.ipc.MessageConverter; +import org.apache.hadoop.hbase.ipc.RegionCoprocessorRpcChannel; +import org.apache.hadoop.hbase.protobuf.ProtobufUtil; +import org.apache.hadoop.hbase.protobuf.RequestConverter; +import org.apache.hadoop.hbase.protobuf.generated.ClientProtos; +import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos; +import org.apache.hadoop.hbase.util.Bytes; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * An implementation of {@link AsyncTable}. Used to communicate with a single HBase table. + * Lightweight. Get as needed and just close when done. + * Instances of this class SHOULD NOT be constructed directly. + * Obtain an instance via {@link Connection}. See {@link ConnectionFactory} + * class comment for an example of how. + * + *

AsyncTableImpl is not a client API. Use {@link AsyncTable} instead. It is marked + * InterfaceAudience.Private indicating that this is an HBase-internal class as defined in + * Hadoop + * Interface Classification + * There are no guarantees for backwards source / binary compatibility and methods or class can + * change or go away without deprecation. + * + * @see AsyncTable + * @see Connection + * @see ConnectionFactory + */ +@InterfaceAudience.Private +@InterfaceStability.Stable +public class AsyncTableImpl implements AsyncTable { + private static final Log LOG = LogFactory.getLog(AsyncTableImpl.class); + + protected static final Descriptors.MethodDescriptor GET_DESCRIPTOR = ClientProtos.ClientService + .getDescriptor().findMethodByName("Get"); + + protected static final Descriptors.MethodDescriptor MUTATE_DESCRIPTOR = ClientProtos.ClientService + .getDescriptor().findMethodByName("Mutate"); + + protected static final Descriptors.MethodDescriptor MULTI_DESCRIPTOR = ClientProtos.ClientService + .getDescriptor().findMethodByName("Multi"); + + public static final Descriptors.MethodDescriptor EXEC_SERVICE_DESCRIPTOR = ClientProtos + .ClientService.getDescriptor().findMethodByName("ExecService"); + + private final ClusterConnection connection; + private final Configuration configuration; + private final TableName tableName; + private final TableConfiguration tableConfiguration; + private final RpcRetryingCallerFactory rpcCallerFactory; + + private final int operationTimeout; + + private final Consistency defaultConsistency = Consistency.STRONG; + private final int scannerBatching; + private final long scannerMaxResultSize; + + /** + * Creates an object to access a HBase table. + * Used by HBase internally. DO NOT USE. See {@link ConnectionFactory} class comment for how to + * get a {@link Table} instance (use {@link Table} instead of {@link AsyncTableImpl}). + * @param tableName Name of the table. + * @param connection HConnection to be used. + * @param tableConfig config for the table + * @param rpcCallerFactory factory for RPC callers + * @throws IOException if a remote or network exception occurs + */ + @InterfaceAudience.Private + protected AsyncTableImpl(TableName tableName, final ClusterConnection connection, + final TableConfiguration tableConfig, final RpcRetryingCallerFactory rpcCallerFactory) + throws IOException { + if (connection == null || connection.isClosed()) { + throw new IllegalArgumentException("Connection is null or closed."); + } + this.tableName = tableName; + this.connection = connection; + this.configuration = connection.getConfiguration(); + + if (rpcCallerFactory == null) { + this.rpcCallerFactory = connection.getNewRpcRetryingCallerFactory(configuration); + } else { + this.rpcCallerFactory = rpcCallerFactory; + } + + if (tableConfig == null) { + tableConfiguration = new TableConfiguration(configuration); + } else { + this.tableConfiguration = tableConfig; + } + + this.scannerBatching = tableConfiguration.getScannerCaching(); + this.scannerMaxResultSize = tableConfiguration.getScannerMaxResultSize(); + + this.operationTimeout = tableName.isSystemTable() ? + tableConfiguration.getMetaOperationTimeout() : tableConfiguration.getOperationTimeout(); + } + + /** + * {@inheritDoc} + */ + @Override + public Configuration getConfiguration() { + return configuration; + } + + @Override + public TableName getName() { + return tableName; + } + + static final MessageConverter GET_RESPONSE_CONVERTER = + new MessageConverter() { + @Override + public Result convert(ClientProtos.GetResponse msg, CellScanner cellScanner) { + assert msg != null; + return ProtobufUtil.toResult(msg.getResult()); + } + }; + + static final MessageConverter MUTATE_CONVERTER = + new MessageConverter() { + @Override + public Result convert(ClientProtos.MutateResponse msg, CellScanner cellScanner) { + assert msg != null; + ClientProtos.Result r = msg.getResult(); + if(r == null){ + return null; + }else{ + return ProtobufUtil.toResult(r); + } + } + }; + + static final MessageConverter MUTATE_CHECK_CONVERTER = + new MessageConverter() { + @Override + public Boolean convert(ClientProtos.MutateResponse msg, CellScanner cellScanner) { + assert msg != null; + return msg.getProcessed(); + } + }; + + static final MessageConverter MULTI_CONVERTER = + new MessageConverter() { + @Override + public Void convert(ClientProtos.MultiResponse msg, CellScanner cellScanner) { + assert msg != null; + return null; + } + }; + + /** + * {@inheritDoc} + */ + @Override + public Future get(Get get) { + return this.get(get, GET_RESPONSE_CONVERTER); + } + + /** + * {@inheritDoc} + */ + @Override + public Future get(List gets) { + PromiseKeeper pk = new PromiseKeeper<>(connection.getEventExecutor(), + gets.size(), new Result[gets.size()]); + for(Get get : gets) { + pk.addFuture(this.get(get)); + } + return pk; + } + + @Override + public AsyncResultScanner getScanner(Scan scan) throws IOException { + if (scan.getBatch() > 0 && scan.isSmall()) { + throw new IllegalArgumentException("Small scan should not be used with batching"); + } + if (scan.getCaching() <= 0) { + scan.setCaching(scannerBatching); + } + if (scan.getMaxResultSize() <= 0) { + scan.setMaxResultSize(scannerMaxResultSize); + } + + if (scan.isReversed()) { + if (scan.isSmall()) { + return new AsyncClientSmallReversedScanner(getConfiguration(), scan, getName(), + this.connection, this.rpcCallerFactory, + tableConfiguration.getReplicaCallTimeoutMicroSecondScan()); + } else { + return new AsyncReversedClientScanner(getConfiguration(), scan, getName(), + this.connection, this.rpcCallerFactory, + tableConfiguration.getReplicaCallTimeoutMicroSecondScan()); + } + } + + if (scan.isSmall()) { + return new AsyncClientSmallScanner(getConfiguration(), scan, getName(), + this.connection, this.rpcCallerFactory, tableConfiguration + .getReplicaCallTimeoutMicroSecondScan()); + } else { + return new AsyncClientScanner(getConfiguration(), scan, getName(), this.connection, + this.rpcCallerFactory, tableConfiguration.getReplicaCallTimeoutMicroSecondScan()); + } + } + + /** + * Handles Get with optional response converter + * + * @param get The Get Request + * @param converter for the GetResponse + * @param Type of Response object expected + * @return ResponsePromise with type of value expected + * @see Get + */ + private Future get(final Get get, + final MessageConverter converter) { + if (get.getConsistency() == null){ + get.setConsistency(defaultConsistency); + } + + if (get.getConsistency() == Consistency.STRONG) { + final AsyncRegionServerCallable callable = new AsyncRegionServerCallable( + this.connection, getName(), get.getRow()) { + @Override + public Future call(int callTimeout) { + ClientProtos.GetRequest request; + try { + request = RequestConverter + .buildGetRequest(getLocation().getRegionInfo().getRegionName(), get); + } catch (IOException e) { + return new FailedFuture<>(getEventExecutor(),e); + } + + return getChannel().callMethod( + GET_DESCRIPTOR, + request, + null, + ClientProtos.GetResponse.getDefaultInstance(), + converter, + null, + callTimeout, + getPriority(tableName)); + } + }; + return rpcCallerFactory.newCaller().callAsyncWithRetries(callable, + this.operationTimeout); + } + + // Call that takes into account the replica + RpcRetryingCallerWithReadReplicas callable = new RpcRetryingCallerWithReadReplicas<>( + rpcCallerFactory, + tableName, this.connection, get, converter, + tableConfiguration.getRetriesNumber(), + operationTimeout, + tableConfiguration.getPrimaryCallTimeoutMicroSecond()); + return callable.call(); + } + + /** + * Handles Mutate (Put/Delete/Append/Increment) + * + * @param mutation The Mutate Request + * @return ResponsePromise with Result if Append or Increment + */ + @Override + public Future mutate(final Mutation mutation) { + final AsyncRegionServerCallable callable = new AsyncRegionServerCallable( + this.connection, getName(), mutation.getRow()) { + @Override + public Future call(int callTimeout) { + ClientProtos.MutateRequest request; + try { + if(mutation instanceof Put) { + validatePut((Put) mutation); + + request = RequestConverter + .buildMutateRequest( + getLocation().getRegionInfo().getRegionName(), (Put) mutation); + } else if(mutation instanceof Delete) { + request = RequestConverter + .buildMutateRequest( + getLocation().getRegionInfo().getRegionName(), (Delete) mutation); + } else if(mutation instanceof Increment) { + NonceGenerator ng = getConnection().getNonceGenerator(); + request = RequestConverter + .buildMutateRequest( + getLocation().getRegionInfo().getRegionName(), + (Increment) mutation, + ng.getNonceGroup(), + ng.newNonce() + ); + } else if(mutation instanceof Append) { + NonceGenerator ng = getConnection().getNonceGenerator(); + request = RequestConverter + .buildMutateRequest( + getLocation().getRegionInfo().getRegionName(), + (Append) mutation, + ng.getNonceGroup(), + ng.newNonce()); + }else{ + throw new NotImplementedException("Unknown mutation "+mutation.getClass().getName()); + } + } catch (IOException e) { + return new FailedFuture<>(getEventExecutor(),e); + } + + return getChannel().callMethod( + MUTATE_DESCRIPTOR, + request, + null, + ClientProtos.MutateResponse.getDefaultInstance(), MUTATE_CONVERTER, null, + callTimeout, + getPriority(tableName)); + } + }; + return rpcCallerFactory.newCaller().callAsyncWithRetries(callable, this + .operationTimeout); + + } + + @Override + public Future mutate(final RowMutations mutation) { + final AsyncRegionServerCallable callable = new AsyncRegionServerCallable( + this.connection, getName(), mutation.getRow()) { + @Override + public Future call(int callTimeout) { + ClientProtos.RegionAction.Builder regionMutationBuilder; + try { + regionMutationBuilder = RequestConverter + .buildRegionAction(getLocation().getRegionInfo().getRegionName(), mutation); + } catch (IOException e) { + return new FailedFuture<>(getEventExecutor(),e); + } + regionMutationBuilder.setAtomic(true); + ClientProtos.MultiRequest request = + ClientProtos.MultiRequest + .newBuilder().addRegionAction(regionMutationBuilder.build()).build(); + + return getChannel().callMethod( + MULTI_DESCRIPTOR, + request, + null, + ClientProtos.MultiResponse.getDefaultInstance(), MULTI_CONVERTER, null, + callTimeout, + getPriority(tableName)); + } + }; + return rpcCallerFactory.newCaller().callAsyncWithRetries(callable, this + .operationTimeout); + } + + @Override + public Future checkAndMutate(final byte[] family, + final byte[] qualifier, final CompareFilter.CompareOp compareOp, final byte[] value, + final Mutation mutation) { + final AsyncRegionServerCallable callable = new AsyncRegionServerCallable( + this.connection, getName(), mutation.getRow()) { + @Override + public Future call(int callTimeout) { + ClientProtos.MutateRequest request; + HBaseProtos.CompareType compareType = HBaseProtos.CompareType.valueOf(compareOp.name()); + BinaryComparator comparator = new BinaryComparator(value); + try { + if(mutation instanceof Put) { + validatePut((Put) mutation); + + request = RequestConverter + .buildMutateRequest( + getLocation().getRegionInfo().getRegionName(), + row, family, qualifier, comparator, compareType, (Put) mutation + ); + } else if(mutation instanceof Delete) { + request = RequestConverter + .buildMutateRequest( + getLocation().getRegionInfo().getRegionName(), + row, family, qualifier, comparator, compareType, (Delete) mutation + ); + } else if(mutation instanceof Increment) { + throw new RuntimeException( + "Mutation needs to be a Put or Delete for a checked mutation"); + } else if(mutation instanceof Append) { + throw new RuntimeException( + "Mutation needs to be a Put or Delete for a checked mutation"); + }else{ + throw new NotImplementedException("Unknown mutation "+mutation.getClass().getName()); + } + } catch (IOException e) { + return new FailedFuture<>(getEventExecutor(),e); + } + + return getChannel().callMethod( + MUTATE_DESCRIPTOR, + request, + null, + ClientProtos.MutateResponse.getDefaultInstance(), MUTATE_CHECK_CONVERTER, null, + callTimeout, + getPriority(tableName)); + } + }; + return rpcCallerFactory.newCaller().callAsyncWithRetries(callable, + this.operationTimeout); + } + + @Override + public Future checkAndMutate(final byte[] family, + final byte[] qualifier, final CompareFilter.CompareOp compareOp, final byte[] value, + final RowMutations mutations) { + final AsyncRegionServerCallable callable = new AsyncRegionServerCallable( + this.connection, getName(), mutations.getRow()) { + @Override + public Future call(int callTimeout) { + ClientProtos.MultiRequest request; + HBaseProtos.CompareType compareType = HBaseProtos.CompareType.valueOf(compareOp.name()); + BinaryComparator comparator = new BinaryComparator(value); + try { + request = RequestConverter + .buildMutateRequest( + getLocation().getRegionInfo().getRegionName(), + row, family, qualifier, comparator, compareType, mutations + ); + } catch (IOException e) { + return new FailedFuture<>(getEventExecutor(),e); + } + + return getChannel().callMethod( + MUTATE_DESCRIPTOR, + request, + null, + ClientProtos.MutateResponse.getDefaultInstance(), MUTATE_CHECK_CONVERTER, null, + callTimeout, + getPriority(tableName)); + } + }; + return rpcCallerFactory.newCaller().callAsyncWithRetries(callable, + this.operationTimeout); + } + + /** + * {@inheritDoc} + */ + @Override + public CoprocessorRpcChannel coprocessorService(byte[] row) { + return new RegionCoprocessorRpcChannel(connection, tableName, row); + } + + @Override + public Future> coprocessorService(final Class + service, byte[] startKey, byte[] endKey, final AsyncBatch.Call callable) { + // get regions covered by the row range + List keys; + try { + keys = getStartKeysInRange(startKey, endKey); + } catch (IOException e) { + return new FailedFuture<>(connection.getEventExecutor(), e); + } + + KeyedPromiseKeeper keyedPromiseKeeper = + new KeyedPromiseKeeper<>(connection.getEventExecutor(),Bytes.BYTES_COMPARATOR); + + for (final KeyAndRegion r : keys) { + final RegionCoprocessorRpcChannel channel = + new RegionCoprocessorRpcChannel(connection, tableName, r.getKey()); + + try { + T instance = ProtobufUtil.newServiceStub(service, channel); + keyedPromiseKeeper.addPromise( + channel.getLastRegion(), + callable.call(instance) + ); + } catch (Exception e) { + return new FailedFuture<>(connection.getEventExecutor(), new IOException(e)); + } + } + + return keyedPromiseKeeper; + } + + /** + * Get the start keys within range + * @param start start key + * @param end end key + * @return List of region names + * @throws IOException if fetching fails + */ + private List getStartKeysInRange(byte[] start, byte[] end) throws IOException { + if (start == null) { + start = HConstants.EMPTY_START_ROW; + } + if (end == null) { + end = HConstants.EMPTY_END_ROW; + } + return getKeysAndRegionsInRange(connection, tableName, start, end, true, false); + } + + @Override + public Future> batchCoprocessorService( + Descriptors.MethodDescriptor methodDescriptor, Message request, byte[] startKey, + byte[] endKey, R responsePrototype) { + + if (startKey == null) { + startKey = HConstants.EMPTY_START_ROW; + } + if (endKey == null) { + endKey = HConstants.EMPTY_END_ROW; + } + // get regions covered by the row range + List keysAndRegions; + try { + keysAndRegions = getKeysAndRegionsInRange(connection, tableName, startKey, endKey, true, + false); + } catch (IOException e) { + return new FailedFuture<>(connection.getEventExecutor(),e); + } + + KeyedPromiseKeeper keyedPromiseKeeper = + new KeyedPromiseKeeper<>(connection.getEventExecutor(),Bytes.BYTES_COMPARATOR); + + // check if we have any calls to make + if (keysAndRegions.isEmpty()) { + LOG.info("No regions were selected by key range start=" + Bytes.toStringBinary(startKey) + + ", end=" + Bytes.toStringBinary(endKey)); + keyedPromiseKeeper.setSuccess(new HashMap()); + return keyedPromiseKeeper; + } + + final MessageConverter converter = new MessageConverter() { + @Override + public R convert(R msg, CellScanner cellScanner) throws IOException { + return msg; + } + }; + + for (KeyAndRegion keyAndRegion : keysAndRegions) { + final byte[] region = keyAndRegion.getRegion().getRegionInfo().getRegionName(); + RegionCoprocessorServiceExec exec = + new RegionCoprocessorServiceExec( + region, keyAndRegion.getKey(), methodDescriptor, request); + + keyedPromiseKeeper.addPromise( + keyAndRegion.getKey(), + callCoprocessorService(exec, responsePrototype, converter)); + } + + return keyedPromiseKeeper; + } + + @Override + public Future batchCoprocessorService( + Descriptors.MethodDescriptor methodDescriptor, Message request, byte[] startKey, + byte[] endKey, R responsePrototype, final Batch.Callback callback) { + if (startKey == null) { + startKey = HConstants.EMPTY_START_ROW; + } + if (endKey == null) { + endKey = HConstants.EMPTY_END_ROW; + } + // get regions covered by the row range + List keysAndRegions; + try { + keysAndRegions = getKeysAndRegionsInRange(connection, tableName, startKey, endKey, true, + false); + } catch (IOException e) { + return new FailedFuture<>(connection.getEventExecutor(),e); + } + + // check if we have any calls to make + if (keysAndRegions.isEmpty()) { + LOG.info("No regions were selected by key range start=" + Bytes.toStringBinary(startKey) + + ", end=" + Bytes.toStringBinary(endKey)); + return new SuccessfulFuture<>(connection.getEventExecutor(), null); + } + + VoidPromiseKeeper promise = new VoidPromiseKeeper( + connection.getEventExecutor(), keysAndRegions.size()); + + final MessageConverter converter = new MessageConverter() { + @Override + public R convert(R msg, CellScanner cellScanner) throws IOException { + return msg; + } + }; + + for (final KeyAndRegion keyAndRegion : keysAndRegions) { + final byte[] region = keyAndRegion.getRegion().getRegionInfo().getRegionName(); + RegionCoprocessorServiceExec exec = + new RegionCoprocessorServiceExec( + region, keyAndRegion.getKey(), methodDescriptor, request); + + Future future = callCoprocessorService(exec, responsePrototype, converter); + future.addListener(new GenericFutureListener>() { + @Override + public void operationComplete(Future future) throws Exception { + callback.update(region, keyAndRegion.getKey(), future.getNow()); + } + }); + promise.addFuture(future); + } + + return promise; + } + + /** + * Call a single coprocessor call + * @param exec command to execute + * @param responsePrototype prototype of response + * @param converter to convert the Message + * @param Type of Response expected + * @return ResponsePromise for a single call + */ + private Future callCoprocessorService( + final RegionCoprocessorServiceExec exec, final R responsePrototype, + final MessageConverter converter) { + final AsyncRegionServerCallable callable = new AsyncRegionServerCallable( + this.connection, getName(), exec.getRow()) { + @Override + public Future call(int callTimeout) { + Message request = exec.getRequest(); + + return getChannel().callMethod(EXEC_SERVICE_DESCRIPTOR, request, null, responsePrototype, + converter, null, callTimeout, getPriority(tableName)); + } + }; + return rpcCallerFactory.newCaller().callAsyncWithRetries(callable, + this.operationTimeout); + } + + /** + * Get the corresponding start keys and regions for an arbitrary range of + * keys. + *

+ * @param startKey Starting row in range, inclusive + * @param endKey Ending row in range + * @param includeEndKey true if endRow is inclusive, false if exclusive + * @param reload true to reload information or false to use cached information + * @return A pair of list of start keys and list of HRegionLocations that + * contain the specified range + * @throws IOException if a remote or network exception occurs + */ + protected static List getKeysAndRegionsInRange(ClusterConnection connection, + TableName name, + final byte[] startKey, final byte[] endKey, final boolean includeEndKey, + final boolean reload) throws IOException { + final boolean endKeyIsEndOfTable = Bytes.equals(endKey, HConstants.EMPTY_END_ROW); + if ((Bytes.compareTo(startKey, endKey) > 0) && !endKeyIsEndOfTable) { + throw new IllegalArgumentException( + "Invalid range: " + Bytes.toStringBinary(startKey) + + " > " + Bytes.toStringBinary(endKey)); + } + List regionsInRange = new ArrayList<>(); + byte[] currentKey = startKey; + + try(RegionLocator regionLocator = connection.getRegionLocator(name)){ + HRegionLocation regionLocation; + do { + regionLocation = regionLocator.getRegionLocation(currentKey, reload); + regionsInRange.add(new KeyAndRegion(currentKey, regionLocation)); + currentKey = regionLocation.getRegionInfo().getEndKey(); + } while (!Bytes.equals(currentKey, HConstants.EMPTY_END_ROW) && (endKeyIsEndOfTable + || Bytes.compareTo(currentKey, endKey) < 0 + || (includeEndKey && Bytes.compareTo(currentKey, endKey) == 0))); + return regionsInRange; + } + } + + /** + * Object to store start key and region together + */ + protected static final class KeyAndRegion { + private final byte[] key; + private final HRegionLocation region; + + /** + * Constructor + * @param key where scan should start + * @param region to operate on + */ + private KeyAndRegion(byte[] key, HRegionLocation region) { + this.key = key; + this.region = region; + } + + /** + * Get the key to start region walk + * @return key + */ + public byte[] getKey() { + return key; + } + + /** + * Get region to walk + * @return region location + */ + public HRegionLocation getRegion() { + return region; + } + } + + /** + * Validates the Put + * @param put to validate + * @throws IllegalArgumentException if Put is invalid + */ + private void validatePut(final Put put) throws IllegalArgumentException { + HTable.validatePut(put, tableConfiguration.getMaxKeyValueSize()); + } + + /** + * Get priority for the tables + * @param tn TableName of table to get priority of + * @return priority + */ + public static int getPriority(TableName tn) { + return (tn != null && tn.isSystemTable())? HConstants.SYSTEMTABLE_QOS: HConstants.NORMAL_QOS; + } + + @Override + public String toString() { + return tableName + ";" + connection; + } + + @Override + public void close() throws IOException { + // Nothing to close at this moment + } +} \ No newline at end of file diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorImpl.java index 249edec..8e11253 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorImpl.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorImpl.java @@ -52,7 +52,7 @@ public class BufferedMutatorImpl implements BufferedMutator { protected ClusterConnection connection; // non-final so can be overridden in test private final TableName tableName; private volatile Configuration conf; - private List writeAsyncBuffer = new LinkedList<>(); + private final List writeAsyncBuffer = new LinkedList<>(); private long writeBufferSize; private final int maxKeyValueSize; protected long currentWriteBufferSize = 0; diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientAsyncPrefetchScanner.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientAsyncPrefetchScanner.java index f0903db..446a695 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientAsyncPrefetchScanner.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientAsyncPrefetchScanner.java @@ -20,7 +20,6 @@ package org.apache.hadoop.hbase.client; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.classification.InterfaceAudience; -import org.apache.hadoop.hbase.ipc.RpcControllerFactory; import org.apache.hadoop.hbase.util.Threads; import java.io.IOException; @@ -64,9 +63,8 @@ public class ClientAsyncPrefetchScanner extends ClientScanner { public ClientAsyncPrefetchScanner(Configuration configuration, Scan scan, TableName name, ClusterConnection connection, RpcRetryingCallerFactory rpcCallerFactory, - RpcControllerFactory rpcControllerFactory, ExecutorService pool, - int replicaCallTimeoutMicroSecondScan) throws IOException { - super(configuration, scan, name, connection, rpcCallerFactory, rpcControllerFactory, pool, + ExecutorService pool, int replicaCallTimeoutMicroSecondScan) throws IOException { + super(configuration, scan, name, connection, rpcCallerFactory, pool, replicaCallTimeoutMicroSecondScan); } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientScanner.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientScanner.java index 566bf4b..b2bd408 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientScanner.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientScanner.java @@ -32,7 +32,6 @@ import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.UnknownScannerException; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.exceptions.OutOfOrderScannerNextException; -import org.apache.hadoop.hbase.ipc.RpcControllerFactory; import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.generated.MapReduceProtos; import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException; @@ -45,6 +44,7 @@ import java.util.Arrays; import java.util.LinkedList; import java.util.List; import java.util.Queue; +import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; /** @@ -87,7 +87,6 @@ public abstract class ClientScanner extends AbstractClientScanner { protected final int scannerTimeout; protected boolean scanMetricsPublished = false; protected RpcRetryingCaller caller; - protected RpcControllerFactory rpcControllerFactory; protected Configuration conf; //The timeout on the primary. Applicable if there are multiple replicas for a region //In that case, we will only wait for this much timeout on the primary before going @@ -109,7 +108,7 @@ public abstract class ClientScanner extends AbstractClientScanner { */ public ClientScanner(final Configuration conf, final Scan scan, final TableName tableName, ClusterConnection connection, RpcRetryingCallerFactory rpcFactory, - RpcControllerFactory controllerFactory, ExecutorService pool, int primaryOperationTimeout) + ExecutorService pool, int primaryOperationTimeout) throws IOException { if (LOG.isTraceEnabled()) { LOG.trace("Scan table=" + tableName @@ -148,7 +147,6 @@ public abstract class ClientScanner extends AbstractClientScanner { } this.caller = rpcFactory. newCaller(); - this.rpcControllerFactory = controllerFactory; this.conf = conf; initCache(); @@ -306,9 +304,13 @@ public abstract class ClientScanner extends AbstractClientScanner { if (Thread.interrupted()) { throw new InterruptedIOException(); } - // callWithoutRetries is at this layer. Within the ScannerCallableWithReplicas, - // we do a callWithRetries - return caller.callWithoutRetries(callable, scannerTimeout); + try { + return caller.callAsyncWithRetries(callable, scannerTimeout).get(); + } catch (InterruptedException e) { + throw new InterruptedIOException(e.getMessage()); + } catch (ExecutionException e) { + throw (IOException) e.getCause(); + } } @InterfaceAudience.Private @@ -316,12 +318,11 @@ public abstract class ClientScanner extends AbstractClientScanner { int nbRows) { scan.setStartRow(localStartKey); ScannerCallable s = - new ScannerCallable(getConnection(), getTable(), scan, this.scanMetrics, - this.rpcControllerFactory); + new ScannerCallable(getConnection(), getTable(), scan, this.scanMetrics); s.setCaching(nbRows); ScannerCallableWithReplicas sr = new ScannerCallableWithReplicas(tableName, getConnection(), - s, pool, primaryOperationTimeout, scan, - retries, scannerTimeout, caching, conf, caller); + s, primaryOperationTimeout, scan, + scannerTimeout, caller); return sr; } @@ -473,7 +474,6 @@ public abstract class ClientScanner extends AbstractClientScanner { // Set this to zero so we don't try and do an rpc and close on remote server when // the exception we got was UnknownScanner or the Server is going down. callable = null; - // This continue will take us to while at end of loop where we will set up new scanner. continue; } @@ -775,7 +775,7 @@ public abstract class ClientScanner extends AbstractClientScanner { // do not return any rows, do not advance the scanner callable.setCaching(0); try { - this.caller.callWithoutRetries(callable, this.scannerTimeout); + this.caller.callAsyncWithoutRetries(callable, this.scannerTimeout).get(); } catch (Exception e) { return false; } finally { diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientSimpleScanner.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientSimpleScanner.java index 3998ac0..2cb0606 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientSimpleScanner.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientSimpleScanner.java @@ -36,9 +36,8 @@ public class ClientSimpleScanner extends ClientScanner { public ClientSimpleScanner(Configuration configuration, Scan scan, TableName name, ClusterConnection connection, RpcRetryingCallerFactory rpcCallerFactory, - RpcControllerFactory rpcControllerFactory, ExecutorService pool, - int replicaCallTimeoutMicroSecondScan) throws IOException { - super(configuration, scan, name, connection, rpcCallerFactory, rpcControllerFactory, pool, + ExecutorService pool, int replicaCallTimeoutMicroSecondScan) throws IOException { + super(configuration, scan, name, connection, rpcCallerFactory, pool, replicaCallTimeoutMicroSecondScan); } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientSmallReversedScanner.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientSmallReversedScanner.java index 28502dc..d9c88da 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientSmallReversedScanner.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientSmallReversedScanner.java @@ -21,6 +21,8 @@ package org.apache.hadoop.hbase.client; import java.io.IOException; +import java.io.InterruptedIOException; +import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; import org.apache.commons.logging.Log; @@ -65,8 +67,6 @@ public class ClientSmallReversedScanner extends ReversedClientScanner { * Connection identifying the cluster * @param rpcFactory * Factory used to create the {@link RpcRetryingCaller} - * @param controllerFactory - * Factory used to access RPC payloads * @param pool * Threadpool for RPC threads * @param primaryOperationTimeout @@ -76,9 +76,9 @@ public class ClientSmallReversedScanner extends ReversedClientScanner { */ public ClientSmallReversedScanner(final Configuration conf, final Scan scan, final TableName tableName, ClusterConnection connection, RpcRetryingCallerFactory rpcFactory, - RpcControllerFactory controllerFactory, ExecutorService pool, int primaryOperationTimeout) + ExecutorService pool, int primaryOperationTimeout) throws IOException { - this(conf, scan, tableName, connection, rpcFactory, controllerFactory, pool, + this(conf, scan, tableName, connection, rpcFactory, pool, primaryOperationTimeout, new SmallScannerCallableFactory()); } @@ -96,8 +96,6 @@ public class ClientSmallReversedScanner extends ReversedClientScanner { * Connection identifying the cluster * @param rpcFactory * Factory used to create the {@link RpcRetryingCaller} - * @param controllerFactory - * Factory used to access RPC payloads * @param pool * Threadpool for RPC threads * @param primaryOperationTimeout @@ -110,9 +108,9 @@ public class ClientSmallReversedScanner extends ReversedClientScanner { @VisibleForTesting ClientSmallReversedScanner(final Configuration conf, final Scan scan, final TableName tableName, ClusterConnection connection, RpcRetryingCallerFactory rpcFactory, - RpcControllerFactory controllerFactory, ExecutorService pool, int primaryOperationTimeout, + ExecutorService pool, int primaryOperationTimeout, SmallScannerCallableFactory callableFactory) throws IOException { - super(conf, scan, tableName, connection, rpcFactory, controllerFactory, pool, + super(conf, scan, tableName, connection, rpcFactory, pool, primaryOperationTimeout); this.callableFactory = callableFactory; } @@ -165,8 +163,8 @@ public class ClientSmallReversedScanner extends ReversedClientScanner { } smallScanCallable = callableFactory.getCallable(getConnection(), getTable(), scan, - getScanMetrics(), localStartKey, cacheNum, rpcControllerFactory, getPool(), - getPrimaryOperationTimeout(), getRetries(), getScannerTimeout(), getConf(), caller); + getScanMetrics(), localStartKey, cacheNum, getPrimaryOperationTimeout(), + getScannerTimeout(), caller); if (this.scanMetrics != null && regionChanged) { this.scanMetrics.countOfRegions.incrementAndGet(); @@ -208,7 +206,13 @@ public class ClientSmallReversedScanner extends ReversedClientScanner { // exhausted current region. // callWithoutRetries is at this layer. Within the ScannerCallableWithReplicas, // we do a callWithRetries - values = this.caller.callWithoutRetries(smallScanCallable, scannerTimeout); + try { + values = this.caller.callAsyncWithoutRetries(smallScanCallable, scannerTimeout).get(); + } catch (InterruptedException e) { + throw new InterruptedIOException(e.getMessage()); + } catch (ExecutionException e) { + throw (IOException) e.getCause(); + } this.currentRegion = smallScanCallable.getHRegionInfo(); long currentTime = System.currentTimeMillis(); if (this.scanMetrics != null) { diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientSmallScanner.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientSmallScanner.java index f9bdd55..a64b1a5 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientSmallScanner.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientSmallScanner.java @@ -19,18 +19,19 @@ package org.apache.hadoop.hbase.client; import com.google.common.annotations.VisibleForTesting; -import com.google.protobuf.ServiceException; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.CellScanner; import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.client.metrics.ScanMetrics; +import org.apache.hadoop.hbase.ipc.IOExceptionConverter; +import org.apache.hadoop.hbase.ipc.MessageConverter; import org.apache.hadoop.hbase.ipc.RpcControllerFactory; -import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.RequestConverter; import org.apache.hadoop.hbase.protobuf.ResponseConverter; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanRequest; @@ -39,6 +40,7 @@ import org.apache.hadoop.hbase.util.Bytes; import java.io.IOException; import java.io.InterruptedIOException; +import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; /** @@ -68,8 +70,6 @@ public class ClientSmallScanner extends ClientSimpleScanner { * Connection identifying the cluster * @param rpcFactory * Factory used to create the {@link RpcRetryingCaller} - * @param controllerFactory - * Factory used to access RPC payloads * @param pool * Threadpool for RPC threads * @param primaryOperationTimeout @@ -81,7 +81,7 @@ public class ClientSmallScanner extends ClientSimpleScanner { ClusterConnection connection, RpcRetryingCallerFactory rpcFactory, RpcControllerFactory controllerFactory, ExecutorService pool, int primaryOperationTimeout) throws IOException { - this(conf, scan, tableName, connection, rpcFactory, controllerFactory, pool, + this(conf, scan, tableName, connection, rpcFactory, pool, primaryOperationTimeout, new SmallScannerCallableFactory()); } @@ -100,8 +100,6 @@ public class ClientSmallScanner extends ClientSimpleScanner { * Connection identifying the cluster * @param rpcFactory * Factory used to create the {@link RpcRetryingCaller} - * @param controllerFactory - * Factory used to access RPC payloads * @param pool * Threadpool for RPC threads * @param primaryOperationTimeout @@ -113,9 +111,9 @@ public class ClientSmallScanner extends ClientSimpleScanner { @VisibleForTesting ClientSmallScanner(final Configuration conf, final Scan scan, final TableName tableName, ClusterConnection connection, RpcRetryingCallerFactory rpcFactory, - RpcControllerFactory controllerFactory, ExecutorService pool, int primaryOperationTimeout, + ExecutorService pool, int primaryOperationTimeout, SmallScannerCallableFactory callableFactory) throws IOException { - super(conf, scan, tableName, connection, rpcFactory, controllerFactory, pool, + super(conf, scan, tableName, connection, rpcFactory, pool, primaryOperationTimeout); this.callableFactory = callableFactory; } @@ -168,8 +166,8 @@ public class ClientSmallScanner extends ClientSimpleScanner { + Bytes.toStringBinary(localStartKey) + "'"); } smallScanCallable = callableFactory.getCallable(getConnection(), getTable(), scan, - getScanMetrics(), localStartKey, cacheNum, rpcControllerFactory, getPool(), - getPrimaryOperationTimeout(), getRetries(), getScannerTimeout(), getConf(), caller); + getScanMetrics(), localStartKey, cacheNum, + getPrimaryOperationTimeout(), getScannerTimeout(), caller); if (this.scanMetrics != null && regionChanged) { this.scanMetrics.countOfRegions.incrementAndGet(); } @@ -179,45 +177,51 @@ public class ClientSmallScanner extends ClientSimpleScanner { static class SmallScannerCallable extends ScannerCallable { public SmallScannerCallable( ClusterConnection connection, TableName table, Scan scan, - ScanMetrics scanMetrics, RpcControllerFactory controllerFactory, int caching, int id) { - super(connection, table, scan, scanMetrics, controllerFactory, id); + ScanMetrics scanMetrics, int caching, int id) { + super(connection, table, scan, scanMetrics, id); this.setCaching(caching); } @Override - public Result[] call(int timeout) throws IOException { - if (this.closed) return null; - if (Thread.interrupted()) { - throw new InterruptedIOException(); + public Future call(int timeout) { + if (this.closed) { + return null; } - ScanRequest request = RequestConverter.buildScanRequest(getLocation() - .getRegionInfo().getRegionName(), getScan(), getCaching(), true); - ScanResponse response = null; - controller = controllerFactory.newController(); + ScanRequest request; try { - controller.setPriority(getTableName()); - controller.setCallTimeout(timeout); - response = getStub().scan(controller, request); - Result[] results = ResponseConverter.getResults(controller.cellScanner(), - response); - if (response.hasMoreResultsInRegion()) { - setHasMoreResultsContext(true); - setServerHasMoreResults(response.getMoreResultsInRegion()); - } else { - setHasMoreResultsContext(false); - } - // We need to update result metrics since we are overriding call() - updateResultsMetrics(results); - return results; - } catch (ServiceException se) { - throw ProtobufUtil.getRemoteException(se); + request = RequestConverter + .buildScanRequest(getLocation().getRegionInfo().getRegionName(), getScan(), + getCaching(), true); + } catch (IOException e) { + return new FailedFuture<>(getEventExecutor(),e); } + + return sendRequest(request, timeout, new MessageConverter() { + @Override + public Result[] convert(ScanResponse response, CellScanner cellScanner) throws IOException { + Result[] results = ResponseConverter.getResults(cellScanner, response); + if (response.hasMoreResultsInRegion()) { + setHasMoreResultsContext(true); + setServerHasMoreResults(response.getMoreResultsInRegion()); + } else { + setHasMoreResultsContext(false); + } + // We need to update result metrics since we are overriding call() + updateResultsMetrics(results); + return results; + } + }, new IOExceptionConverter() { + @Override + public IOException convert(IOException e) { + return null; + } + }); } @Override public ScannerCallable getScannerCallableForReplica(int id) { return new SmallScannerCallable((ClusterConnection)connection, tableName, getScan(), - scanMetrics, controllerFactory, getCaching(), id); + scanMetrics, getCaching(), id); } } @@ -255,7 +259,13 @@ public class ClientSmallScanner extends ClientSimpleScanner { // exhausted current region. // callWithoutRetries is at this layer. Within the ScannerCallableWithReplicas, // we do a callWithRetries - values = this.caller.callWithoutRetries(smallScanCallable, scannerTimeout); + try { + values = this.caller.callAsyncWithoutRetries(smallScanCallable, scannerTimeout).get(); + } catch (InterruptedException e) { + throw new InterruptedIOException(e.getMessage()); + } catch (ExecutionException e) { + throw (IOException) e.getCause(); + } this.currentRegion = smallScanCallable.getHRegionInfo(); long currentTime = System.currentTimeMillis(); if (this.scanMetrics != null) { @@ -300,15 +310,15 @@ public class ClientSmallScanner extends ClientSimpleScanner { public ScannerCallableWithReplicas getCallable(ClusterConnection connection, TableName table, Scan scan, ScanMetrics scanMetrics, byte[] localStartKey, int cacheNum, - RpcControllerFactory controllerFactory, ExecutorService pool, int primaryOperationTimeout, - int retries, int scannerTimeout, Configuration conf, RpcRetryingCaller caller) { + int primaryOperationTimeout, int scannerTimeout, + RpcRetryingCaller caller) { scan.setStartRow(localStartKey); SmallScannerCallable s = new SmallScannerCallable( - connection, table, scan, scanMetrics, controllerFactory, cacheNum, 0); + connection, table, scan, scanMetrics, cacheNum, 0); ScannerCallableWithReplicas scannerCallableWithReplicas = new ScannerCallableWithReplicas(table, connection, - s, pool, primaryOperationTimeout, scan, retries, - scannerTimeout, cacheNum, conf, caller); + s, primaryOperationTimeout, scan, + scannerTimeout, caller); return scannerCallableWithReplicas; } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterConnection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterConnection.java index 07b055a..a13f852 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterConnection.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterConnection.java @@ -21,6 +21,8 @@ package org.apache.hadoop.hbase.client; import java.io.IOException; import java.util.List; +import io.netty.channel.EventLoop; +import io.netty.util.concurrent.EventExecutor; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.MasterNotRunningException; @@ -30,7 +32,9 @@ import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.ZooKeeperConnectionException; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.client.backoff.ClientBackoffPolicy; +import org.apache.hadoop.hbase.ipc.AsyncRpcChannel; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService; +import org.apache.hadoop.hbase.protobuf.generated.ClientProtos; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ClientService; import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MasterService; @@ -228,6 +232,15 @@ public interface ClusterConnection extends HConnection { ClientService.BlockingInterface getClient(final ServerName serverName) throws IOException; /** + * Get an Async RPC channel for Client communications + * + * @param serverName to connect to + * @return RpcChannel to communicate with server + * @throws IOException if a remote or network exception occurs + */ + AsyncRpcChannel getAsyncClientChannel(ServerName serverName) throws IOException; + + /** * Find region location hosting passed row * @param tableName table name * @param row Row to find. @@ -286,7 +299,7 @@ public interface ClusterConnection extends HConnection { * @return RpcRetryingCallerFactory */ RpcRetryingCallerFactory getNewRpcRetryingCallerFactory(Configuration conf); - + /** * @return the current statistics tracker associated with this connection */ @@ -296,4 +309,9 @@ public interface ClusterConnection extends HConnection { * @return the configured client backoff policy */ ClientBackoffPolicy getBackoffPolicy(); + + /** + * @return the eventExecutor for this ClusterConnection + */ + EventExecutor getEventExecutor(); } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Connection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Connection.java index dab4905..67da677 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Connection.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Connection.java @@ -86,6 +86,25 @@ public interface Connection extends Abortable, Closeable { */ Table getTable(TableName tableName) throws IOException; + + /** + * Retrieve an Async Table implementation for accessing a table. + * The returned Table is not thread safe, a new instance should be created for each using thread. + * This is a lightweight operation, pooling or caching of the returned Table + * is neither required nor desired. + *

+ * The caller is responsible for calling {@link Table#close()} on the returned + * table instance. + *

+ * This method does not check table existence. An exception + * will be thrown if the table does not exist only when the first operation is + * attempted. + * @param tableName the name of the table + * @return a Table to use for interactions with this table + */ + AsyncTable getAsyncTable(TableName tableName) throws IOException; + + /** * Retrieve a Table implementation for accessing a table. * The returned Table is not thread safe, a new instance should be created for each using thread. diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionAdapter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionAdapter.java index 1d8a793..b1581cb 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionAdapter.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionAdapter.java @@ -21,6 +21,8 @@ import java.io.IOException; import java.util.List; import java.util.concurrent.ExecutorService; +import io.netty.channel.EventLoop; +import io.netty.util.concurrent.EventExecutor; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.HTableDescriptor; @@ -32,6 +34,7 @@ import org.apache.hadoop.hbase.ZooKeeperConnectionException; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.client.backoff.ClientBackoffPolicy; import org.apache.hadoop.hbase.client.coprocessor.Batch.Callback; +import org.apache.hadoop.hbase.ipc.AsyncRpcChannel; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ClientService; import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MasterService; @@ -92,6 +95,11 @@ abstract class ConnectionAdapter implements ClusterConnection { } @Override + public AsyncTable getAsyncTable(TableName tableName) throws IOException { + return wrappedConnection.getAsyncTable(tableName); + } + + @Override public HTableInterface getTable(String tableName, ExecutorService pool) throws IOException { return wrappedConnection.getTable(tableName, pool); @@ -337,6 +345,11 @@ abstract class ConnectionAdapter implements ClusterConnection { } @Override + public AsyncRpcChannel getAsyncClientChannel(ServerName serverName) throws IOException { + return wrappedConnection.getAsyncClientChannel(serverName); + } + + @Override public AdminService.BlockingInterface getAdmin( ServerName serverName, boolean getMaster) throws IOException { return wrappedConnection.getAdmin(serverName, getMaster); @@ -464,4 +477,9 @@ abstract class ConnectionAdapter implements ClusterConnection { public ClientBackoffPolicy getBackoffPolicy() { return wrappedConnection.getBackoffPolicy(); } + + @Override + public EventExecutor getEventExecutor() { + return wrappedConnection.getEventExecutor(); + } } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java index 35ff34f..3e22056 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java @@ -23,6 +23,7 @@ import com.google.protobuf.BlockingRpcChannel; import com.google.protobuf.RpcController; import com.google.protobuf.ServiceException; +import io.netty.util.concurrent.EventExecutor; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; @@ -46,6 +47,7 @@ import org.apache.hadoop.hbase.client.backoff.ClientBackoffPolicyFactory; import org.apache.hadoop.hbase.client.coprocessor.Batch; import org.apache.hadoop.hbase.exceptions.RegionMovedException; import org.apache.hadoop.hbase.exceptions.RegionOpeningException; +import org.apache.hadoop.hbase.ipc.AsyncRpcChannel; import org.apache.hadoop.hbase.ipc.RpcClient; import org.apache.hadoop.hbase.ipc.RpcClientFactory; import org.apache.hadoop.hbase.ipc.RpcControllerFactory; @@ -314,6 +316,11 @@ class ConnectionImplementation implements ClusterConnection, Closeable { } @Override + public AsyncTable getAsyncTable(TableName tableName) throws IOException { + return new AsyncTableImpl(tableName,this,tableConfig,rpcCallerFactory); + } + + @Override public HTableInterface getTable(String tableName, ExecutorService pool) throws IOException { return getTable(TableName.valueOf(tableName), pool); } @@ -841,7 +848,7 @@ class ConnectionImplementation implements ClusterConnection, Closeable { ReversedClientScanner rcs = null; try { rcs = new ClientSmallReversedScanner(conf, s, TableName.META_TABLE_NAME, this, - rpcCallerFactory, rpcControllerFactory, getMetaLookupPool(), 0); + rpcCallerFactory, getMetaLookupPool(), 0); regionInfoRow = rcs.next(); } finally { if (rcs != null) { @@ -1308,6 +1315,36 @@ class ConnectionImplementation implements ClusterConnection, Closeable { return stub; } + @Override + public AsyncRpcChannel getAsyncClientChannel(final ServerName sn) throws IOException { + return this.getAsyncRpcChannel(ClientProtos.ClientService.getDescriptor().getName(),sn); + } + + /** + * Get an async rpc channel + * @param serviceName to get channel for + * @param serverName + * @return AsyncRpcChannel to communicate with + * @throws IOException + */ + private AsyncRpcChannel getAsyncRpcChannel(String serviceName, final ServerName serverName) + throws IOException { + if (isDeadServer(serverName)) { + throw new RegionServerStoppedException(serverName + " is dead."); + } + String key = getStubKey(AsyncRpcChannel.class.getName(), serverName.getHostname(), serverName.getPort()); + this.connectionLock.putIfAbsent(key, key); + AsyncRpcChannel channel; + synchronized (this.connectionLock.get(key)) { + channel = (AsyncRpcChannel)this.stubs.get(key); + if (channel == null) { + channel = this.rpcClient.createRpcChannel(serviceName, serverName, user); + this.stubs.put(key, channel); + } + } + return channel; + } + static String getStubKey(final String serviceName, final String rsHostname, int port) { // Sometimes, servers go down and they come back up with the same hostname but a different // IP address. Force a resolution of the rsHostname by trying to instantiate an @@ -1383,7 +1420,6 @@ class ConnectionImplementation implements ClusterConnection, Closeable { mss.userCount++; } - @Override public MasterKeepAliveConnection getKeepAliveMasterService() throws MasterNotRunningException { synchronized (masterAndZKLock) { @@ -1793,8 +1829,8 @@ class ConnectionImplementation implements ClusterConnection, Closeable { public void updateCachedLocations(final TableName tableName, byte[] rowkey, final Object exception, final HRegionLocation source) { assert source != null; - updateCachedLocations(tableName, source.getRegionInfo().getRegionName() - , rowkey, exception, source.getServerName()); + updateCachedLocations(tableName, source.getRegionInfo().getRegionName(), rowkey, exception, + source.getServerName()); } /** @@ -1965,6 +2001,11 @@ class ConnectionImplementation implements ClusterConnection, Closeable { return this.backoffPolicy; } + @Override + public EventExecutor getEventExecutor() { + return this.rpcClient.getEventExecutor(); + } + /* * Return the number of cached region for a table. It will only be called * from a unit test. diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionUtils.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionUtils.java index 323915b..b9611b1 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionUtils.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionUtils.java @@ -28,6 +28,7 @@ import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService; +import org.apache.hadoop.hbase.protobuf.generated.ClientProtos; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ClientService; import org.apache.hadoop.hbase.security.User; diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/DelayedPromise.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/DelayedPromise.java new file mode 100644 index 0000000..a2e1496 --- /dev/null +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/DelayedPromise.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 io.netty.util.concurrent.EventExecutor; +import io.netty.util.concurrent.GenericFutureListener; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.ipc.Promise; + +/** + * Wrapper for response promise + * @param Type of Result for Promise + */ +@InterfaceAudience.Private +public class DelayedPromise extends Promise { + + /** + * Constructor + * + * @param eventLoop to handle events on + */ + public DelayedPromise(EventExecutor eventLoop) { + super(eventLoop); + } + + /** + * Wrap given promise + * @param future to wrap into this promise + */ + public void wrapFuture(Future future) { + if(this.isCancelled()){ + return; + } + + future.addListener(new GenericFutureListener>() { + @Override + public void operationComplete(Future future) throws Exception { + if (future.isSuccess()) { + setSuccess(future.getNow()); + } else if (future.isCancelled()) { + cancel(true); + } else { + setFailure(future.cause()); + } + } + }); + } +} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/FailedFuture.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/FailedFuture.java new file mode 100644 index 0000000..5ebfd08 --- /dev/null +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/FailedFuture.java @@ -0,0 +1,74 @@ +/** + * 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 io.netty.util.concurrent.CompleteFuture; +import io.netty.util.concurrent.EventExecutor; +import io.netty.util.internal.PlatformDependent; +import org.apache.hadoop.hbase.classification.InterfaceAudience; + +/** + * A Failed Response future + * @param Value type for Future + */ +@InterfaceAudience.Private +public final class FailedFuture extends CompleteFuture implements Future { + + private final Throwable cause; + + /** + * Creates a new instance. + * + * @param executor the {@link EventExecutor} associated with this future + * @param cause the cause of failure + */ + public FailedFuture(EventExecutor executor, Throwable cause) { + super(executor); + if (cause == null) { + throw new NullPointerException("cause"); + } + this.cause = cause; + } + + @Override + public Throwable cause() { + return cause; + } + + @Override + public boolean isSuccess() { + return false; + } + + @Override + public io.netty.util.concurrent.Future sync() { + PlatformDependent.throwException(cause); + return this; + } + + @Override + public io.netty.util.concurrent.Future syncUninterruptibly() { + PlatformDependent.throwException(cause); + return this; + } + + @Override + public V getNow() { + return null; + } +} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/FastFailInterceptorContext.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/FastFailInterceptorContext.java index 9eb56bc..2a01e49 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/FastFailInterceptorContext.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/FastFailInterceptorContext.java @@ -111,7 +111,7 @@ class FastFailInterceptorContext extends return prepare(callable, 0); } - public FastFailInterceptorContext prepare(RetryingCallable callable, + public FastFailInterceptorContext prepare(ProtoRetryingCallable callable, int tries) { if (callable instanceof RegionServerCallable) { RegionServerCallable retryingCallable = (RegionServerCallable) callable; diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Future.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Future.java new file mode 100644 index 0000000..960f6bd --- /dev/null +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Future.java @@ -0,0 +1,29 @@ +/** + * 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.classification.InterfaceAudience; + +/** + * Promise for responses + * @param Value type + */ +@InterfaceAudience.Public +public interface Future extends io.netty.util.concurrent.Future { + +} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java index 6ba0b87..db4083b 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java @@ -628,11 +628,11 @@ public class HTable implements HTableInterface { if (scan.isReversed()) { if (scan.isSmall()) { return new ClientSmallReversedScanner(getConfiguration(), scan, getName(), - this.connection, this.rpcCallerFactory, this.rpcControllerFactory, + this.connection, this.rpcCallerFactory, pool, tableConfiguration.getReplicaCallTimeoutMicroSecondScan()); } else { return new ReversedClientScanner(getConfiguration(), scan, getName(), - this.connection, this.rpcCallerFactory, this.rpcControllerFactory, + this.connection, this.rpcCallerFactory, pool, tableConfiguration.getReplicaCallTimeoutMicroSecondScan()); } } @@ -644,12 +644,10 @@ public class HTable implements HTableInterface { } else { if (async) { return new ClientAsyncPrefetchScanner(getConfiguration(), scan, getName(), this.connection, - this.rpcCallerFactory, this.rpcControllerFactory, - pool, tableConfiguration.getReplicaCallTimeoutMicroSecondScan()); + this.rpcCallerFactory, pool, tableConfiguration.getReplicaCallTimeoutMicroSecondScan()); } else { return new ClientSimpleScanner(getConfiguration(), scan, getName(), this.connection, - this.rpcCallerFactory, this.rpcControllerFactory, - pool, tableConfiguration.getReplicaCallTimeoutMicroSecondScan()); + this.rpcCallerFactory, pool, tableConfiguration.getReplicaCallTimeoutMicroSecondScan()); } } } @@ -710,12 +708,17 @@ public class HTable implements HTableInterface { } // Call that takes into account the replica - RpcRetryingCallerWithReadReplicas callable = new RpcRetryingCallerWithReadReplicas( - rpcControllerFactory, tableName, this.connection, get, pool, - tableConfiguration.getRetriesNumber(), - operationTimeout, - tableConfiguration.getPrimaryCallTimeoutMicroSecond()); - return callable.call(); + RpcRetryingCallerWithReadReplicas callable = new RpcRetryingCallerWithReadReplicas<>( + rpcCallerFactory, tableName, this.connection, get, AsyncTableImpl.GET_RESPONSE_CONVERTER, + tableConfiguration.getRetriesNumber(), operationTimeout, + tableConfiguration.getPrimaryCallTimeoutMicroSecond()); + try { + return callable.call().get(); + } catch (InterruptedException e) { + throw new IOException(e); + } catch (ExecutionException e) { + throw (IOException) e.getCause(); + } } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/KeyedPromiseKeeper.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/KeyedPromiseKeeper.java new file mode 100644 index 0000000..49a251e --- /dev/null +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/KeyedPromiseKeeper.java @@ -0,0 +1,105 @@ +/** + * 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 io.netty.util.concurrent.DefaultPromise; +import io.netty.util.concurrent.EventExecutor; +import org.apache.hadoop.hbase.classification.InterfaceAudience; + +import java.util.Comparator; +import java.util.HashMap; +import java.util.Map; +import java.util.TreeMap; +import java.util.concurrent.ExecutionException; + +/** + * Keeps all promises which are added to this promise + * @param Type of result returned by promise + */ +@InterfaceAudience.Private +public class KeyedPromiseKeeper extends DefaultPromise> implements Future> { + + private final Map> promises; + private final Map responses; + + /** + * Constructor + * @param executor to run promise keeper on + */ + public KeyedPromiseKeeper(EventExecutor executor, Comparator comparator) { + super(executor); + + this.promises = new HashMap<>(); + this.responses = new TreeMap<>(comparator); + } + + /** + * Add a promise to the single promise listener + * @param promise to listen to + */ + public void addPromise(K key, Future promise){ + this.promises.put(key,promise); + promise.addListener(new KeyedResponseFuture(key)); + } + + /** + * Cancel all open promises + */ + private void cancelAllPromises() { + for(Future p : promises.values()){ + if(p != null) { + p.cancel(true); + } + } + } + + /** + * Indexed response promise listener + */ + private class KeyedResponseFuture implements ResponseFutureListener { + private final K key; + + /** + * Constructor + * @param i index for the response + */ + public KeyedResponseFuture(K i) { + this.key = i; + } + + @Override + public void operationComplete(Future future) { + if(!isDone()) { + // Set promise ref to be null to indicate it is fulfulled + promises.remove(this.key); + try { + responses.put(this.key,future.get()); + if(promises.isEmpty()){ + setSuccess(responses); + } + } catch (InterruptedException e) { + cancel(true); + cancelAllPromises(); + } catch (ExecutionException e) { + setFailure(e.getCause()); + cancelAllPromises(); + } + } + } + } +} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/NoOpRetryingInterceptorContext.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/NoOpRetryingInterceptorContext.java index 1ccf43c..8d0183c 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/NoOpRetryingInterceptorContext.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/NoOpRetryingInterceptorContext.java @@ -36,7 +36,7 @@ class NoOpRetryingInterceptorContext extends RetryingCallerInterceptorContext { @Override public RetryingCallerInterceptorContext prepare( - RetryingCallable callable, int tries) { + ProtoRetryingCallable callable, int tries) { // Do Nothing return this; } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/PromiseKeeper.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/PromiseKeeper.java new file mode 100644 index 0000000..44f7095 --- /dev/null +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/PromiseKeeper.java @@ -0,0 +1,108 @@ +/** + * 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 io.netty.util.concurrent.DefaultPromise; +import io.netty.util.concurrent.EventExecutor; +import org.apache.hadoop.hbase.classification.InterfaceAudience; + +import java.util.concurrent.ExecutionException; + +/** + * Keeps all promises which are added to this promise + * @param Type of result returned by promise + */ +@InterfaceAudience.Private +public class PromiseKeeper extends DefaultPromise implements Future { + + private int indexCounter = 0; + private int totalCount = 0; + private int doneCount = 0; + private final Future[] promises; + private final R[] responses; + + /** + * Constructor + * @param executor to run promise keeper on + * @param maxPromises max amount of promises to be able to listen to + * @param responses the empty responses array + */ + public PromiseKeeper(EventExecutor executor, int maxPromises, R[] responses) { + super(executor); + this.promises = new Future[maxPromises]; + this.responses = responses; + } + + /** + * Add a promise to the single promise listener + * @param promise to listen to + */ + public void addFuture(Future promise){ + totalCount++; + int i = indexCounter++; + this.promises[i] = promise; + promise.addListener(new IndexedResponseFuture(i)); + } + + /** + * Cancel all open promises + */ + private void cancelAllPromises() { + for(Future p : promises){ + if(p != null) { + p.cancel(true); + } + } + } + + /** + * Indexed response promise listener + */ + private class IndexedResponseFuture implements ResponseFutureListener { + private final int index; + + /** + * Constructor + * @param i index for the response + */ + public IndexedResponseFuture(int i) { + this.index = i; + } + + @Override + public void operationComplete(Future future) { + if(!isDone()) { + // Set promise ref to be null to indicate it is fulfulled + promises[index] = null; + try { + responses[index] = future.get(); + doneCount++; + if(doneCount == totalCount){ + setSuccess(responses); + } + } catch (InterruptedException e) { + cancel(true); + cancelAllPromises(); + } catch (ExecutionException e) { + setFailure(e.getCause()); + cancelAllPromises(); + } + } + } + } +} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ProtoRetryingCallable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ProtoRetryingCallable.java new file mode 100644 index 0000000..57eac7d --- /dev/null +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ProtoRetryingCallable.java @@ -0,0 +1,62 @@ +/** + * + * 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.classification.InterfaceAudience; + +import java.io.IOException; + +/** + * All generic methods for a Callable that can be retried. It is extended with Sync and + * Async versions. + * @param + */ +@InterfaceAudience.Private +public interface ProtoRetryingCallable { + /** + * Prepare by setting up any connections to servers, etc., ahead of call invocation. + * @param reload Set this to true if need to requery locations + * @throws IOException e + */ + void prepare(final boolean reload) throws IOException; + + /** + * Called when call throws an exception and we are going to retry; take action to + * make it so we succeed on next call (clear caches, do relookup of locations, etc.). + * @param t + * @param retrying True if we are in retrying mode (we are not in retrying mode when max + * retries == 1; we ARE in retrying mode if retries > 1 even when we are the last attempt) + */ + void throwable(final Throwable t, boolean retrying); + + /** + * @return Some details from the implementation that we would like to add to a terminating + * exception; i.e. a fatal exception is being thrown ending retries and we might like to add + * more implementation-specific detail on to the exception being thrown. + */ + String getExceptionMessageAdditionalDetail(); + + /** + * @param pause + * @param tries + * @return Suggestion on how much to sleep between retries + */ + long sleep(final long pause, final int tries); +} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionServerCallable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionServerCallable.java index 9989d56..fc7282c 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionServerCallable.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionServerCallable.java @@ -20,19 +20,10 @@ package org.apache.hadoop.hbase.client; import java.io.IOException; -import java.net.ConnectException; -import java.net.SocketTimeoutException; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.hbase.HRegionLocation; -import org.apache.hadoop.hbase.NotServingRegionException; +import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.classification.InterfaceAudience; -import org.apache.hadoop.hbase.exceptions.RegionMovedException; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ClientService; -import org.apache.hadoop.hbase.util.Bytes; /** * Implementations call a RegionServer and implement {@link #call(int)}. @@ -45,16 +36,11 @@ import org.apache.hadoop.hbase.util.Bytes; * @param the class that the ServerCallable handles */ @InterfaceAudience.Private -public abstract class RegionServerCallable implements RetryingCallable { +public abstract class RegionServerCallable extends AbstractRegionServerCallable + implements RetryingCallable { // Public because used outside of this package over in ipc. - private static final Log LOG = LogFactory.getLog(RegionServerCallable.class); - protected final Connection connection; - protected final TableName tableName; - protected final byte[] row; - protected HRegionLocation location; - private ClientService.BlockingInterface stub; - protected final static int MIN_WAIT_DEAD_SERVER = 10000; + private ClientService.BlockingInterface stub; /** * @param connection Connection to use. @@ -62,102 +48,26 @@ public abstract class RegionServerCallable implements RetryingCallable { * @param row The row we want in tableName. */ public RegionServerCallable(Connection connection, TableName tableName, byte [] row) { - this.connection = connection; - this.tableName = tableName; - this.row = row; + super(connection, tableName, row); } - /** - * Prepare for connection to the server hosting region with row from tablename. Does lookup - * to find region location and hosting server. - * @param reload Set this to true if connection should re-find the region - * @throws IOException e - */ @Override - public void prepare(final boolean reload) throws IOException { - try (RegionLocator regionLocator = connection.getRegionLocator(tableName)) { - this.location = regionLocator.getRegionLocation(row, reload); - } - if (this.location == null) { - throw new IOException("Failed to find location, tableName=" + tableName + - ", row=" + Bytes.toString(row) + ", reload=" + reload); - } - setStub(getConnection().getClient(this.location.getServerName())); + void setClientByServiceName(ServerName service) throws IOException { + this.setStub(getConnection().getClient(service)); } /** - * @return {@link HConnection} instance used by this Callable. + * @return Client Rpc protobuf communication stub */ - HConnection getConnection() { - return (HConnection) this.connection; - } - protected ClientService.BlockingInterface getStub() { return this.stub; } - void setStub(final ClientService.BlockingInterface stub) { - this.stub = stub; - } - - protected HRegionLocation getLocation() { - return this.location; - } - - protected void setLocation(final HRegionLocation location) { - this.location = location; - } - - public TableName getTableName() { - return this.tableName; - } - - public byte [] getRow() { - return this.row; - } - - @Override - public void throwable(Throwable t, boolean retrying) { - if (t instanceof SocketTimeoutException || - t instanceof ConnectException || - t instanceof RetriesExhaustedException || - (location != null && getConnection().isDeadServer(location.getServerName()))) { - // if thrown these exceptions, we clear all the cache entries that - // map to that slow/dead server; otherwise, let cache miss and ask - // hbase:meta again to find the new location - if (this.location != null) getConnection().clearCaches(location.getServerName()); - } else if (t instanceof RegionMovedException) { - getConnection().updateCachedLocations(tableName, row, t, location); - } else if (t instanceof NotServingRegionException && !retrying) { - // Purge cache entries for this specific region from hbase:meta cache - // since we don't call connect(true) when number of retries is 1. - getConnection().deleteCachedRegionLocation(location); - } - } - - @Override - public String getExceptionMessageAdditionalDetail() { - return "row '" + Bytes.toString(row) + "' on table '" + tableName + "' at " + location; - } - - @Override - public long sleep(long pause, int tries) { - // Tries hasn't been bumped up yet so we use "tries + 1" to get right pause time - long sleep = ConnectionUtils.getPauseTime(pause, tries + 1); - if (sleep < MIN_WAIT_DEAD_SERVER - && (location == null || getConnection().isDeadServer(location.getServerName()))) { - sleep = ConnectionUtils.addJitter(MIN_WAIT_DEAD_SERVER, 0.10f); - } - return sleep; - } - /** - * @return the HRegionInfo for the current region + * Set the client protobuf communication stub + * @param stub to set */ - public HRegionInfo getHRegionInfo() { - if (this.location == null) { - return null; - } - return this.location.getRegionInfo(); + void setStub(final ClientService.BlockingInterface stub) { + this.stub = stub; } -} +} \ No newline at end of file diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ResponseFutureListener.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ResponseFutureListener.java new file mode 100644 index 0000000..dcc2582 --- /dev/null +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ResponseFutureListener.java @@ -0,0 +1,30 @@ +/** + * 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 io.netty.util.concurrent.GenericFutureListener; +import org.apache.hadoop.hbase.classification.InterfaceAudience; + +/** + * Specific interface for the Response future listener + * @param Value type. + */ +@InterfaceAudience.Private +public interface ResponseFutureListener + extends GenericFutureListener> { +} \ No newline at end of file diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RetryingCallable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RetryingCallable.java index e468d3c..fca9c33 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RetryingCallable.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RetryingCallable.java @@ -19,8 +19,6 @@ package org.apache.hadoop.hbase.client; -import java.io.IOException; - import org.apache.hadoop.hbase.classification.InterfaceAudience; /** @@ -29,23 +27,7 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience; * @param */ @InterfaceAudience.Private -public interface RetryingCallable { - /** - * Prepare by setting up any connections to servers, etc., ahead of {@link #call(int)} invocation. - * @param reload Set this to true if need to requery locations - * @throws IOException e - */ - void prepare(final boolean reload) throws IOException; - - /** - * Called when {@link #call(int)} throws an exception and we are going to retry; take action to - * make it so we succeed on next call (clear caches, do relookup of locations, etc.). - * @param t - * @param retrying True if we are in retrying mode (we are not in retrying mode when max - * retries == 1; we ARE in retrying mode if retries > 1 even when we are the last attempt) - */ - void throwable(final Throwable t, boolean retrying); - +public interface RetryingCallable extends ProtoRetryingCallable { /** * Computes a result, or throws an exception if unable to do so. * @@ -54,18 +36,4 @@ public interface RetryingCallable { * @throws Exception if unable to compute a result */ T call(int callTimeout) throws Exception; - - /** - * @return Some details from the implementation that we would like to add to a terminating - * exception; i.e. a fatal exception is being thrown ending retries and we might like to add - * more implementation-specific detail on to the exception being thrown. - */ - String getExceptionMessageAdditionalDetail(); - - /** - * @param pause - * @param tries - * @return Suggestion on how much to sleep between retries - */ - long sleep(final long pause, final int tries); } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RetryingCallerInterceptorContext.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RetryingCallerInterceptorContext.java index a9f414f..f044416 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RetryingCallerInterceptorContext.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RetryingCallerInterceptorContext.java @@ -65,5 +65,5 @@ abstract class RetryingCallerInterceptorContext { * retrying call */ public abstract RetryingCallerInterceptorContext prepare( - RetryingCallable callable, int tries); + ProtoRetryingCallable callable, int tries); } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RetryingPromise.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RetryingPromise.java new file mode 100644 index 0000000..fc071f8 --- /dev/null +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RetryingPromise.java @@ -0,0 +1,112 @@ +/** + * 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 io.netty.channel.EventLoop; +import io.netty.util.concurrent.EventExecutor; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.ipc.Promise; + +/** + * Retrying response future + * @param Type of object expected in the Future + */ +@InterfaceAudience.Private +public class RetryingPromise extends Promise { + private final TryHandler tryHandler; + private int tries = 0; + private Future currentPromise; + + /** + * Constructor + * @param executor for the Future + * @param tryHandler Handling the call and failure + */ + public RetryingPromise(EventExecutor executor, TryHandler tryHandler) { + super(executor); + this.tryHandler = tryHandler; + } + + @Override + public boolean cancel(boolean mayInterupt){ + if (this.currentPromise != null && !currentPromise.isCancelled()){ + this.currentPromise.cancel(mayInterupt); + } + return super.cancel(mayInterupt); + } + + /** + * Do the actual call for the try. + */ + public void call() { + if (this.isCancelled()){ + return; + } + + this.currentPromise = this.tryHandler.call(tries); + + currentPromise.addListener(new ResponseFutureListener() { + @Override + public void operationComplete(Future future) throws Exception { + currentPromise = null; + if (future.isSuccess()) { + setSuccess(future.getNow()); + } else if(future.isCancelled()){ + if(!isCancelled()){ + cancel(true); + } + } else { + try { + if (tryHandler.handleFail(tries, future.cause())) { + // Returned true so should try again + tries++; + call(); + } else { + // Returned false but with no exception so should return empty result + setSuccess(null); + } + } catch (Throwable e) { + setFailure(e); + } + } + } + }); + } + + /** + * Handles the try + * @param Type of response from the try + */ + public interface TryHandler { + /** + * Call method + * @param tries amount of tries + * @return Response future + */ + Future call(int tries); + + /** + * Handles fails + * @param tries Current try number + * @param e exception thrown + * @return true if should continue to try, false if it should stop + * @throws Throwable if failure is a fail that cant be recovered + */ + boolean handleFail(int tries, Throwable e) throws Throwable; + } +} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ReversedClientScanner.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ReversedClientScanner.java index dde82ba..f0b0d41 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ReversedClientScanner.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ReversedClientScanner.java @@ -19,6 +19,8 @@ package org.apache.hadoop.hbase.client; import java.io.IOException; +import java.io.InterruptedIOException; +import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; import org.apache.commons.logging.Log; @@ -27,7 +29,6 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.classification.InterfaceAudience; -import org.apache.hadoop.hbase.ipc.RpcControllerFactory; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.ExceptionUtil; @@ -51,9 +52,9 @@ public class ReversedClientScanner extends ClientSimpleScanner { */ public ReversedClientScanner(Configuration conf, Scan scan, TableName tableName, ClusterConnection connection, - RpcRetryingCallerFactory rpcFactory, RpcControllerFactory controllerFactory, - ExecutorService pool, int primaryOperationTimeout) throws IOException { - super(conf, scan, tableName, connection, rpcFactory, controllerFactory, pool, + RpcRetryingCallerFactory rpcFactory, ExecutorService pool, int primaryOperationTimeout) + throws IOException { + super(conf, scan, tableName, connection, rpcFactory, pool, primaryOperationTimeout); } @@ -65,7 +66,13 @@ public class ReversedClientScanner extends ClientSimpleScanner { this.callable.setClose(); // callWithoutRetries is at this layer. Within the ScannerCallableWithReplicas, // we do a callWithRetries - this.caller.callWithoutRetries(callable, scannerTimeout); + try { + this.caller.callAsyncWithoutRetries(callable, scannerTimeout).get(); + } catch (InterruptedException e) { + throw new InterruptedIOException(e.getMessage()); + } catch (ExecutionException e) { + throw (IOException) e.getCause(); + } this.callable = null; } @@ -115,7 +122,13 @@ public class ReversedClientScanner extends ClientSimpleScanner { // beginning of the region // callWithoutRetries is at this layer. Within the ScannerCallableWithReplicas, // we do a callWithRetries - this.caller.callWithoutRetries(callable, scannerTimeout); + try { + this.caller.callAsyncWithoutRetries(callable, scannerTimeout).get(); + } catch (InterruptedException e) { + throw new InterruptedIOException(e.getMessage()); + } catch (ExecutionException e) { + throw (IOException) e.getCause(); + } this.currentRegion = callable.getHRegionInfo(); if (this.scanMetrics != null) { this.scanMetrics.countOfRegions.incrementAndGet(); @@ -133,12 +146,11 @@ public class ReversedClientScanner extends ClientSimpleScanner { scan.setStartRow(localStartKey); ScannerCallable s = new ReversedScannerCallable(getConnection(), getTable(), scan, this.scanMetrics, - locateStartRow, this.rpcControllerFactory); + locateStartRow); s.setCaching(nbRows); ScannerCallableWithReplicas sr = - new ScannerCallableWithReplicas(getTable(), getConnection(), s, pool, - primaryOperationTimeout, scan, getRetries(), getScannerTimeout(), caching, getConf(), - caller); + new ScannerCallableWithReplicas(getTable(), getConnection(), s, + primaryOperationTimeout, scan, getScannerTimeout(), caller); return sr; } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ReversedScannerCallable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ReversedScannerCallable.java index 0c2d345..d99eb5e 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ReversedScannerCallable.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ReversedScannerCallable.java @@ -30,10 +30,8 @@ import org.apache.hadoop.hbase.RegionLocations; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.client.metrics.ScanMetrics; -import org.apache.hadoop.hbase.ipc.RpcControllerFactory; import org.apache.hadoop.hbase.util.Bytes; - /** * A reversed ScannerCallable which supports backward scanning. */ @@ -52,12 +50,10 @@ public class ReversedScannerCallable extends ScannerCallable { * @param scan * @param scanMetrics * @param locateStartRow The start row for locating regions - * @param rpcFactory to create an {@link com.google.protobuf.RpcController} - * to talk to the regionserver */ public ReversedScannerCallable(ClusterConnection connection, TableName tableName, Scan scan, - ScanMetrics scanMetrics, byte[] locateStartRow, RpcControllerFactory rpcFactory) { - super(connection, tableName, scan, scanMetrics, rpcFactory); + ScanMetrics scanMetrics, byte[] locateStartRow) { + super(connection, tableName, scan, scanMetrics); this.locateStartRow = locateStartRow; } @@ -67,30 +63,15 @@ public class ReversedScannerCallable extends ScannerCallable { * @param scan * @param scanMetrics * @param locateStartRow The start row for locating regions - * @param rpcFactory to create an {@link com.google.protobuf.RpcController} - * to talk to the regionserver * @param replicaId the replica id */ public ReversedScannerCallable(ClusterConnection connection, TableName tableName, Scan scan, - ScanMetrics scanMetrics, byte[] locateStartRow, RpcControllerFactory rpcFactory, - int replicaId) { - super(connection, tableName, scan, scanMetrics, rpcFactory, replicaId); + ScanMetrics scanMetrics, byte[] locateStartRow, int replicaId) { + super(connection, tableName, scan, scanMetrics, replicaId); this.locateStartRow = locateStartRow; } /** - * @deprecated use - * {@link #ReversedScannerCallable(ClusterConnection, TableName, Scan, - * ScanMetrics, byte[], RpcControllerFactory )} - */ - @Deprecated - public ReversedScannerCallable(ClusterConnection connection, TableName tableName, - Scan scan, ScanMetrics scanMetrics, byte[] locateStartRow) { - this(connection, tableName, scan, scanMetrics, locateStartRow, RpcControllerFactory - .instantiate(connection.getConfiguration())); - } - - /** * @param reload force reload of server location * @throws IOException */ @@ -102,8 +83,8 @@ public class ReversedScannerCallable extends ScannerCallable { if (!instantiated || reload) { if (locateStartRow == null) { // Just locate the region with the row - RegionLocations rl = RpcRetryingCallerWithReadReplicas.getRegionLocations(reload, id, - getConnection(), tableName, row); + RegionLocations rl = RpcRetryingCallerWithReadReplicas + .getRegionLocations(reload, id, getConnection(), tableName, row); this.location = id < rl.size() ? rl.getRegionLocation(id) : null; if (this.location == null) { throw new IOException("Failed to find location, tableName=" @@ -123,7 +104,7 @@ public class ReversedScannerCallable extends ScannerCallable { } this.location = locatedRegions.get(locatedRegions.size() - 1); } - setStub(getConnection().getClient(getLocation().getServerName())); + setClientByServiceName(getLocation().getServerName()); checkIfRegionServerIsRemote(); instantiated = true; } @@ -158,8 +139,8 @@ public class ReversedScannerCallable extends ScannerCallable { List regionList = new ArrayList(); byte[] currentKey = startKey; do { - RegionLocations rl = RpcRetryingCallerWithReadReplicas.getRegionLocations(reload, id, - getConnection(), tableName, currentKey); + RegionLocations rl = RpcRetryingCallerWithReadReplicas + .getRegionLocations(reload, id, getConnection(), tableName, currentKey); HRegionLocation regionLocation = id < rl.size() ? rl.getRegionLocation(id) : null; if (regionLocation != null && regionLocation.getRegionInfo().containsRow(currentKey)) { regionList.add(regionLocation); @@ -177,7 +158,7 @@ public class ReversedScannerCallable extends ScannerCallable { @Override public ScannerCallable getScannerCallableForReplica(int id) { ReversedScannerCallable r = new ReversedScannerCallable(this.cConnection, this.tableName, - this.getScan(), this.scanMetrics, this.locateStartRow, controllerFactory, id); + this.getScan(), this.scanMetrics, this.locateStartRow, id); r.setCaching(this.getCaching()); return r; } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCaller.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCaller.java index 807c227..b29fd16 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCaller.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCaller.java @@ -22,9 +22,6 @@ import org.apache.hadoop.hbase.classification.InterfaceStability; import java.io.IOException; -/** - * - */ @InterfaceAudience.Public @InterfaceStability.Evolving public interface RpcRetryingCaller { @@ -43,7 +40,7 @@ public interface RpcRetryingCaller { /** * Call the server once only. - * {@link RetryingCallable} has a strange shape so we can do retrys. Use this invocation if you + * {@link RetryingCallable} has a strange shape so we can do retries. Use this invocation if you * want to do a single call only (A call to {@link RetryingCallable#call(int)} will not likely * succeed). * @return an object of type T @@ -52,4 +49,21 @@ public interface RpcRetryingCaller { */ T callWithoutRetries(RetryingCallable callable, int callTimeout) throws IOException, RuntimeException; + + /** + * Call the method Async with retries + * @param callable the async RegionServer callable + * @param operationTimeout timeout for the operation + * @return Future with the result + */ + Future callAsyncWithRetries(AsyncRetryingCallable callable, int operationTimeout); + + /** + * Call the server once only. + * {@link RetryingCallable} has a strange shape so we can do retries. Use this invocation if you + * want to do a single call only (A call to {@link RetryingCallable#call(int)} will not likely + * succeed). + * @return an object of type T + */ + Future callAsyncWithoutRetries(AsyncRetryingCallable callable, int callTimeout); } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerImpl.java index dd56b17..035f86f 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerImpl.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerImpl.java @@ -167,6 +167,96 @@ public class RpcRetryingCallerImpl implements RpcRetryingCaller { } } + @Override + public Future callAsyncWithRetries(final AsyncRetryingCallable callable, + final int callTimeout) { + this.globalStartTime = EnvironmentEdgeManager.currentTime(); + context.clear(); + + RetryingPromise future = + new RetryingPromise<>(callable.getEventExecutor(), new + RetryingPromise.TryHandler() { + List exceptions = new ArrayList<>(); + long expectedSleep; + + @Override + public Future call(int tries) { + try { + callable.prepare(tries != 0); // if called with false, check table status on ZK + interceptor.intercept(context.prepare(callable, tries)); + + return callable.call(getRemainingTime(callTimeout)); + } catch (IOException e) { + return new FailedFuture<>(callable.getEventExecutor(),e); + } + } + + @Override + public boolean handleFail(int tries, Throwable e) throws Throwable { + try { + if (e instanceof PreemptiveFastFailException) { + throw e; + } + + ExceptionUtil.rethrowIfInterrupt(e); + if (tries > startLogErrorsCnt) { + LOG.info("Call exception, tries=" + tries + ", retries=" + retries + ", started=" + + (EnvironmentEdgeManager.currentTime() - globalStartTime) + " ms ago, " + + "cancelled=" + cancelled.get() + ", msg=" + + callable.getExceptionMessageAdditionalDetail()); + } + + // translateException throws exception when should not retry: i.e. when request is bad + interceptor.handleFailure(context, e); + e = translateException(e); + callable.throwable(e, retries != 1); + RetriesExhaustedException.ThrowableWithExtraContext qt = + new RetriesExhaustedException.ThrowableWithExtraContext(e, + EnvironmentEdgeManager.currentTime(), toString()); + exceptions.add(qt); + if (tries >= retries - 1) { + throw new RetriesExhaustedException(tries, exceptions); + } + // If the server is dead, we need to wait a little before retrying, to give + // a chance to the regions to be + // tries hasn't been bumped up yet so we use "tries + 1" to get right pause time + expectedSleep = callable.sleep(pause, tries + 1); + + // If, after the planned sleep, there won't be enough time left, we stop now. + long duration = singleCallDuration(expectedSleep); + if (duration > callTimeout) { + String msg = "callTimeout=" + callTimeout + ", callDuration=" + duration + + ": " + callable.getExceptionMessageAdditionalDetail(); + throw (new SocketTimeoutException(msg).initCause(e)); + } + } finally { + interceptor.updateFailureInfo(context); + } + + try { + if (expectedSleep > 0) { + synchronized (cancelled) { + if (cancelled.get()) return false; + cancelled.wait(expectedSleep); + } + } + if (cancelled.get()){ + return false; + } + } catch (InterruptedException e1) { + throw new InterruptedIOException( + "Interrupted after " + tries + " tries on " + retries); + } + + return true; + } + }); + + future.call(); + + return future; + } + /** * @return Calculate how long a single call took */ @@ -193,7 +283,20 @@ public class RpcRetryingCallerImpl implements RpcRetryingCaller { } } } - + + @Override + public Future callAsyncWithoutRetries(AsyncRetryingCallable callable, + int callTimeout) { + // The code of this method should be shared with withRetries. + this.globalStartTime = EnvironmentEdgeManager.currentTime(); + try { + callable.prepare(false); + return callable.call(callTimeout); + } catch (IOException e) { + return new FailedFuture<>(callable.getEventExecutor(),e); + } + } + /** * Get the good or the remote exception if any, throws the DoNotRetryIOException. * @param t the throwable to analyze diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerWithReadReplicas.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerWithReadReplicas.java index 8f28881..90c52b1 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerWithReadReplicas.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerWithReadReplicas.java @@ -1,5 +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 @@ -16,19 +15,14 @@ * 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.util.Collections; import java.util.List; -import java.util.concurrent.CancellationException; +import java.util.TimerTask; import java.util.concurrent.ExecutionException; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; import org.apache.commons.logging.Log; @@ -38,19 +32,13 @@ import org.apache.hadoop.hbase.DoNotRetryIOException; import org.apache.hadoop.hbase.HBaseIOException; import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.RegionLocations; -import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.classification.InterfaceAudience; -import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController; -import org.apache.hadoop.hbase.ipc.RpcControllerFactory; -import org.apache.hadoop.hbase.protobuf.ProtobufUtil; +import org.apache.hadoop.hbase.ipc.MessageConverter; import org.apache.hadoop.hbase.protobuf.RequestConverter; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; -import com.google.protobuf.ServiceException; - - /** * Caller that goes to replica if the primary region does no answer within a configurable * timeout. If the timeout is reached, it calls all the secondary replicas, and returns @@ -58,35 +46,34 @@ import com.google.protobuf.ServiceException; * be marked as stale. */ @InterfaceAudience.Private -public class RpcRetryingCallerWithReadReplicas { +public class RpcRetryingCallerWithReadReplicas { private static final Log LOG = LogFactory.getLog(RpcRetryingCallerWithReadReplicas.class); - protected final ExecutorService pool; protected final ClusterConnection cConnection; protected final Configuration conf; protected final Get get; + private final MessageConverter converter; protected final TableName tableName; protected final int timeBeforeReplicas; private final int callTimeout; private final int retries; - private final RpcControllerFactory rpcControllerFactory; private final RpcRetryingCallerFactory rpcRetryingCallerFactory; + private final int priority; - public RpcRetryingCallerWithReadReplicas( - RpcControllerFactory rpcControllerFactory, TableName tableName, - ClusterConnection cConnection, final Get get, - ExecutorService pool, int retries, int callTimeout, + public RpcRetryingCallerWithReadReplicas(RpcRetryingCallerFactory rpcRetryingCallerFactory, + TableName tableName, ClusterConnection cConnection, final Get get, + MessageConverter converter, int retries, int callTimeout, int timeBeforeReplicas) { - this.rpcControllerFactory = rpcControllerFactory; this.tableName = tableName; this.cConnection = cConnection; this.conf = cConnection.getConfiguration(); this.get = get; - this.pool = pool; + this.converter = converter; this.retries = retries; this.callTimeout = callTimeout; this.timeBeforeReplicas = timeBeforeReplicas; - this.rpcRetryingCallerFactory = new RpcRetryingCallerFactory(conf); + this.rpcRetryingCallerFactory = rpcRetryingCallerFactory; + this.priority = AsyncTableImpl.getPriority(tableName); } /** @@ -95,22 +82,26 @@ public class RpcRetryingCallerWithReadReplicas { * - we need to stop retrying when the call is completed * - we can be interrupted */ - class ReplicaRegionServerCallable extends RegionServerCallable implements Cancellable { + abstract class AbstractReplicaRegionServerCallable extends AsyncRegionServerCallable + implements Cancellable { final int id; - private final PayloadCarryingRpcController controller; + private final MessageConverter converter; + + private boolean isCanceled; + private Future future; - public ReplicaRegionServerCallable(int id, HRegionLocation location) { + public AbstractReplicaRegionServerCallable(int id, HRegionLocation location, + MessageConverter converter) { super(RpcRetryingCallerWithReadReplicas.this.cConnection, RpcRetryingCallerWithReadReplicas.this.tableName, get.getRow()); this.id = id; this.location = location; - this.controller = rpcControllerFactory.newController(); - controller.setPriority(tableName); + this.converter = converter; } @Override public void cancel() { - controller.startCancel(); + isCanceled = true; } /** @@ -120,7 +111,7 @@ public class RpcRetryingCallerWithReadReplicas { */ @Override public void prepare(final boolean reload) throws IOException { - if (controller.isCanceled()) return; + if (isCanceled) return; if (Thread.interrupted()) { throw new InterruptedIOException(); @@ -137,39 +128,110 @@ public class RpcRetryingCallerWithReadReplicas { throw new HBaseIOException("There is no location for replica id #" + id); } - ServerName dest = location.getServerName(); + setClientByServiceName(location.getServerName()); + } + + @Override + public Future call(int callTimeout) { + if (isCanceled) return null; - setStub(cConnection.getClient(dest)); + byte[] reg = location.getRegionInfo().getRegionName(); + + ClientProtos.GetRequest request; + try { + request = RequestConverter.buildGetRequest(reg, get); + } catch (IOException e) { + return new FailedFuture<>(getConnection().getEventExecutor(),e); + } + + this.future = getChannel().callMethod( + AsyncTableImpl.GET_DESCRIPTOR, request, null, ClientProtos.GetResponse.getDefaultInstance(), + converter, null, callTimeout, priority + ); + return this.future; } @Override - public Result call(int callTimeout) throws Exception { - if (controller.isCanceled()) return null; + public boolean isCancelled() { + return (future != null && future.isCancelled()) + || isCanceled; + } + } - if (Thread.interrupted()) { - throw new InterruptedIOException(); + /** + * A RegionServerCallable that takes into account the replicas, i.e. + * - the call can be on any replica + * - we need to stop retrying when the call is completed + * - we can be interrupted + */ + class ReplicaRegionServerCallable extends AsyncRegionServerCallable implements + Cancellable { + final int id; + private boolean isCanceled; + private Future future; + private final MessageConverter converter; + + public ReplicaRegionServerCallable(int id, HRegionLocation location, + MessageConverter converter) { + super(RpcRetryingCallerWithReadReplicas.this.cConnection, + RpcRetryingCallerWithReadReplicas.this.tableName, get.getRow()); + this.id = id; + this.location = location; + this.converter = converter; + } + + @Override + public void cancel() { + isCanceled = true; + } + + /** + * Two responsibilities + * - if the call is already completed (by another replica) stops the retries. + * - set the location to the right region, depending on the replica. + */ + @Override + public void prepare(final boolean reload) throws IOException { + if (isCanceled) return; + + if (reload || location == null) { + RegionLocations rl = getRegionLocations(false, id, cConnection, tableName, get.getRow()); + location = id < rl.size() ? rl.getRegionLocation(id) : null; } - byte[] reg = location.getRegionInfo().getRegionName(); + if (location == null || location.getServerName() == null) { + // With this exception, there will be a retry. The location can be null for a replica + // when the table is created or after a split. + throw new HBaseIOException("There is no location for replica id #" + id); + } + + setClientByServiceName(location.getServerName()); + } - ClientProtos.GetRequest request = - RequestConverter.buildGetRequest(reg, get); - controller.setCallTimeout(callTimeout); + @Override + public Future call(int callTimeout) { + if (isCanceled) return null; + + byte[] reg = location.getRegionInfo().getRegionName(); + ClientProtos.GetRequest request; try { - ClientProtos.GetResponse response = getStub().get(controller, request); - if (response == null) { - return null; - } - return ProtobufUtil.toResult(response.getResult()); - } catch (ServiceException se) { - throw ProtobufUtil.getRemoteException(se); + request = RequestConverter.buildGetRequest(reg, get); + } catch (IOException e) { + return new FailedFuture<>(getConnection().getEventExecutor(),e); } + + this.future = getChannel().callMethod( + AsyncTableImpl.GET_DESCRIPTOR, request, null, + ClientProtos.GetResponse.getDefaultInstance(), converter, null, callTimeout, priority + ); + return this.future; } @Override public boolean isCancelled() { - return controller.isCanceled(); + return (future != null && future.isCancelled()) + || isCanceled; } } @@ -190,55 +252,37 @@ public class RpcRetryingCallerWithReadReplicas { * Globally, the number of retries, timeout and so on still applies, but it's per replica, * not global. We continue until all retries are done, or all timeouts are exceeded. */ - public synchronized Result call() - throws DoNotRetryIOException, InterruptedIOException, RetriesExhaustedException { + public Future call() { boolean isTargetReplicaSpecified = (get.getReplicaId() >= 0); - RegionLocations rl = getRegionLocations(true, (isTargetReplicaSpecified ? get.getReplicaId() - : RegionReplicaUtil.DEFAULT_REPLICA_ID), cConnection, tableName, get.getRow()); - ResultBoundedCompletionService cs = - new ResultBoundedCompletionService(this.rpcRetryingCallerFactory, pool, rl.size()); + try { + final RegionLocations rl = getRegionLocations(true, (isTargetReplicaSpecified ? get + .getReplicaId() + : RegionReplicaUtil.DEFAULT_REPLICA_ID), cConnection, tableName, get.getRow()); - if(isTargetReplicaSpecified) { - addCallsForReplica(cs, rl, get.getReplicaId(), get.getReplicaId()); - } else { - addCallsForReplica(cs, rl, 0, 0); - try { - // wait for the timeout to see whether the primary responds back - Future f = cs.poll(timeBeforeReplicas, TimeUnit.MICROSECONDS); // Yes, microseconds - if (f != null) { - return f.get(); //great we got a response - } - } catch (ExecutionException e) { - throwEnrichedException(e, retries); - } catch (CancellationException e) { - throw new InterruptedIOException(); - } catch (InterruptedException e) { - throw new InterruptedIOException(); - } + final SinglePromiseKeeper pk = new SinglePromiseKeeper<>( + this.cConnection.getEventExecutor(), rl.size()); - // submit call for the all of the secondaries at once - addCallsForReplica(cs, rl, 1, rl.size() - 1); - } - - try { - try { - Future f = cs.take(); - return f.get(); - } catch (ExecutionException e) { - throwEnrichedException(e, retries); + if(isTargetReplicaSpecified) { + addCallsForReplica(pk, rl, get.getReplicaId(), get.getReplicaId()); + } else { + addCallsForReplica(pk, rl, 0, 0); + + pk.addScheduledTask(TimeUnit.MILLISECONDS.convert(timeBeforeReplicas, TimeUnit.MICROSECONDS), + new TimerTask() { + @Override + public void run() { + // submit call for the all of the secondaries at once + addCallsForReplica(pk, rl, 1, rl.size() - 1); + } + } + ); } - } catch (CancellationException e) { - throw new InterruptedIOException(); - } catch (InterruptedException e) { - throw new InterruptedIOException(); - } finally { - // We get there because we were interrupted or because one or more of the - // calls succeeded or failed. In all case, we stop all our tasks. - cs.cancelAll(); - } - return null; // unreachable + return pk; + } catch (RetriesExhaustedException | DoNotRetryIOException | InterruptedIOException e) { + return new FailedFuture<>(cConnection.getEventExecutor(),e); + } } /** @@ -270,18 +314,21 @@ public class RpcRetryingCallerWithReadReplicas { /** * Creates the calls and submit them - * - * @param cs - the completion service to use for submitting + * @param promiseKeeper - the completion service to use for submitting * @param rl - the region locations * @param min - the id of the first replica, inclusive * @param max - the id of the last replica, inclusive. */ - private void addCallsForReplica(ResultBoundedCompletionService cs, + private void addCallsForReplica(SinglePromiseKeeper promiseKeeper, RegionLocations rl, int min, int max) { for (int id = min; id <= max; id++) { HRegionLocation hrl = rl.getRegionLocation(id); - ReplicaRegionServerCallable callOnReplica = new ReplicaRegionServerCallable(id, hrl); - cs.submit(callOnReplica, callTimeout, id); + ReplicaRegionServerCallable callable = new ReplicaRegionServerCallable<>(id, hrl, + converter); + + Future promise = + rpcRetryingCallerFactory.newCaller().callAsyncWithRetries(callable, callTimeout); + promiseKeeper.addPromise(promise); } } @@ -296,11 +343,7 @@ public class RpcRetryingCallerWithReadReplicas { } else { rl = cConnection.locateRegion(tableName, row, useCache, true, replicaId); } - } catch (DoNotRetryIOException e) { - throw e; - } catch (RetriesExhaustedException e) { - throw e; - } catch (InterruptedIOException e) { + } catch (DoNotRetryIOException | InterruptedIOException | RetriesExhaustedException e) { throw e; } catch (IOException e) { throw new RetriesExhaustedException("Can't get the location", e); diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallable.java index afce287..1f7c735 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallable.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallable.java @@ -24,6 +24,8 @@ import java.net.UnknownHostException; import java.util.Map; import java.util.Map.Entry; +import com.google.protobuf.Descriptors; +import io.netty.util.concurrent.GenericFutureListener; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; @@ -32,6 +34,7 @@ import org.apache.hadoop.hbase.CellScanner; import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.DoNotRetryIOException; import org.apache.hadoop.hbase.HBaseIOException; +import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.NotServingRegionException; @@ -41,37 +44,66 @@ import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.UnknownScannerException; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.client.metrics.ScanMetrics; -import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController; -import org.apache.hadoop.hbase.ipc.RpcControllerFactory; -import org.apache.hadoop.hbase.protobuf.ProtobufUtil; +import org.apache.hadoop.hbase.ipc.IOExceptionConverter; +import org.apache.hadoop.hbase.ipc.MessageConverter; import org.apache.hadoop.hbase.protobuf.RequestConverter; import org.apache.hadoop.hbase.protobuf.ResponseConverter; +import org.apache.hadoop.hbase.protobuf.generated.ClientProtos; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanRequest; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanResponse; import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException; import org.apache.hadoop.ipc.RemoteException; import org.apache.hadoop.net.DNS; -import com.google.protobuf.ServiceException; import com.google.protobuf.TextFormat; +import static org.apache.hadoop.hbase.client.AsyncTableImpl.getPriority; + /** * Scanner operations such as create, next, etc. * Used by {@link ResultScanner}s made by {@link HTable}. Passed to a retrying caller such as * {@link RpcRetryingCaller} so fails are retried. */ @InterfaceAudience.Private -public class ScannerCallable extends RegionServerCallable { - public static final String LOG_SCANNER_LATENCY_CUTOFF - = "hbase.client.log.scanner.latency.cutoff"; - public static final String LOG_SCANNER_ACTIVITY = "hbase.client.log.scanner.activity"; - +public class ScannerCallable extends AsyncRegionServerCallable { // Keeping LOG public as it is being used in TestScannerHeartbeatMessages public static final Log LOG = LogFactory.getLog(ScannerCallable.class); + + public static final String LOG_SCANNER_LATENCY_CUTOFF = "hbase.client.log.scanner.latency.cutoff"; + public static final String LOG_SCANNER_ACTIVITY = "hbase.client.log.scanner.activity"; + + protected static final Descriptors.MethodDescriptor SCAN_DESCRIPTOR = ClientProtos.ClientService + .getDescriptor().findMethodByName("Scan"); + + private final MessageConverter scan_open_converter = + new MessageConverter() { + @Override + public Long convert(ScanResponse response, CellScanner cellScanner) throws IOException { + long sid = response.getScannerId(); + if (logScannerActivity) { + LOG.info("Open scanner=" + sid + " for scan=" + scan.toString() + + " on region " + getLocation().toString()); + } + return sid; + } + }; + + private final MessageConverter no_converter = + new MessageConverter() { + @Override + public ScanResponse convert(ScanResponse response, CellScanner cellScanner) + throws IOException { + return response; + } + }; + protected long scannerId = -1L; + + private final int defaultScanTimeout; + protected boolean instantiated = false; protected boolean closed = false; - private Scan scan; + private final Scan scan; private int caching = 1; protected final ClusterConnection cConnection; protected ScanMetrics scanMetrics; @@ -98,8 +130,6 @@ public class ScannerCallable extends RegionServerCallable { // indicate if it is a remote server call protected boolean isRegionServerRemote = true; private long nextCallSeq = 0; - protected RpcControllerFactory controllerFactory; - protected PayloadCarryingRpcController controller; /** * @param connection which connection @@ -107,23 +137,21 @@ public class ScannerCallable extends RegionServerCallable { * @param scan the scan to execute * @param scanMetrics the ScanMetrics to used, if it is null, ScannerCallable won't collect * metrics - * @param rpcControllerFactory factory to use when creating - * {@link com.google.protobuf.RpcController} */ public ScannerCallable(ClusterConnection connection, TableName tableName, Scan scan, - ScanMetrics scanMetrics, RpcControllerFactory rpcControllerFactory) { - this(connection, tableName, scan, scanMetrics, rpcControllerFactory, 0); + ScanMetrics scanMetrics) { + this(connection, tableName, scan, scanMetrics, 0); } /** - * - * @param connection - * @param tableName - * @param scan - * @param scanMetrics + * Constructor + * @param connection to connect with + * @param tableName to connect to + * @param scan to scan with + * @param scanMetrics to store metrics in * @param id the replicaId */ public ScannerCallable(ClusterConnection connection, TableName tableName, Scan scan, - ScanMetrics scanMetrics, RpcControllerFactory rpcControllerFactory, int id) { + ScanMetrics scanMetrics, int id) { super(connection, tableName, scan.getStartRow()); this.id = id; this.cConnection = connection; @@ -132,11 +160,9 @@ public class ScannerCallable extends RegionServerCallable { Configuration conf = connection.getConfiguration(); logScannerActivity = conf.getBoolean(LOG_SCANNER_ACTIVITY, false); logCutOffLatency = conf.getInt(LOG_SCANNER_LATENCY_CUTOFF, 1000); - this.controllerFactory = rpcControllerFactory; - } - PayloadCarryingRpcController getController() { - return controller; + defaultScanTimeout = conf.getInt(HConstants.HBASE_CLIENT_OPERATION_TIMEOUT, + HConstants.DEFAULT_HBASE_CLIENT_OPERATION_TIMEOUT); } /** @@ -148,8 +174,8 @@ public class ScannerCallable extends RegionServerCallable { if (Thread.interrupted()) { throw new InterruptedIOException(); } - RegionLocations rl = RpcRetryingCallerWithReadReplicas.getRegionLocations(!reload, - id, getConnection(), getTableName(), getRow()); + RegionLocations rl = RpcRetryingCallerWithReadReplicas + .getRegionLocations(!reload, id, getConnection(), getTableName(), getRow()); location = id < rl.size() ? rl.getRegionLocation(id) : null; if (location == null || location.getServerName() == null) { // With this exception, there will be a retry. The location can be null for a replica @@ -157,7 +183,8 @@ public class ScannerCallable extends RegionServerCallable { throw new HBaseIOException("There is no location for replica id #" + id); } ServerName dest = location.getServerName(); - setStub(super.getConnection().getClient(dest)); + + super.setClientByServiceName(dest); if (!instantiated || reload) { checkIfRegionServerIsRemote(); instantiated = true; @@ -184,127 +211,150 @@ public class ScannerCallable extends RegionServerCallable { } } - @Override - public Result [] call(int callTimeout) throws IOException { - if (Thread.interrupted()) { - throw new InterruptedIOException(); - } + public Future call(final int callTimeout) { if (closed) { if (scannerId != -1) { close(); } + return new SuccessfulFuture<>(cConnection.getEventExecutor(), null); } else { if (scannerId == -1L) { - this.scannerId = openScanner(); - } else { - Result [] rrs = null; - ScanRequest request = null; - // Reset the heartbeat flag prior to each RPC in case an exception is thrown by the server - setHeartbeatMessage(false); - try { - incRPCcallsMetrics(); - request = - RequestConverter.buildScanRequest(scannerId, caching, false, nextCallSeq, - this.scanMetrics != null); - ScanResponse response = null; - controller = controllerFactory.newController(); - controller.setPriority(getTableName()); - controller.setCallTimeout(callTimeout); - try { - response = getStub().scan(controller, request); - // Client and RS maintain a nextCallSeq number during the scan. Every next() call - // from client to server will increment this number in both sides. Client passes this - // number along with the request and at RS side both the incoming nextCallSeq and its - // nextCallSeq will be matched. In case of a timeout this increment at the client side - // should not happen. If at the server side fetching of next batch of data was over, - // there will be mismatch in the nextCallSeq number. Server will throw - // OutOfOrderScannerNextException and then client will reopen the scanner with startrow - // as the last successfully retrieved row. - // See HBASE-5974 - nextCallSeq++; - long timestamp = System.currentTimeMillis(); - setHeartbeatMessage(response.hasHeartbeatMessage() && response.getHeartbeatMessage()); - // Results are returned via controller - CellScanner cellScanner = controller.cellScanner(); - rrs = ResponseConverter.getResults(cellScanner, response); - if (logScannerActivity) { - long now = System.currentTimeMillis(); - if (now - timestamp > logCutOffLatency) { - int rows = rrs == null ? 0 : rrs.length; - LOG.info("Took " + (now-timestamp) + "ms to fetch " - + rows + " rows from scanner=" + scannerId); + final DelayedPromise responsePromise = + new DelayedPromise<>(cConnection.getEventExecutor()); + + openScanner().addListener(new GenericFutureListener>() { + @Override + public void operationComplete(Future future) throws Exception { + // Delayed response promise could be canceled + if(!responsePromise.isDone()) { + if (future.isSuccess()) { + scannerId = future.getNow(); + responsePromise.setSuccess(null); + } else { + responsePromise.setFailure(future.cause()); } } - updateServerSideMetrics(response); - // moreResults is only used for the case where a filter exhausts all elements - if (response.hasMoreResults() && !response.getMoreResults()) { - scannerId = -1L; - closed = true; - // Implied that no results were returned back, either. - return null; - } - // moreResultsInRegion explicitly defines when a RS may choose to terminate a batch due - // to size or quantity of results in the response. - if (response.hasMoreResultsInRegion()) { - // Set what the RS said - setHasMoreResultsContext(true); - setServerHasMoreResults(response.getMoreResultsInRegion()); - } else { - // Server didn't respond whether it has more results or not. - setHasMoreResultsContext(false); - } - } catch (ServiceException se) { - throw ProtobufUtil.getRemoteException(se); } - updateResultsMetrics(rrs); - } catch (IOException e) { - if (logScannerActivity) { - LOG.info("Got exception making request " + TextFormat.shortDebugString(request) + }); + + return responsePromise; + } else { + return this.doCall(callTimeout); + } + } + } + + /** + * Do the actual scan call + * @param callTimeout for scanner + * @return Promise with results + */ + private Future doCall(int callTimeout) { + // Reset the heartbeat flag prior to each RPC in case an exception is thrown by the server + setHeartbeatMessage(false); + incRPCcallsMetrics(); + final ScanRequest request = + RequestConverter.buildScanRequest(scannerId, caching, false, nextCallSeq, + this.scanMetrics != null); + + return sendRequest(request, callTimeout, scan_converter, new IOExceptionConverter() { + @Override + public IOException convert(IOException e) { + if (logScannerActivity) { + LOG.info("Got exception making request " + TextFormat.shortDebugString(request) + " to " + getLocation(), e); + } + IOException ioe = e; + if (e instanceof RemoteException) { + ioe = ((RemoteException) e).unwrapRemoteException(); + } + if (logScannerActivity && (ioe instanceof UnknownScannerException)) { + try { + HRegionLocation loc = + getConnection().relocateRegion(getTableName(), scan.getStartRow()); + LOG.info("Scanner=" + scannerId + + " expired, current region location is " + loc.toString()); + } catch (Throwable t) { + LOG.info("Failed to relocate region", t); } - IOException ioe = e; - if (e instanceof RemoteException) { - ioe = ((RemoteException) e).unwrapRemoteException(); + } + // The below convertion of exceptions into DoNotRetryExceptions is a little strange. + // Why not just have these exceptions implment DNRIOE you ask? Well, usually we want + // ServerCallable#withRetries to just retry when it gets these exceptions. In here in + // a scan when doing a next in particular, we want to break out and get the scanner to + // reset itself up again. Throwing a DNRIOE is how we signal this to happen (its ugly, + // yeah and hard to follow and in need of a refactor). + if (ioe instanceof NotServingRegionException) { + // Throw a DNRE so that we break out of cycle of calling NSRE + // when what we need is to open scanner against new location. + // Attach NSRE to signal client that it needs to re-setup scanner. + if (scanMetrics != null) { + scanMetrics.countOfNSRE.incrementAndGet(); } - if (logScannerActivity && (ioe instanceof UnknownScannerException)) { - try { - HRegionLocation location = - getConnection().relocateRegion(getTableName(), scan.getStartRow()); - LOG.info("Scanner=" + scannerId - + " expired, current region location is " + location.toString()); - } catch (Throwable t) { - LOG.info("Failed to relocate region", t); + return new DoNotRetryIOException("Resetting the scanner -- see exception cause", ioe); + } else if (ioe instanceof RegionServerStoppedException) { + // Throw a DNRE so that we break out of cycle of the retries and instead go and + // open scanner against new location. + return new DoNotRetryIOException("Resetting the scanner -- see exception cause", ioe); + } else { + // The outer layers will retry + return ioe; + } + } + }); + } + + private final MessageConverter scan_converter = + new MessageConverter() { + @Override + public Result[] convert(ScanResponse response, CellScanner cellScanner) throws IOException { + // Client and RS maintain a nextCallSeq number during the scan. Every next() call + // from client to server will increment this number in both sides. Client passes this + // number along with the request and at RS side both the incoming nextCallSeq and its + // nextCallSeq will be matched. In case of a timeout this increment at the client side + // should not happen. If at the server side fetching of next batch of data was over, + // there will be mismatch in the nextCallSeq number. Server will throw + // OutOfOrderScannerNextException and then client will reopen the scanner with startrow + // as the last successfully retrieved row. + // See HBASE-5974 + nextCallSeq++; + long timestamp = System.currentTimeMillis(); + setHeartbeatMessage(response.hasHeartbeatMessage() && response.getHeartbeatMessage()); + // Results are returned via controller + Result[] rrs = ResponseConverter.getResults(cellScanner, response); + if (logScannerActivity) { + long now = System.currentTimeMillis(); + if (now - timestamp > logCutOffLatency) { + int rows = rrs == null ? 0 : rrs.length; + LOG.info("Took " + (now-timestamp) + "ms to fetch " + + rows + " rows from scanner=" + scannerId); } } - // The below convertion of exceptions into DoNotRetryExceptions is a little strange. - // Why not just have these exceptions implment DNRIOE you ask? Well, usually we want - // ServerCallable#withRetries to just retry when it gets these exceptions. In here in - // a scan when doing a next in particular, we want to break out and get the scanner to - // reset itself up again. Throwing a DNRIOE is how we signal this to happen (its ugly, - // yeah and hard to follow and in need of a refactor). - if (ioe instanceof NotServingRegionException) { - // Throw a DNRE so that we break out of cycle of calling NSRE - // when what we need is to open scanner against new location. - // Attach NSRE to signal client that it needs to re-setup scanner. - if (this.scanMetrics != null) { - this.scanMetrics.countOfNSRE.incrementAndGet(); - } - throw new DoNotRetryIOException("Resetting the scanner -- see exception cause", ioe); - } else if (ioe instanceof RegionServerStoppedException) { - // Throw a DNRE so that we break out of cycle of the retries and instead go and - // open scanner against new location. - throw new DoNotRetryIOException("Resetting the scanner -- see exception cause", ioe); + updateServerSideMetrics(response); + // moreResults is only used for the case where a filter exhausts all elements + if (response.hasMoreResults() && !response.getMoreResults()) { + scannerId = -1L; + closed = true; + // Implied that no results were returned back, either. + return null; + } + // moreResultsInRegion explicitly defines when a RS may choose to terminate a batch due + // to size or quantity of results in the response. + if (response.hasMoreResultsInRegion()) { + // Set what the RS said + setHasMoreResultsContext(true); + setServerHasMoreResults(response.getMoreResultsInRegion()); } else { - // The outer layers will retry - throw ioe; + // Server didn't respond whether it has more results or not. + setHasMoreResultsContext(false); } + + updateResultsMetrics(rrs); + + return rrs; } - return rrs; - } - } - return null; - } + }; /** * @return true when the most recent RPC response indicated that the response was a heartbeat @@ -365,40 +415,39 @@ public class ScannerCallable extends RegionServerCallable { if (this.scannerId == -1L) { return; } - try { - incRPCcallsMetrics(); - ScanRequest request = - RequestConverter.buildScanRequest(this.scannerId, 0, true, this.scanMetrics != null); - try { - getStub().scan(null, request); - } catch (ServiceException se) { - throw ProtobufUtil.getRemoteException(se); + incRPCcallsMetrics(); + ScanRequest request = + RequestConverter.buildScanRequest(this.scannerId, 0, true, this.scanMetrics != null); + + sendRequest(request, defaultScanTimeout, no_converter, new IOExceptionConverter() { + @Override + public IOException convert(IOException e) { + LOG.warn("Ignore, probably already closed", e); + return e; } - } catch (IOException e) { - LOG.warn("Ignore, probably already closed", e); - } + }); this.scannerId = -1L; } - protected long openScanner() throws IOException { + protected Future openScanner() { incRPCcallsMetrics(); - ScanRequest request = - RequestConverter.buildScanRequest( - getLocation().getRegionInfo().getRegionName(), - this.scan, 0, false); try { - ScanResponse response = getStub().scan(null, request); - long id = response.getScannerId(); - if (logScannerActivity) { - LOG.info("Open scanner=" + id + " for scan=" + scan.toString() - + " on region " + getLocation().toString()); - } - return id; - } catch (ServiceException se) { - throw ProtobufUtil.getRemoteException(se); + ScanRequest request = RequestConverter + .buildScanRequest(getLocation().getRegionInfo().getRegionName(), this.scan, 0, false); + + return sendRequest(request,defaultScanTimeout, scan_open_converter, null); + } catch (IOException e) { + return new FailedFuture<>(getConnection().getEventExecutor(), e); } } + protected Future sendRequest(ScanRequest request, int timeout, + MessageConverter converter, IOExceptionConverter exceptionConverter){ + return getChannel().callMethod(SCAN_DESCRIPTOR,request,null, + ClientProtos.ScanResponse.getDefaultInstance(), converter, exceptionConverter, timeout, + getPriority(tableName)); + } + protected Scan getScan() { return scan; } @@ -444,7 +493,7 @@ public class ScannerCallable extends RegionServerCallable { public ScannerCallable getScannerCallableForReplica(int id) { ScannerCallable s = new ScannerCallable(this.getConnection(), this.tableName, - this.getScan(), this.scanMetrics, controllerFactory, id); + this.getScan(), this.scanMetrics, id); s.setCaching(this.caching); return s; } @@ -474,4 +523,4 @@ public class ScannerCallable extends RegionServerCallable { protected void setHasMoreResultsContext(boolean serverHasMoreResultsContext) { this.serverHasMoreResultsContext = serverHasMoreResultsContext; } -} +} \ No newline at end of file diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallableWithReplicas.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallableWithReplicas.java index 1708efe..7d35179 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallableWithReplicas.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallableWithReplicas.java @@ -22,27 +22,19 @@ import static org.apache.hadoop.hbase.client.ClientScanner.createClosestRowBefor import java.io.IOException; import java.io.InterruptedIOException; -import java.util.ArrayList; -import java.util.HashSet; -import java.util.List; -import java.util.Set; -import java.util.concurrent.Callable; -import java.util.concurrent.CancellationException; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Future; +import java.util.TimerTask; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; +import io.netty.util.concurrent.EventExecutor; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.DoNotRetryIOException; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.RegionLocations; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.util.Pair; import com.google.common.annotations.VisibleForTesting; @@ -59,38 +51,31 @@ import com.google.common.annotations.VisibleForTesting; * */ @InterfaceAudience.Private -class ScannerCallableWithReplicas implements RetryingCallable { +class ScannerCallableWithReplicas implements AsyncRetryingCallable { private static final Log LOG = LogFactory.getLog(ScannerCallableWithReplicas.class); + private final RpcRetryingCaller caller; + volatile ScannerCallable currentScannerCallable; AtomicBoolean replicaSwitched = new AtomicBoolean(false); final ClusterConnection cConnection; - protected final ExecutorService pool; protected final int timeBeforeReplicas; private final Scan scan; - private final int retries; private Result lastResult; - private final RpcRetryingCaller caller; private final TableName tableName; - private Configuration conf; - private int scannerTimeout; - private Set outstandingCallables = new HashSet(); - private boolean someRPCcancelled = false; //required for testing purposes only + protected boolean someRPCcancelled = false; //required for testing purposes only + private final int scannerTimeout; public ScannerCallableWithReplicas(TableName tableName, ClusterConnection cConnection, - ScannerCallable baseCallable, ExecutorService pool, int timeBeforeReplicas, Scan scan, - int retries, int scannerTimeout, int caching, Configuration conf, - RpcRetryingCaller caller) { + ScannerCallable baseCallable, int timeBeforeReplicas, Scan scan, + int scannerTimeout, RpcRetryingCaller caller) { this.currentScannerCallable = baseCallable; this.cConnection = cConnection; - this.pool = pool; if (timeBeforeReplicas < 0) { throw new IllegalArgumentException("Invalid value of operation timeout on the primary"); } this.timeBeforeReplicas = timeBeforeReplicas; this.scan = scan; - this.retries = retries; this.tableName = tableName; - this.conf = conf; this.scannerTimeout = scannerTimeout; this.caller = caller; } @@ -128,7 +113,7 @@ class ScannerCallableWithReplicas implements RetryingCallable { } @Override - public Result [] call(int timeout) throws IOException { + public Future call(int timeout) { // If the active replica callable was closed somewhere, invoke the RPC to // really close it. In the case of regular scanners, this applies. We make couple // of RPCs to a RegionServer, and when that region is exhausted, we set @@ -139,128 +124,64 @@ class ScannerCallableWithReplicas implements RetryingCallable { if (LOG.isTraceEnabled()) { LOG.trace("Closing scanner id=" + currentScannerCallable.scannerId); } - Result[] r = currentScannerCallable.call(timeout); + Future r = currentScannerCallable.call(timeout); currentScannerCallable = null; return r; } + // We need to do the following: //1. When a scan goes out to a certain replica (default or not), we need to // continue to hit that until there is a failure. So store the last successfully invoked // replica //2. We should close the "losing" scanners (scanners other than the ones we hear back // from first) - // - RegionLocations rl = RpcRetryingCallerWithReadReplicas.getRegionLocations(true, - RegionReplicaUtil.DEFAULT_REPLICA_ID, cConnection, tableName, - currentScannerCallable.getRow()); - - // allocate a boundedcompletion pool of some multiple of number of replicas. - // We want to accomodate some RPCs for redundant replica scans (but are still in progress) - ResultBoundedCompletionService> cs = - new ResultBoundedCompletionService>( - new RpcRetryingCallerFactory(ScannerCallableWithReplicas.this.conf), pool, - rl.size() * 5); - - List exceptions = null; - int submitted = 0, completed = 0; - AtomicBoolean done = new AtomicBoolean(false); - replicaSwitched.set(false); - // submit call for the primary replica. - submitted += addCallsForCurrentReplica(cs, rl); try { - // wait for the timeout to see whether the primary responds back - Future> f = cs.poll(timeBeforeReplicas, - TimeUnit.MICROSECONDS); // Yes, microseconds - if (f != null) { - Pair r = f.get(); - if (r != null && r.getSecond() != null) { - updateCurrentlyServingReplica(r.getSecond(), r.getFirst(), done, pool); + final RegionLocations rl = RpcRetryingCallerWithReadReplicas + .getRegionLocations(true, RegionReplicaUtil.DEFAULT_REPLICA_ID, cConnection, tableName, + currentScannerCallable.getRow()); + + // We want to accomodate some RPCs for redundant replica scans (but are still in progress) + final ScannerReplicaPromiseKeeper pk = + new ScannerReplicaPromiseKeeper( + this, cConnection.getEventExecutor(), rl.size() * 5); + + replicaSwitched.set(false); + // submit call for the primary replica. + addCallsForCurrentReplica(pk); + + pk.addScheduledTask(TimeUnit.MICROSECONDS.toMillis(timeBeforeReplicas), new TimerTask() { + @Override + public void run() { + // submit call for the all of the secondaries at once + // TODO: this may be an overkill for large region replication + addCallsForOtherReplicas(pk, 0, rl.size() - 1); } - return r == null ? null : r.getFirst(); //great we got a response - } - } catch (ExecutionException e) { - // the primary call failed with RetriesExhaustedException or DoNotRetryIOException - // but the secondaries might still succeed. Continue on the replica RPCs. - exceptions = new ArrayList(rl.size()); - exceptions.add(e); - completed++; - } catch (CancellationException e) { - throw new InterruptedIOException(e.getMessage()); - } catch (InterruptedException e) { - throw new InterruptedIOException(e.getMessage()); - } - // submit call for the all of the secondaries at once - // TODO: this may be an overkill for large region replication - submitted += addCallsForOtherReplicas(cs, rl, 0, rl.size() - 1); - try { - while (completed < submitted) { - try { - Future> f = cs.take(); - Pair r = f.get(); - if (r != null && r.getSecond() != null) { - updateCurrentlyServingReplica(r.getSecond(), r.getFirst(), done, pool); - } - return r == null ? null : r.getFirst(); // great we got an answer - } catch (ExecutionException e) { - // if not cancel or interrupt, wait until all RPC's are done - // one of the tasks failed. Save the exception for later. - if (exceptions == null) exceptions = new ArrayList(rl.size()); - exceptions.add(e); - completed++; - } - } - } catch (CancellationException e) { - throw new InterruptedIOException(e.getMessage()); - } catch (InterruptedException e) { - throw new InterruptedIOException(e.getMessage()); - } finally { - // We get there because we were interrupted or because one or more of the - // calls succeeded or failed. In all case, we stop all our tasks. - cs.cancelAll(); - } + }); - if (exceptions != null && !exceptions.isEmpty()) { - RpcRetryingCallerWithReadReplicas.throwEnrichedException(exceptions.get(0), - retries); // just rethrow the first exception for now. + return pk; + } catch (RetriesExhaustedException | DoNotRetryIOException | InterruptedIOException e) { + return new FailedFuture<>(cConnection.getEventExecutor(), e); } - return null; // unreachable } - private void updateCurrentlyServingReplica(ScannerCallable scanner, Result[] result, - AtomicBoolean done, ExecutorService pool) { - if (done.compareAndSet(false, true)) { - if (currentScannerCallable != scanner) replicaSwitched.set(true); - currentScannerCallable = scanner; - // store where to start the replica scanner from if we need to. - if (result != null && result.length != 0) this.lastResult = result[result.length - 1]; - if (LOG.isTraceEnabled()) { - LOG.trace("Setting current scanner as id=" + currentScannerCallable.scannerId + - " associated with replica=" + currentScannerCallable.getHRegionInfo().getReplicaId()); - } - // close all outstanding replica scanners but the one we heard back from - outstandingCallables.remove(scanner); - for (ScannerCallable s : outstandingCallables) { - if (LOG.isTraceEnabled()) { - LOG.trace("Closing scanner id=" + s.scannerId + - ", replica=" + s.getHRegionInfo().getRegionId() + - " because slow and replica=" + - this.currentScannerCallable.getHRegionInfo().getReplicaId() + " succeeded"); - } - // Submit the "close" to the pool since this might take time, and we don't - // want to wait for the "close" to happen yet. The "wait" will happen when - // the table is closed (when the awaitTermination of the underlying pool is called) - s.setClose(); - final RetryingRPC r = new RetryingRPC(s); - pool.submit(new Callable(){ - @Override - public Void call() throws Exception { - r.call(scannerTimeout); - return null; - } - }); - } - // now clear outstandingCallables since we scheduled a close for all the contained scanners - outstandingCallables.clear(); + @Override + public EventExecutor getEventExecutor() { + return cConnection.getEventExecutor(); + } + + /** + * Called by the ScannerReplicaPromiseKEeper + * @param scanner which succeeded + * @param result of success + */ + protected void updateCurrentlyServingReplica(ScannerCallable scanner, Result[] result) { + if (currentScannerCallable != scanner) replicaSwitched.set(true); + currentScannerCallable = scanner; + // store where to start the replica scanner from if we need to. + if (result != null && result.length != 0) this.lastResult = result[result.length - 1]; + if (LOG.isTraceEnabled()) { + LOG.trace("Setting current scanner as id=" + currentScannerCallable.scannerId + + " associated with replica=" + currentScannerCallable.getHRegionInfo().getReplicaId()); } } @@ -284,15 +205,14 @@ class ScannerCallableWithReplicas implements RetryingCallable { } private int addCallsForCurrentReplica( - ResultBoundedCompletionService> cs, RegionLocations rl) { - RetryingRPC retryingOnReplica = new RetryingRPC(currentScannerCallable); - outstandingCallables.add(currentScannerCallable); - cs.submit(retryingOnReplica, scannerTimeout, currentScannerCallable.id); + ScannerReplicaPromiseKeeper pk) { + pk.addPromise(currentScannerCallable, + caller.callAsyncWithRetries(currentScannerCallable, scannerTimeout)); return 1; } private int addCallsForOtherReplicas( - ResultBoundedCompletionService> cs, RegionLocations rl, + ScannerReplicaPromiseKeeper pk, int min, int max) { if (scan.getConsistency() == Consistency.STRONG) { return 0; // not scheduling on other replicas for strong consistency @@ -303,9 +223,9 @@ class ScannerCallableWithReplicas implements RetryingCallable { } ScannerCallable s = currentScannerCallable.getScannerCallableForReplica(id); setStartRowForReplicaCallable(s); - outstandingCallables.add(s); - RetryingRPC retryingOnReplica = new RetryingRPC(s); - cs.submit(retryingOnReplica, scannerTimeout, id); + + pk.addPromise(s, + caller.callAsyncWithRetries(s, scannerTimeout)); } return max - min + 1; } @@ -340,78 +260,6 @@ class ScannerCallableWithReplicas implements RetryingCallable { return someRPCcancelled; } - class RetryingRPC implements RetryingCallable>, Cancellable { - final ScannerCallable callable; - RpcRetryingCaller caller; - private volatile boolean cancelled = false; - - RetryingRPC(ScannerCallable callable) { - this.callable = callable; - // For the Consistency.STRONG (default case), we reuse the caller - // to keep compatibility with what is done in the past - // For the Consistency.TIMELINE case, we can't reuse the caller - // since we could be making parallel RPCs (caller.callWithRetries is synchronized - // and we can't invoke it multiple times at the same time) - this.caller = ScannerCallableWithReplicas.this.caller; - if (scan.getConsistency() == Consistency.TIMELINE) { - this.caller = new RpcRetryingCallerFactory(ScannerCallableWithReplicas.this.conf). - newCaller(); - } - } - - @Override - public Pair call(int callTimeout) throws IOException { - // since the retries is done within the ResultBoundedCompletionService, - // we don't invoke callWithRetries here - if (cancelled) { - return null; - } - Result[] res = this.caller.callWithoutRetries(this.callable, callTimeout); - return new Pair(res, this.callable); - } - - @Override - public void prepare(boolean reload) throws IOException { - if (cancelled) return; - - if (Thread.interrupted()) { - throw new InterruptedIOException(); - } - - callable.prepare(reload); - } - - @Override - public void throwable(Throwable t, boolean retrying) { - callable.throwable(t, retrying); - } - - @Override - public String getExceptionMessageAdditionalDetail() { - return callable.getExceptionMessageAdditionalDetail(); - } - - @Override - public long sleep(long pause, int tries) { - return callable.sleep(pause, tries); - } - - @Override - public void cancel() { - cancelled = true; - caller.cancel(); - if (callable.getController() != null) { - callable.getController().startCancel(); - } - someRPCcancelled = true; - } - - @Override - public boolean isCancelled() { - return cancelled; - } - } - @Override public void prepare(boolean reload) throws IOException { } @@ -430,4 +278,24 @@ class ScannerCallableWithReplicas implements RetryingCallable { public long sleep(long pause, int tries) { return currentScannerCallable.sleep(pause, tries); } -} + + /** + * Internal call to close given scanner + * @param s scanner callable to close + */ + protected void closeScanner(ScannerCallable s) { + if (LOG.isTraceEnabled()) { + LOG.trace("Closing scanner id=" + s.scannerId + + ", replica=" + s.getHRegionInfo().getRegionId() + + " because slow and replica=" + + this.currentScannerCallable.getHRegionInfo().getReplicaId() + " succeeded"); + } + + // Submit the "close" to the pool since this might take time, and we don't + // want to wait for the "close" to happen yet. The "wait" will happen when + // the table is closed (when the awaitTermination of the underlying pool is called) + s.setClose(); + + caller.callAsyncWithoutRetries(s,scannerTimeout); + } +} \ No newline at end of file diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerReplicaPromiseKeeper.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerReplicaPromiseKeeper.java new file mode 100644 index 0000000..0083c94 --- /dev/null +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerReplicaPromiseKeeper.java @@ -0,0 +1,137 @@ +/** + * 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 io.netty.util.concurrent.DefaultPromise; +import io.netty.util.concurrent.EventExecutor; +import org.apache.hadoop.hbase.classification.InterfaceAudience; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Timer; +import java.util.TimerTask; +import java.util.concurrent.ExecutionException; + +/** + * Keeps a Single promise + */ +@InterfaceAudience.Private +public class ScannerReplicaPromiseKeeper extends DefaultPromise + implements Future { + + private final Map promises; + private final List exceptions; + private final ScannerCallableWithReplicas callableWithReplicas; + + private Timer timer; + + private final ResponseFutureListener listener = new ResponseFutureListener() { + @Override + public void operationComplete(Future future) { + synchronized (promises) { + if (!isDone()) { + try { + Result[] results = future.get(); + + callableWithReplicas.updateCurrentlyServingReplica(promises.get(future), results); + + promises.remove(future); + closeRemainingScans(); + setSuccess(results); + }catch(InterruptedException e){ + synchronized (promises) { + promises.remove(future); + } + cancel(true); + }catch(ExecutionException e){ + promises.remove(future); + if (promises.isEmpty()) { + setFailure(exceptions.get(0)); + } else { + exceptions.add(e); + } + } + } + + if (timer != null) { + timer.cancel(); + timer = null; + } + + for (Future promise : promises.keySet()) { + callableWithReplicas.someRPCcancelled = true; + if (!promise.isDone()) { + promise.cancel(true); + } + } + promises.clear(); + } + } + }; + + private void closeRemainingScans() { + synchronized (promises) { + for (ScannerCallable callable : promises.values()) { + callableWithReplicas.closeScanner(callable); + } + } + } + + /** + * Constructor + * @param callableWithReplicas callable with replicas to change on success + * @param executor to run promise keeper on + * @param maxPromises max amount of promises to be able to listen to + */ + public ScannerReplicaPromiseKeeper( + ScannerCallableWithReplicas callableWithReplicas, + EventExecutor executor, int maxPromises) { + super(executor); + this.callableWithReplicas = callableWithReplicas; + this.promises = new HashMap<>(maxPromises); + this.exceptions = new ArrayList<>(maxPromises); + } + + /** + * Add a promise to the single promise listener + * @param callable for the scanner + * @param promise to listen to + */ + public void addPromise(ScannerCallable callable, Future promise){ + synchronized (promises) { + this.promises.put(promise, callable); + promise.addListener(listener); + } + } + + /** + * Add a scheduled task to apply when a certain timeout has been reached + * @param timeToScheduleInMs time in ms to schedule additive task + * @param timerTask task to time. + */ + public void addScheduledTask(long timeToScheduleInMs, TimerTask timerTask) { + if(this.timer == null) { + this.timer = new Timer(true); + this.timer.schedule(timerTask,timeToScheduleInMs); + }else{ + throw new RuntimeException("There is already a scheduled task added"); + } + } +} \ No newline at end of file diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/SinglePromiseKeeper.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/SinglePromiseKeeper.java new file mode 100644 index 0000000..83cddd2 --- /dev/null +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/SinglePromiseKeeper.java @@ -0,0 +1,112 @@ +/** + * 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 io.netty.util.concurrent.DefaultPromise; +import io.netty.util.concurrent.EventExecutor; +import org.apache.hadoop.hbase.classification.InterfaceAudience; + +import java.util.ArrayList; +import java.util.List; +import java.util.Timer; +import java.util.TimerTask; +import java.util.concurrent.ExecutionException; + +/** + * Keeps a Single promise + * @param Type of result returned by promise + */ +@InterfaceAudience.Private +public class SinglePromiseKeeper extends DefaultPromise + implements Future { + + private final List promises; + private final List exceptions; + + private int taskCounter = 0; + + private Timer timer; + + private final ResponseFutureListener listener = new ResponseFutureListener() { + @Override + public void operationComplete(Future future) { + if(!isDone()) { + promises.remove(future); + + try { + setSuccess(future.get()); + } catch (InterruptedException e) { + cancel(true); + } catch (ExecutionException e) { + if(promises.isEmpty()) { + setFailure(exceptions.get(0)); + }else{ + exceptions.add(e); + } + } + } + + if(timer != null){ + timer.cancel(); + timer = null; + } + + for(Future promise : promises){ + if(!promise.isDone()) { + promise.cancel(true); + } + } + promises.clear(); + } + }; + + /** + * Constructor + * @param executor to run promise keeper on + * @param maxPromises max amount of promises to be able to listen to + */ + public SinglePromiseKeeper(EventExecutor executor, int maxPromises) { + super(executor); + this.promises = new ArrayList<>(maxPromises); + this.exceptions = new ArrayList<>(maxPromises); + } + + /** + * Add a promise to the single promise listener + * @param promise to listen to + */ + public void addPromise(Future promise){ + this.taskCounter++; + this.promises.add(promise); + promise.addListener(listener); + } + + /** + * Add a scheduled task to apply when a certain timeout has been reached + * @param timeToScheduleInMs time in ms to schedule additive task + * @param timerTask task to time. + */ + public void addScheduledTask(long timeToScheduleInMs, TimerTask timerTask) { + if(this.timer == null) { + this.timer = new Timer(true); + this.timer.schedule(timerTask,timeToScheduleInMs); + }else{ + throw new RuntimeException("There is already a scheduled task added"); + } + } +} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/StatsTrackingRpcRetryingCaller.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/StatsTrackingRpcRetryingCaller.java index e82f1e8..dddae0e 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/StatsTrackingRpcRetryingCaller.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/StatsTrackingRpcRetryingCaller.java @@ -56,7 +56,37 @@ public class StatsTrackingRpcRetryingCaller implements RpcRetryingCaller { return updateStatsAndUnwrap(result, callable); } - private T updateStatsAndUnwrap(T result, RetryingCallable callable) { + @Override + public Future callAsyncWithRetries(final AsyncRetryingCallable callable, + int operationTimeout) { + Future future = delegate.callAsyncWithRetries(callable, operationTimeout); + future.addListener(new ResponseFutureListener() { + @Override + public void operationComplete(Future future) throws Exception { + if(future.isSuccess()) { + updateStatsAndUnwrap(future.getNow(), callable); + } + } + }); + return future; + } + + @Override + public Future callAsyncWithoutRetries(final AsyncRetryingCallable callable, int + callTimeout) { + Future future = delegate.callAsyncWithoutRetries(callable, callTimeout); + future.addListener(new ResponseFutureListener() { + @Override + public void operationComplete(Future future) throws Exception { + if(future.isSuccess()) { + updateStatsAndUnwrap(future.getNow(), callable); + } + } + }); + return future; + } + + private T updateStatsAndUnwrap(T result, ProtoRetryingCallable callable) { // don't track stats about requests that aren't to regionservers if (!(callable instanceof RegionServerCallable)) { return result; diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/SuccessfulFuture.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/SuccessfulFuture.java new file mode 100644 index 0000000..2812a89 --- /dev/null +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/SuccessfulFuture.java @@ -0,0 +1,68 @@ +/** + * 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 io.netty.util.concurrent.CompleteFuture; +import io.netty.util.concurrent.EventExecutor; +import org.apache.hadoop.hbase.classification.InterfaceAudience; + +/** + * A Failed Response future + * @param Value type for Future + */ +@InterfaceAudience.Private +public final class SuccessfulFuture extends CompleteFuture implements Future { + + private final V result; + + /** + * Creates a new instance. + * + * @param executor the {@link EventExecutor} associated with this future + * @param result of promise + */ + public SuccessfulFuture(EventExecutor executor, V result) { + super(executor); + this.result = result; + } + + @Override + public Throwable cause() { + return null; + } + + @Override + public boolean isSuccess() { + return true; + } + + @Override + public io.netty.util.concurrent.Future sync() { + return this; + } + + @Override + public io.netty.util.concurrent.Future syncUninterruptibly() { + return this; + } + + @Override + public V getNow() { + return result; + } +} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableImpl.java new file mode 100644 index 0000000..5371d34 --- /dev/null +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableImpl.java @@ -0,0 +1,585 @@ +/** + * + * 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 com.google.protobuf.Descriptors; +import com.google.protobuf.Message; +import com.google.protobuf.Service; +import com.google.protobuf.ServiceException; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.classification.InterfaceStability; +import org.apache.hadoop.hbase.client.coprocessor.Batch; +import org.apache.hadoop.hbase.client.coprocessor.Batch.Callback; +import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp; +import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel; +import org.apache.hadoop.hbase.ipc.RegionCoprocessorRpcChannel; +import org.apache.hadoop.hbase.protobuf.ProtobufUtil; +import org.apache.hadoop.hbase.util.Bytes; + +import java.io.IOException; +import java.io.InterruptedIOException; +import java.util.Collections; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.Queue; +import java.util.TreeMap; +import java.util.concurrent.Callable; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; + +import static org.apache.hadoop.hbase.client.AsyncTableImpl.getKeysAndRegionsInRange; + +/** + * An implementation of {@link Table}. Used to communicate with a single HBase table. + * Lightweight. Get as needed and just close when done. + * Instances of this class SHOULD NOT be constructed directly. + * Obtain an instance via {@link Connection}. See {@link ConnectionFactory} + * class comment for an example of how. + * + *

TableImpl is not a client API. Use {@link Table} instead. It is marked + * InterfaceAudience.Private indicating that this is an HBase-internal class as defined in + * Hadoop + * Interface Classification + * There are no guarantees for backwards source / binary compatibility and methods or class can + * change or go away without deprecation. + * + * @see Table + * @see Admin + * @see Connection + * @see ConnectionFactory + */ +@InterfaceAudience.Private +@InterfaceStability.Stable +public class TableImpl implements Table { + private static final Log LOG = LogFactory.getLog(TableImpl.class); + + private final ClusterConnection connection; + private final AsyncTable asyncTable; + private final TableConfiguration tableConfiguration; + + private boolean closed = false; + private long writeBuffer; + + private BufferedMutatorImpl mutator; + + /** + * Creates an object to access a HBase table. + * Used by HBase internally. DO NOT USE. See {@link ConnectionFactory} class comment for how to + * get a {@link Table} instance (use {@link Table} instead of {@link TableImpl}). + * @param tableName Name of the table. + * @param connection HConnection to be used. + * @param tableConfig configuration for the Table + * @param rpcCallerFactory + * @throws IOException if a remote or network exception occurs + */ + @InterfaceAudience.Private + protected TableImpl(TableName tableName, final ClusterConnection connection, + TableConfiguration tableConfig, RpcRetryingCallerFactory rpcCallerFactory) + throws IOException { + if (connection == null || connection.isClosed()) { + throw new IllegalArgumentException("Connection is null or closed."); + } + + this.connection = connection; + + if (tableConfig == null) { + tableConfiguration = new TableConfiguration(connection.getConfiguration()); + } else { + this.tableConfiguration = tableConfig; + } + + this.asyncTable = new AsyncTableImpl(tableName, connection, + tableConfiguration, rpcCallerFactory); + } + + /** + * {@inheritDoc} + */ + @Override + public Configuration getConfiguration() { + return asyncTable.getConfiguration(); + } + + @Override + public HTableDescriptor getTableDescriptor() throws IOException { + try(Admin admin = connection.getAdmin()){ + return admin.getTableDescriptor(asyncTable.getName()); + } + } + + @Override + public boolean exists(Get get) throws IOException { + try { + get.setCheckExistenceOnly(true); + + Result r = asyncTable.get(get).get(); + return r.getExists(); + } catch (InterruptedException e) { + throw new InterruptedIOException(e.getMessage()); + } catch (ExecutionException e) { + throw (IOException) e.getCause(); + } + } + + @Override + public boolean[] existsAll(List gets) throws IOException { + try { + for (Get get : gets) { + get.setCheckExistenceOnly(true); + } + + Result[] rs = asyncTable.get(gets).get(); + + boolean[] exists = new boolean[rs.length]; + for (int i = 0; i < rs.length; i++) { + exists[i] = rs[i].getExists(); + } + + return exists; + } catch (InterruptedException e) { + throw new InterruptedIOException(e.getMessage()); + } catch (ExecutionException e) { + throw (IOException) e.getCause(); + } + } + + @Override + public void batch(List actions, Object[] results) + throws IOException, InterruptedException { + connection.processBatch(actions, getName(), connection.getEventExecutor(), results); + } + + @Override + public Object[] batch(List actions) throws IOException, InterruptedException { + Object[] results = new Object[actions.size()]; + this.batch(actions, results); + return results; + } + + @Override + public void batchCallback(List actions, Object[] results, Callback callback) + throws IOException, InterruptedException { + connection.processBatchCallback(actions, getName(), connection.getEventExecutor(), results, + callback); + } + + @Override + public Object[] batchCallback(List actions, Callback callback) + throws IOException, InterruptedException { + Object[] results = new Object[actions.size()]; + this.batchCallback(actions, results, callback); + return results; + } + + @Override + public Result get(Get get) throws IOException { + try { + return asyncTable.get(get).get(); + } catch (InterruptedException e) { + throw new InterruptedIOException(e.getMessage()); + } catch (ExecutionException e) { + throw (IOException) e.getCause(); + } + } + + @Override + public Result[] get(List gets) throws IOException { + try { + return asyncTable.get(gets).get(); + } catch (InterruptedException e) { + throw new InterruptedIOException(e.getMessage()); + } catch (ExecutionException e) { + throw (IOException) e.getCause(); + } + } + + @Override + public ResultScanner getScanner(Scan scan) throws IOException { + return new WrappedAsyncClientScanner(scan); + } + + @Override + public ResultScanner getScanner(byte[] family) throws IOException { + Scan scan = new Scan(); + scan.addFamily(family); + return this.getScanner(scan); + } + + @Override + public ResultScanner getScanner(byte[] family, byte[] qualifier) throws IOException { + Scan scan = new Scan(); + scan.addColumn(family, qualifier); + return this.getScanner(scan); + } + + @Override + public void put(Put put) throws IOException { + try { + this.asyncTable.mutate(put).get(); + } catch (InterruptedException e) { + throw new InterruptedIOException(e.getMessage()); + } catch (ExecutionException e) { + throw (IOException) e.getCause(); + } + } + + @Override + public void put(List puts) throws IOException { + getBufferedMutator().mutate(puts); + getBufferedMutator().flush(); + } + + @Override + public boolean checkAndPut(byte[] row, byte[] family, byte[] qualifier, byte[] value, Put put) + throws IOException { + return this.checkAndPut(row,family,qualifier,CompareOp.EQUAL,value,put); + } + + @Override + public boolean checkAndPut(byte[] row, byte[] family, byte[] qualifier, CompareOp compareOp, + byte[] value, Put put) throws IOException { + try { + return asyncTable.checkAndMutate(family, qualifier, compareOp, value, put).get(); + } catch (InterruptedException e) { + throw new InterruptedIOException(e.getMessage()); + } catch (ExecutionException e) { + throw (IOException) e.getCause(); + } + } + + @Override + public void delete(Delete delete) throws IOException { + try { + asyncTable.mutate(delete).get(); + } catch (InterruptedException e) { + throw new InterruptedIOException(e.getMessage()); + } catch (ExecutionException e) { + throw (IOException) e.getCause(); + } + } + + @Override + public void delete(List deletes) throws IOException { + getBufferedMutator().mutate(deletes); + getBufferedMutator().flush(); + } + + @Override + public boolean checkAndDelete(byte[] row, byte[] family, byte[] qualifier, byte[] value, + Delete delete) throws IOException { + return this.checkAndDelete(row, family, qualifier, CompareOp.EQUAL, value, delete); + } + + @Override + public boolean checkAndDelete(byte[] row, byte[] family, byte[] qualifier, CompareOp compareOp, + byte[] value, Delete delete) throws IOException { + try { + return asyncTable.checkAndMutate(family, qualifier, compareOp, value, delete).get(); + } catch (InterruptedException e) { + throw new InterruptedIOException(e.getMessage()); + } catch (ExecutionException e) { + throw (IOException) e.getCause(); + } + } + + @Override + public void mutateRow(RowMutations rm) throws IOException { + try { + asyncTable.mutate(rm).get(); + } catch (InterruptedException e) { + throw new InterruptedIOException(e.getMessage()); + } catch (ExecutionException e) { + throw (IOException) e.getCause(); + } + } + + @Override + public Result append(Append append) throws IOException { + try { + return asyncTable.mutate(append).get(); + } catch (InterruptedException e) { + throw new InterruptedIOException(e.getMessage()); + } catch (ExecutionException e) { + throw (IOException) e.getCause(); + } + } + + @Override + public Result increment(Increment increment) throws IOException { + try { + return asyncTable.mutate(increment).get(); + } catch (InterruptedException e) { + throw new InterruptedIOException(e.getMessage()); + } catch (ExecutionException e) { + throw (IOException) e.getCause(); + } + } + + @Override + public long incrementColumnValue(byte[] row, byte[] family, byte[] qualifier, long amount) + throws IOException { + return this.incrementColumnValue(row, family, qualifier, amount, Durability.SYNC_WAL); + } + + @Override + public long incrementColumnValue(byte[] row, byte[] family, byte[] qualifier, long amount, + Durability durability) throws IOException { + Increment increment = new Increment(row); + increment.addColumn(family, qualifier, amount); + increment.setDurability(durability); + + try { + Cell cell = asyncTable.mutate(increment).get().getColumnLatestCell(family, qualifier); + return Bytes.toLong(cell.getValueArray(),cell.getValueOffset(), cell.getValueLength()); + } catch (InterruptedException e) { + throw new InterruptedIOException(e.getMessage()); + } catch (ExecutionException e) { + throw (IOException) e.getCause(); + } + } + + @Override + public boolean checkAndMutate(byte[] row, byte[] family, byte[] qualifier, CompareOp compareOp, + byte[] value, RowMutations mutation) throws IOException { + try { + return asyncTable.checkAndMutate(family, qualifier, compareOp, value, mutation).get(); + } catch (InterruptedException e) { + throw new InterruptedIOException(e.getMessage()); + } catch (ExecutionException e) { + throw (IOException) e.getCause(); + } + } + + @Override + public TableName getName() { + return asyncTable.getName(); + } + + @Override + public void close() throws IOException { + if (this.closed) { + return; + } + if(mutator != null){ + mutator.close(); + } + asyncTable.close(); + this.closed = true; + } + + /** + * {@inheritDoc} + */ + @Override + public CoprocessorRpcChannel coprocessorService(byte[] row) { + return new RegionCoprocessorRpcChannel(connection, asyncTable.getName(), row); + } + + /** + * {@inheritDoc} + */ + @Override + public Map coprocessorService(final Class service, + byte[] startKey, byte[] endKey, final Batch.Call callable) + throws ServiceException, Throwable { + final Map results = Collections + .synchronizedMap(new TreeMap(Bytes.BYTES_COMPARATOR)); + coprocessorService(service, startKey, endKey, callable, new Batch.Callback() { + @Override + public void update(byte[] region, byte[] row, R value) { + if (region != null) { + results.put(region, value); + } + } + }); + return results; + } + + /** + * {@inheritDoc} + */ + @Override + public void coprocessorService(final Class service, + byte[] startKey, byte[] endKey, final Batch.Call callable, + final Callback callback) throws ServiceException, Throwable { + // get regions covered by the row range + List keys = getStartKeysInRange(startKey, endKey); + + Map> futures = + new TreeMap<>(Bytes.BYTES_COMPARATOR); + for (final AsyncTableImpl.KeyAndRegion r : keys) { + final RegionCoprocessorRpcChannel channel = + new RegionCoprocessorRpcChannel(connection, asyncTable.getName(), r.getKey()); + + Future future = connection.getEventExecutor().submit(new Callable() { + @Override + public R call() throws Exception { + T instance = ProtobufUtil.newServiceStub(service, channel); + R result = callable.call(instance); + byte[] region = channel.getLastRegion(); + if (callback != null) { + callback.update(region, r.getKey(), result); + } + return result; + } + }); + futures.put(r.getKey(), future); + } + for (Map.Entry> e : futures.entrySet()) { + try { + e.getValue().get(); + } catch (ExecutionException ee) { + LOG.warn("Error calling coprocessor service " + service.getName() + " for row " + + Bytes.toStringBinary(e.getKey()), ee); + throw ee.getCause(); + } catch (InterruptedException ie) { + throw new InterruptedIOException("Interrupted calling coprocessor service " + + service.getName() + " for row " + Bytes.toStringBinary(e.getKey())).initCause(ie); + } + } + } + + private List getStartKeysInRange(byte[] start, byte[] end) + throws IOException { + if (start == null) { + start = HConstants.EMPTY_START_ROW; + } + if (end == null) { + end = HConstants.EMPTY_END_ROW; + } + return getKeysAndRegionsInRange(connection, asyncTable.getName(), start, end, true, false); + } + + @Override + public long getWriteBufferSize() { + return writeBuffer; + } + + @Override + public void setWriteBufferSize(long writeBufferSize) throws IOException { + this.writeBuffer = writeBufferSize; + } + + @Override + public Map batchCoprocessorService( + Descriptors.MethodDescriptor methodDescriptor, Message request, byte[] startKey, + byte[] endKey, R responsePrototype) throws ServiceException, InterruptedException { + try { + return asyncTable.batchCoprocessorService(methodDescriptor, request, startKey, endKey, + responsePrototype).get(); + } catch (ExecutionException e) { + throw new ServiceException(e.getCause()); + } + } + + @Override + public void batchCoprocessorService( + Descriptors.MethodDescriptor methodDescriptor, Message request, byte[] startKey, + byte[] endKey, R responsePrototype, Callback callback) throws ServiceException, Throwable { + try { + asyncTable.batchCoprocessorService(methodDescriptor, request, startKey, endKey, + responsePrototype, callback).get(); + } catch (ExecutionException e) { + throw new ServiceException(e.getCause()); + } + } + + @Override + public String toString() { + return asyncTable.getName() + ";" + connection; + } + + BufferedMutator getBufferedMutator() throws IOException { + if (mutator == null) { + this.mutator = (BufferedMutatorImpl) connection.getBufferedMutator( + new BufferedMutatorParams(asyncTable.getName()).pool(connection.getEventExecutor()) + .writeBufferSize(tableConfiguration.getWriteBufferSize()) + .maxKeyValueSize(tableConfiguration.getMaxKeyValueSize())); + } + return mutator; + } + + /** + * Wraps an AsyncClientScanner + */ + private class WrappedAsyncClientScanner extends AbstractClientScanner { + private final AsyncResultScanner scanner; + private final Queue cache = new LinkedList<>(); + + /** + * Constructor + * @param scan to use for scanner + * @throws IOException on failed scanner creation + */ + public WrappedAsyncClientScanner(Scan scan) throws IOException { + super(); + scanner = asyncTable.getScanner(scan); + } + + @Override + public Result next() throws IOException { + // If the scanner is closed and there's nothing left in the cache, next is a no-op. + if (cache.isEmpty() && scanner.isClosed()) { + return null; + } + if (cache.isEmpty()) { + loadCache(); + } + + if (!cache.isEmpty()) { + return cache.poll(); + } + return null; + } + + private void loadCache() throws IOException { + try { + Result[] rs = scanner.nextBatch().get(); + + Collections.addAll(cache, rs); + } catch (InterruptedException e) { + throw new InterruptedIOException(e.getMessage()); + } catch (ExecutionException e) { + throw (IOException) e.getCause(); + } + } + + @Override + public void close() { + this.scanner.close(); + } + + @Override + public boolean renewLease() { + try { + return this.scanner.renewLease().get(); + } catch (InterruptedException | ExecutionException e) { + return false; + } + } + } +} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/VoidPromiseKeeper.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/VoidPromiseKeeper.java new file mode 100644 index 0000000..7fd4580 --- /dev/null +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/VoidPromiseKeeper.java @@ -0,0 +1,104 @@ +/** + * 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 io.netty.util.concurrent.DefaultPromise; +import io.netty.util.concurrent.EventExecutor; +import org.apache.hadoop.hbase.classification.InterfaceAudience; + +import java.util.concurrent.ExecutionException; + +/** + * Keeps all promises which are added to this promise but returns null as response when all are done + */ +@InterfaceAudience.Private +public class VoidPromiseKeeper extends DefaultPromise implements Future { + + private int indexCounter = 0; + private int totalCount = 0; + private int doneCount = 0; + private final Future[] promises; + + /** + * Constructor + * @param executor to run promise keeper on + * @param maxPromises max amount of promises to be able to listen to + */ + public VoidPromiseKeeper(EventExecutor executor, int maxPromises) { + super(executor); + this.promises = new Future[maxPromises]; + } + + /** + * Add a promise to the single promise listener + * @param promise to listen to + */ + public void addFuture(Future promise){ + totalCount++; + int i = indexCounter++; + this.promises[i] = promise; + promise.addListener(new IndexedResponseFuture(i)); + } + + /** + * Cancel all open promises + */ + private void cancelAllPromises() { + for(Future p : promises){ + if(p != null) { + p.cancel(true); + } + } + } + + /** + * Indexed response promise listener + */ + private class IndexedResponseFuture implements ResponseFutureListener { + private final int index; + + /** + * Constructor + * @param i index for the response + */ + public IndexedResponseFuture(int i) { + this.index = i; + } + + @Override + public void operationComplete(Future future) { + if(!isDone()) { + // Set promise ref to be null to indicate it is fulfilled + promises[index] = null; + try { + future.get(); + doneCount++; + if(doneCount == totalCount){ + setSuccess(null); + } + } catch (InterruptedException e) { + cancel(true); + cancelAllPromises(); + } catch (ExecutionException e) { + setFailure(e.getCause()); + cancelAllPromises(); + } + } + } + } +} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/coprocessor/AsyncBatch.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/coprocessor/AsyncBatch.java new file mode 100644 index 0000000..abfd381 --- /dev/null +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/coprocessor/AsyncBatch.java @@ -0,0 +1,76 @@ +/* + * 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.coprocessor; + +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.classification.InterfaceStability; +import org.apache.hadoop.hbase.client.Future; + +/** + * A collection of interfaces and utilities used for interacting with custom RPC + * interfaces exposed by Coprocessors. + */ +@InterfaceAudience.Public +@InterfaceStability.Stable +public abstract class AsyncBatch { + /** + * Defines a unit of work to be executed. + * + *

+ * When used with + * {@link org.apache.hadoop.hbase.client.AsyncTable#coprocessorService(Class, byte[], byte[], + * AsyncBatch.Call)} + * the implementations {@link AsyncBatch.Call#call(Object)} method will be invoked + * with a proxy to each region's coprocessor {@link com.google.protobuf.Service} implementation. + *

+ * @see org.apache.hadoop.hbase.client.coprocessor + * @see org.apache.hadoop.hbase.client.HTable#coprocessorService(byte[]) + * @see org.apache.hadoop.hbase.client.AsyncTable#coprocessorService(Class, byte[], byte[], + * AsyncBatch.Call) + * @param the instance type to be passed to + * {@link AsyncBatch.Call#call(Object)} + * @param the return type from {@link AsyncBatch.Call#call(Object)} + */ + @InterfaceAudience.Public + @InterfaceStability.Stable + public interface Call { + Future call(T instance); + } + + /** + * Defines a generic callback to be triggered for each {@link AsyncBatch.Call#call(Object)} + * result. + * + *

+ * When used with + * {@link org.apache.hadoop.hbase.client.AsyncTable#coprocessorService(Class, byte[], byte[], + * AsyncBatch.Call)} the implementation's {@link AsyncBatch.Callback#update(byte[], byte[], + * Object)} + * method will be called with the {@link AsyncBatch.Call#call(Object)} return value + * from each region in the selected range. + *

+ * @param the return type from the associated {@link AsyncBatch.Call#call(Object)} + * @see org.apache.hadoop.hbase.client.AsyncTable#coprocessorService(Class, byte[], byte[], + * AsyncBatch.Call) + */ + @InterfaceAudience.Public + @InterfaceStability.Stable + public interface Callback { + void update(byte[] region, byte[] row, R result); + } +} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java index 9be370d..68ecbbd 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java @@ -251,6 +251,27 @@ public abstract class AbstractRpcClient implements RpcClient { } /** + * Configure a payload carrying controller + * @param controller to configure + * @param channelOperationTimeout timeout for operation + * @return configured payload controller + */ + static PayloadCarryingRpcController configurePayloadCarryingRpcController( + RpcController controller, int channelOperationTimeout) { + PayloadCarryingRpcController pcrc; + if (controller != null) { + pcrc = (PayloadCarryingRpcController) controller; + if (!pcrc.hasCallTimeout()) { + pcrc.setCallTimeout(channelOperationTimeout); + } + } else { + pcrc = new PayloadCarryingRpcController(); + pcrc.setCallTimeout(channelOperationTimeout); + } + return pcrc; + } + + /** * Blocking rpc channel that goes via hbase rpc. */ @VisibleForTesting @@ -274,16 +295,9 @@ public abstract class AbstractRpcClient implements RpcClient { @Override public Message callBlockingMethod(Descriptors.MethodDescriptor md, RpcController controller, Message param, Message returnType) throws ServiceException { - PayloadCarryingRpcController pcrc; - if (controller != null) { - pcrc = (PayloadCarryingRpcController) controller; - if (!pcrc.hasCallTimeout()) { - pcrc.setCallTimeout(channelOperationTimeout); - } - } else { - pcrc = new PayloadCarryingRpcController(); - pcrc.setCallTimeout(channelOperationTimeout); - } + PayloadCarryingRpcController pcrc = configurePayloadCarryingRpcController( + controller, + channelOperationTimeout); return this.rpcClient.callBlockingMethod(md, pcrc, param, returnType, this.ticket, this.isa); } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AsyncCall.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AsyncCall.java index ec1909a..4a828fb 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AsyncCall.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AsyncCall.java @@ -19,8 +19,6 @@ package org.apache.hadoop.hbase.ipc; import com.google.protobuf.Descriptors; import com.google.protobuf.Message; -import io.netty.channel.EventLoop; -import io.netty.util.concurrent.DefaultPromise; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hbase.CellScanner; @@ -36,43 +34,62 @@ import java.io.IOException; * Represents an Async Hbase call and its response. * * Responses are passed on to its given doneHandler and failures to the rpcController + * @param Type of message returned + * @param Message returned in communication to be converted */ @InterfaceAudience.Private -public class AsyncCall extends DefaultPromise { +public class AsyncCall extends Promise { private static final Log LOG = LogFactory.getLog(AsyncCall.class.getName()); + private final AsyncRpcChannelImpl channel; + final int id; final Descriptors.MethodDescriptor method; final Message param; - final PayloadCarryingRpcController controller; final Message responseDefaultType; + + private final MessageConverter messageConverter; final long startTime; final long rpcTimeout; + private final IOExceptionConverter exceptionConverter; + + // For only the request + private CellScanner cellScanner; + private final int priority; /** * Constructor - * - * @param eventLoop for call + * @param channel which initiated call * @param connectId connection id * @param md the method descriptor * @param param parameters to send to Server - * @param controller controller for response + * @param cellScanner cellScanner containing cells to send as request * @param responseDefaultType the default response type + * @param messageConverter converts the messages to what is the expected output + * @param rpcTimeout timeout for this call in ms + * @param priority for this request */ - public AsyncCall(EventLoop eventLoop, int connectId, Descriptors.MethodDescriptor md, Message - param, PayloadCarryingRpcController controller, Message responseDefaultType) { - super(eventLoop); + public AsyncCall(AsyncRpcChannelImpl channel, int connectId, Descriptors.MethodDescriptor + md, Message param, CellScanner cellScanner, M responseDefaultType, MessageConverter + messageConverter, IOExceptionConverter exceptionConverter, long rpcTimeout, int priority) { + super(channel.getEventExecutor()); + + this.channel = channel; this.id = connectId; this.method = md; this.param = param; - this.controller = controller; this.responseDefaultType = responseDefaultType; + this.messageConverter = messageConverter; + this.exceptionConverter = exceptionConverter; this.startTime = EnvironmentEdgeManager.currentTime(); - this.rpcTimeout = controller.hasCallTimeout() ? controller.getCallTimeout() : 0; + this.rpcTimeout = rpcTimeout; + + this.priority = priority; + this.cellScanner = cellScanner; } /** @@ -96,17 +113,19 @@ public class AsyncCall extends DefaultPromise { * @param value to set * @param cellBlockScanner to set */ - public void setSuccess(Message value, CellScanner cellBlockScanner) { - if (cellBlockScanner != null) { - controller.setCellScanner(cellBlockScanner); - } - + public void setSuccess(M value, CellScanner cellBlockScanner) { if (LOG.isTraceEnabled()) { long callTime = EnvironmentEdgeManager.currentTime() - startTime; LOG.trace("Call: " + method.getName() + ", callTime: " + callTime + "ms"); } - this.setSuccess(value); + try { + this.setSuccess( + this.messageConverter.convert(value, cellBlockScanner) + ); + } catch (IOException e) { + this.setFailed(e); + } } /** @@ -122,6 +141,10 @@ public class AsyncCall extends DefaultPromise { exception = ((RemoteException) exception).unwrapRemoteException(); } + if (this.exceptionConverter != null) { + exception = this.exceptionConverter.convert(exception); + } + this.setFailure(exception); } @@ -133,4 +156,25 @@ public class AsyncCall extends DefaultPromise { public long getRpcTimeout() { return rpcTimeout; } + + /** + * @return Priority for this call + */ + public int getPriority() { + return priority; + } + + /** + * Get the cellScanner for this request. + * @return CellScanner + */ + public CellScanner cellScanner() { + return cellScanner; + } + + @Override + public boolean cancel(boolean mayInterupt){ + this.channel.removePendingCall(this.id); + return super.cancel(mayInterupt); + } } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AsyncCoprocessorRpcChannel.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AsyncCoprocessorRpcChannel.java new file mode 100644 index 0000000..99d636c --- /dev/null +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AsyncCoprocessorRpcChannel.java @@ -0,0 +1,88 @@ +/* + * 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.ipc; + +import com.google.protobuf.Descriptors; +import com.google.protobuf.Message; +import com.google.protobuf.RpcCallback; +import com.google.protobuf.RpcController; +import com.google.protobuf.ServiceException; +import io.netty.util.concurrent.GenericFutureListener; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.classification.InterfaceStability; +import org.apache.hadoop.hbase.client.Future; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.protobuf.ResponseConverter; + +import java.io.IOException; +import java.util.concurrent.ExecutionException; + +/** + * Base class which provides clients with an RPC connection to + * call coprocessor endpoint {@link com.google.protobuf.Service}s. + * Note that clients should not use this class directly, except through + * {@link org.apache.hadoop.hbase.client.HTableInterface#coprocessorService(byte[])}. + */ +@InterfaceAudience.Public +@InterfaceStability.Evolving +public abstract class AsyncCoprocessorRpcChannel implements CoprocessorRpcChannel { + private static final Log LOG = LogFactory.getLog(AsyncCoprocessorRpcChannel.class); + + @Override + @InterfaceAudience.Private + public void callMethod(Descriptors.MethodDescriptor method, + final RpcController controller, + Message request, Message responsePrototype, + final RpcCallback callback) { + callMethodWithPromise(method, request, responsePrototype).addListener( + new GenericFutureListener>() { + @Override + public void operationComplete(Future future) throws Exception { + if (future.isSuccess()) { + callback.run(future.getNow()); + } else if (future.isCancelled()) { + controller.startCancel(); + } else { + LOG.warn("Call failed on IOException", future.cause()); + ResponseConverter.setControllerException(controller, (IOException) future.cause()); + } + } + }); + } + + @Override + @InterfaceAudience.Private + public Message callBlockingMethod(Descriptors.MethodDescriptor method, + RpcController controller, + Message request, Message responsePrototype) + throws ServiceException { + try { + return callMethodWithPromise(method, request, responsePrototype).get(); + } catch (InterruptedException e) { + controller.startCancel(); + return null; + } catch (ExecutionException e) { + throw new ServiceException("Error calling method "+method.getFullName(), e.getCause()); + } + } + + public abstract Future callMethodWithPromise( + Descriptors.MethodDescriptor method, Message request, R responsePrototype); +} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AsyncRpcChannel.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AsyncRpcChannel.java index cfc8b1b..3927fbd 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AsyncRpcChannel.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AsyncRpcChannel.java @@ -17,731 +17,61 @@ */ package org.apache.hadoop.hbase.ipc; -import io.netty.bootstrap.Bootstrap; -import io.netty.buffer.ByteBuf; -import io.netty.buffer.ByteBufOutputStream; -import io.netty.channel.Channel; -import io.netty.channel.ChannelFuture; -import io.netty.channel.ChannelFutureListener; -import io.netty.handler.codec.LengthFieldBasedFrameDecoder; -import io.netty.util.Timeout; -import io.netty.util.TimerTask; -import io.netty.util.concurrent.GenericFutureListener; -import io.netty.util.concurrent.Promise; - -import java.io.IOException; -import java.net.ConnectException; -import java.net.InetSocketAddress; -import java.net.SocketException; -import java.nio.ByteBuffer; -import java.security.PrivilegedExceptionAction; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.Random; -import java.util.concurrent.TimeUnit; - -import javax.security.sasl.SaslException; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.classification.InterfaceAudience; -import org.apache.hadoop.hbase.exceptions.ConnectionClosingException; -import org.apache.hadoop.hbase.protobuf.ProtobufUtil; -import org.apache.hadoop.hbase.protobuf.generated.AuthenticationProtos; -import org.apache.hadoop.hbase.protobuf.generated.RPCProtos; -import org.apache.hadoop.hbase.protobuf.generated.TracingProtos; -import org.apache.hadoop.hbase.security.AuthMethod; -import org.apache.hadoop.hbase.security.SaslClientHandler; -import org.apache.hadoop.hbase.security.SaslUtil; -import org.apache.hadoop.hbase.security.SecurityInfo; -import org.apache.hadoop.hbase.security.User; -import org.apache.hadoop.hbase.security.token.AuthenticationTokenSelector; -import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; -import org.apache.hadoop.io.Text; -import org.apache.hadoop.ipc.RemoteException; -import org.apache.hadoop.security.SecurityUtil; -import org.apache.hadoop.security.UserGroupInformation; -import org.apache.hadoop.security.token.Token; -import org.apache.hadoop.security.token.TokenIdentifier; -import org.apache.hadoop.security.token.TokenSelector; -import org.apache.htrace.Span; -import org.apache.htrace.Trace; - import com.google.protobuf.Descriptors; import com.google.protobuf.Message; -import com.google.protobuf.RpcCallback; +import io.netty.channel.EventLoop; +import io.netty.util.concurrent.EventExecutor; +import org.apache.hadoop.hbase.CellScanner; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.client.Future; + +import java.net.InetSocketAddress; /** - * Netty RPC channel + * Interface for Async Rpc Channels */ @InterfaceAudience.Private -public class AsyncRpcChannel { - private static final Log LOG = LogFactory.getLog(AsyncRpcChannel.class.getName()); - - private static final int MAX_SASL_RETRIES = 5; - - protected final static Map> tokenHandlers = new HashMap<>(); - - static { - tokenHandlers.put(AuthenticationProtos.TokenIdentifier.Kind.HBASE_AUTH_TOKEN, - new AuthenticationTokenSelector()); - } - - final AsyncRpcClient client; - - // Contains the channel to work with. - // Only exists when connected - private Channel channel; - - String name; - final User ticket; - final String serviceName; - final InetSocketAddress address; - - private int ioFailureCounter = 0; - private int connectFailureCounter = 0; - - boolean useSasl; - AuthMethod authMethod; - private int reloginMaxBackoff; - private Token token; - private String serverPrincipal; - - - // NOTE: closed and connected flags below are only changed when a lock on pendingCalls - private final Map pendingCalls = new HashMap(); - private boolean connected = false; - private boolean closed = false; - - private Timeout cleanupTimer; - - private final TimerTask timeoutTask = new TimerTask() { - @Override - public void run(Timeout timeout) throws Exception { - cleanupCalls(); - } - }; - - /** - * Constructor for netty RPC channel - * - * @param bootstrap to construct channel on - * @param client to connect with - * @param ticket of user which uses connection - * @param serviceName name of service to connect to - * @param address to connect to - */ - public AsyncRpcChannel(Bootstrap bootstrap, final AsyncRpcClient client, User ticket, String - serviceName, InetSocketAddress address) { - this.client = client; - - this.ticket = ticket; - this.serviceName = serviceName; - this.address = address; - - this.channel = connect(bootstrap).channel(); - - name = ("IPC Client (" + channel.hashCode() + ") connection to " + - address.toString() + - ((ticket == null) ? - " from an unknown user" : - (" from " + ticket.getName()))); - } - - /** - * Connect to channel - * - * @param bootstrap to connect to - * @return future of connection - */ - private ChannelFuture connect(final Bootstrap bootstrap) { - return bootstrap.remoteAddress(address).connect() - .addListener(new GenericFutureListener() { - @Override - public void operationComplete(final ChannelFuture f) throws Exception { - if (!f.isSuccess()) { - if (f.cause() instanceof SocketException) { - retryOrClose(bootstrap, connectFailureCounter++, f.cause()); - } else { - retryOrClose(bootstrap, ioFailureCounter++, f.cause()); - } - return; - } - channel = f.channel(); - - setupAuthorization(); - - ByteBuf b = channel.alloc().directBuffer(6); - createPreamble(b, authMethod); - channel.writeAndFlush(b).addListener(ChannelFutureListener.CLOSE_ON_FAILURE); - if (useSasl) { - UserGroupInformation ticket = AsyncRpcChannel.this.ticket.getUGI(); - if (authMethod == AuthMethod.KERBEROS) { - if (ticket != null && ticket.getRealUser() != null) { - ticket = ticket.getRealUser(); - } - } - SaslClientHandler saslHandler; - if (ticket == null) { - throw new FatalConnectionException("ticket/user is null"); - } - final UserGroupInformation realTicket = ticket; - saslHandler = ticket.doAs(new PrivilegedExceptionAction() { - @Override - public SaslClientHandler run() throws IOException { - return getSaslHandler(realTicket, bootstrap); - } - }); - if (saslHandler != null) { - // Sasl connect is successful. Let's set up Sasl channel handler - channel.pipeline().addFirst(saslHandler); - } else { - // fall back to simple auth because server told us so. - authMethod = AuthMethod.SIMPLE; - useSasl = false; - } - } else { - startHBaseConnection(f.channel()); - } - } - }); - } - - /** - * Start HBase connection - * - * @param ch channel to start connection on - */ - private void startHBaseConnection(Channel ch) { - ch.pipeline() - .addLast("frameDecoder", new LengthFieldBasedFrameDecoder(Integer.MAX_VALUE, 0, 4, 0, 4)); - ch.pipeline().addLast(new AsyncServerResponseHandler(this)); - try { - writeChannelHeader(ch).addListener(new GenericFutureListener() { - @Override - public void operationComplete(ChannelFuture future) throws Exception { - if (!future.isSuccess()) { - close(future.cause()); - return; - } - List callsToWrite; - synchronized (pendingCalls) { - connected = true; - callsToWrite = new ArrayList(pendingCalls.values()); - } - for (AsyncCall call : callsToWrite) { - writeRequest(call); - } - } - }); - } catch (IOException e) { - close(e); - } - } - - /** - * Get SASL handler - * @param bootstrap to reconnect to - * @return new SASL handler - * @throws java.io.IOException if handler failed to create - */ - private SaslClientHandler getSaslHandler(final UserGroupInformation realTicket, - final Bootstrap bootstrap) throws IOException { - return new SaslClientHandler(realTicket, authMethod, token, serverPrincipal, - client.fallbackAllowed, client.conf.get("hbase.rpc.protection", - SaslUtil.QualityOfProtection.AUTHENTICATION.name().toLowerCase()), - new SaslClientHandler.SaslExceptionHandler() { - @Override - public void handle(int retryCount, Random random, Throwable cause) { - try { - // Handle Sasl failure. Try to potentially get new credentials - handleSaslConnectionFailure(retryCount, cause, realTicket); - - // Try to reconnect - client.newTimeout(new TimerTask() { - @Override - public void run(Timeout timeout) throws Exception { - connect(bootstrap); - } - }, random.nextInt(reloginMaxBackoff) + 1, TimeUnit.MILLISECONDS); - } catch (IOException | InterruptedException e) { - close(e); - } - } - }, new SaslClientHandler.SaslSuccessfulConnectHandler() { - @Override - public void onSuccess(Channel channel) { - startHBaseConnection(channel); - } - }); - } - - /** - * Retry to connect or close - * - * @param bootstrap to connect with - * @param connectCounter amount of tries - * @param e exception of fail - */ - private void retryOrClose(final Bootstrap bootstrap, int connectCounter, Throwable e) { - if (connectCounter < client.maxRetries) { - client.newTimeout(new TimerTask() { - @Override public void run(Timeout timeout) throws Exception { - connect(bootstrap); - } - }, client.failureSleep, TimeUnit.MILLISECONDS); - } else { - client.failedServers.addToFailedServers(address); - close(e); - } - } - +public interface AsyncRpcChannel { /** * Calls method on channel * @param method to call - * @param controller to run call with * @param request to send + * @param cellScanner with cells to send * @param responsePrototype to construct response with + * @param messageConverter for the messages to expected result + * @param exceptionConverter for converting exceptions + * @param rpcTimeout timeout for request + * @param priority for request + * @return Promise for the response Message */ - public Promise callMethod(final Descriptors.MethodDescriptor method, - final PayloadCarryingRpcController controller, final Message request, - final Message responsePrototype) { - final AsyncCall call = - new AsyncCall(channel.eventLoop(), client.callIdCnt.getAndIncrement(), method, request, - controller, responsePrototype); - controller.notifyOnCancel(new RpcCallback() { - @Override - public void run(Object parameter) { - // TODO: do not need to call AsyncCall.setFailed? - synchronized (pendingCalls) { - pendingCalls.remove(call.id); - } - } - }); - // TODO: this should be handled by PayloadCarryingRpcController. - if (controller.isCanceled()) { - // To finish if the call was cancelled before we set the notification (race condition) - call.cancel(true); - return call; - } - - synchronized (pendingCalls) { - if (closed) { - Promise promise = channel.eventLoop().newPromise(); - promise.setFailure(new ConnectException()); - return promise; - } - pendingCalls.put(call.id, call); - // Add timeout for cleanup if none is present - if (cleanupTimer == null && call.getRpcTimeout() > 0) { - cleanupTimer = - client.newTimeout(timeoutTask, call.getRpcTimeout(), - TimeUnit.MILLISECONDS); - } - if (!connected) { - return call; - } - } - writeRequest(call); - return call; - } - - AsyncCall removePendingCall(int id) { - synchronized (pendingCalls) { - return pendingCalls.remove(id); - } - } - - /** - * Write the channel header - * - * @param channel to write to - * @return future of write - * @throws java.io.IOException on failure to write - */ - private ChannelFuture writeChannelHeader(Channel channel) throws IOException { - RPCProtos.ConnectionHeader.Builder headerBuilder = - RPCProtos.ConnectionHeader.newBuilder().setServiceName(serviceName); - - RPCProtos.UserInformation userInfoPB = buildUserInfo(ticket.getUGI(), authMethod); - if (userInfoPB != null) { - headerBuilder.setUserInfo(userInfoPB); - } - - if (client.codec != null) { - headerBuilder.setCellBlockCodecClass(client.codec.getClass().getCanonicalName()); - } - if (client.compressor != null) { - headerBuilder.setCellBlockCompressorClass(client.compressor.getClass().getCanonicalName()); - } - - headerBuilder.setVersionInfo(ProtobufUtil.getVersionInfo()); - RPCProtos.ConnectionHeader header = headerBuilder.build(); - - - int totalSize = IPCUtil.getTotalSizeWhenWrittenDelimited(header); - - ByteBuf b = channel.alloc().directBuffer(totalSize); - - b.writeInt(header.getSerializedSize()); - b.writeBytes(header.toByteArray()); - - return channel.writeAndFlush(b); - } - - /** - * Write request to channel - * - * @param call to write - */ - private void writeRequest(final AsyncCall call) { - try { - final RPCProtos.RequestHeader.Builder requestHeaderBuilder = RPCProtos.RequestHeader - .newBuilder(); - requestHeaderBuilder.setCallId(call.id) - .setMethodName(call.method.getName()).setRequestParam(call.param != null); - - if (Trace.isTracing()) { - Span s = Trace.currentSpan(); - requestHeaderBuilder.setTraceInfo(TracingProtos.RPCTInfo.newBuilder(). - setParentId(s.getSpanId()).setTraceId(s.getTraceId())); - } - - ByteBuffer cellBlock = client.buildCellBlock(call.controller.cellScanner()); - if (cellBlock != null) { - final RPCProtos.CellBlockMeta.Builder cellBlockBuilder = RPCProtos.CellBlockMeta - .newBuilder(); - cellBlockBuilder.setLength(cellBlock.limit()); - requestHeaderBuilder.setCellBlockMeta(cellBlockBuilder.build()); - } - // Only pass priority if there one. Let zero be same as no priority. - if (call.controller.getPriority() != 0) { - requestHeaderBuilder.setPriority(call.controller.getPriority()); - } - - RPCProtos.RequestHeader rh = requestHeaderBuilder.build(); - - int totalSize = IPCUtil.getTotalSizeWhenWrittenDelimited(rh, call.param); - if (cellBlock != null) { - totalSize += cellBlock.remaining(); - } - - ByteBuf b = channel.alloc().directBuffer(4 + totalSize); - try(ByteBufOutputStream out = new ByteBufOutputStream(b)) { - IPCUtil.write(out, rh, call.param, cellBlock); - } - - channel.writeAndFlush(b).addListener(new CallWriteListener(this, call.id)); - } catch (IOException e) { - close(e); - } - } + Future callMethod( + final Descriptors.MethodDescriptor method, + final Message request,final CellScanner cellScanner, + R responsePrototype, MessageConverter messageConverter, IOExceptionConverter + exceptionConverter, long rpcTimeout, int priority); /** - * Set up server authorization - * - * @throws java.io.IOException if auth setup failed + * Get the EventLoop on which this channel operated + * @return EventLoop */ - private void setupAuthorization() throws IOException { - SecurityInfo securityInfo = SecurityInfo.getInfo(serviceName); - this.useSasl = client.userProvider.isHBaseSecurityEnabled(); - - this.token = null; - if (useSasl && securityInfo != null) { - AuthenticationProtos.TokenIdentifier.Kind tokenKind = securityInfo.getTokenKind(); - if (tokenKind != null) { - TokenSelector tokenSelector = tokenHandlers.get(tokenKind); - if (tokenSelector != null) { - token = tokenSelector - .selectToken(new Text(client.clusterId), ticket.getUGI().getTokens()); - } else if (LOG.isDebugEnabled()) { - LOG.debug("No token selector found for type " + tokenKind); - } - } - String serverKey = securityInfo.getServerPrincipal(); - if (serverKey == null) { - throw new IOException("Can't obtain server Kerberos config key from SecurityInfo"); - } - this.serverPrincipal = SecurityUtil.getServerPrincipal(client.conf.get(serverKey), - address.getAddress().getCanonicalHostName().toLowerCase()); - if (LOG.isDebugEnabled()) { - LOG.debug("RPC Server Kerberos principal name for service=" + serviceName + " is " - + serverPrincipal); - } - } - - if (!useSasl) { - authMethod = AuthMethod.SIMPLE; - } else if (token != null) { - authMethod = AuthMethod.DIGEST; - } else { - authMethod = AuthMethod.KERBEROS; - } - - if (LOG.isDebugEnabled()) { - LOG.debug("Use " + authMethod + " authentication for service " + serviceName + - ", sasl=" + useSasl); - } - reloginMaxBackoff = client.conf.getInt("hbase.security.relogin.maxbackoff", 5000); - } - - /** - * Build the user information - * - * @param ugi User Group Information - * @param authMethod Authorization method - * @return UserInformation protobuf - */ - private RPCProtos.UserInformation buildUserInfo(UserGroupInformation ugi, AuthMethod authMethod) { - if (ugi == null || authMethod == AuthMethod.DIGEST) { - // Don't send user for token auth - return null; - } - RPCProtos.UserInformation.Builder userInfoPB = RPCProtos.UserInformation.newBuilder(); - if (authMethod == AuthMethod.KERBEROS) { - // Send effective user for Kerberos auth - userInfoPB.setEffectiveUser(ugi.getUserName()); - } else if (authMethod == AuthMethod.SIMPLE) { - //Send both effective user and real user for simple auth - userInfoPB.setEffectiveUser(ugi.getUserName()); - if (ugi.getRealUser() != null) { - userInfoPB.setRealUser(ugi.getRealUser().getUserName()); - } - } - return userInfoPB.build(); - } - - /** - * Create connection preamble - * - * @param byteBuf to write to - * @param authMethod to write - */ - private void createPreamble(ByteBuf byteBuf, AuthMethod authMethod) { - byteBuf.writeBytes(HConstants.RPC_HEADER); - byteBuf.writeByte(HConstants.RPC_CURRENT_VERSION); - byteBuf.writeByte(authMethod.code); - } + EventExecutor getEventExecutor(); /** * Close connection - * - * @param e exception on close + * @param cause of closure. */ - public void close(final Throwable e) { - client.removeConnection(this); - - // Move closing from the requesting thread to the channel thread - channel.eventLoop().execute(new Runnable() { - @Override - public void run() { - List toCleanup; - synchronized (pendingCalls) { - if (closed) { - return; - } - closed = true; - toCleanup = new ArrayList(pendingCalls.values()); - pendingCalls.clear(); - } - IOException closeException = null; - if (e != null) { - if (e instanceof IOException) { - closeException = (IOException) e; - } else { - closeException = new IOException(e); - } - } - // log the info - if (LOG.isDebugEnabled() && closeException != null) { - LOG.debug(name + ": closing ipc connection to " + address, closeException); - } - if (cleanupTimer != null) { - cleanupTimer.cancel(); - cleanupTimer = null; - } - for (AsyncCall call : toCleanup) { - call.setFailed(closeException != null ? closeException : new ConnectionClosingException( - "Call id=" + call.id + " on server " + address + " aborted: connection is closing")); - } - channel.disconnect().addListener(ChannelFutureListener.CLOSE); - if (LOG.isDebugEnabled()) { - LOG.debug(name + ": closed"); - } - } - }); - } - - /** - * Clean up calls. - * - * @param cleanAll true if all calls should be cleaned, false for only the timed out calls - */ - private void cleanupCalls() { - List toCleanup = new ArrayList(); - long currentTime = EnvironmentEdgeManager.currentTime(); - long nextCleanupTaskDelay = -1L; - synchronized (pendingCalls) { - for (Iterator iter = pendingCalls.values().iterator(); iter.hasNext();) { - AsyncCall call = iter.next(); - long timeout = call.getRpcTimeout(); - if (timeout > 0) { - if (currentTime - call.getStartTime() >= timeout) { - iter.remove(); - toCleanup.add(call); - } else { - if (nextCleanupTaskDelay < 0 || timeout < nextCleanupTaskDelay) { - nextCleanupTaskDelay = timeout; - } - } - } - } - if (nextCleanupTaskDelay > 0) { - cleanupTimer = - client.newTimeout(timeoutTask, nextCleanupTaskDelay, - TimeUnit.MILLISECONDS); - } else { - cleanupTimer = null; - } - } - for (AsyncCall call : toCleanup) { - call.setFailed(new CallTimeoutException("Call id=" + call.id + ", waitTime=" - + (currentTime - call.getStartTime()) + ", rpcTimeout=" + call.getRpcTimeout())); - } - } + void close(Throwable cause); /** * Check if the connection is alive * * @return true if alive */ - public boolean isAlive() { - return channel.isOpen(); - } + boolean isAlive(); /** - * Check if user should authenticate over Kerberos - * - * @return true if should be authenticated over Kerberos - * @throws java.io.IOException on failure of check + * Get the address on which this channel operates + * @return InetSocketAddress */ - private synchronized boolean shouldAuthenticateOverKrb() throws IOException { - UserGroupInformation loginUser = UserGroupInformation.getLoginUser(); - UserGroupInformation currentUser = UserGroupInformation.getCurrentUser(); - UserGroupInformation realUser = currentUser.getRealUser(); - return authMethod == AuthMethod.KERBEROS && - loginUser != null && - //Make sure user logged in using Kerberos either keytab or TGT - loginUser.hasKerberosCredentials() && - // relogin only in case it is the login user (e.g. JT) - // or superuser (like oozie). - (loginUser.equals(currentUser) || loginUser.equals(realUser)); - } - - /** - * If multiple clients with the same principal try to connect - * to the same server at the same time, the server assumes a - * replay attack is in progress. This is a feature of kerberos. - * In order to work around this, what is done is that the client - * backs off randomly and tries to initiate the connection - * again. - * The other problem is to do with ticket expiry. To handle that, - * a relogin is attempted. - *

- * The retry logic is governed by the {@link #shouldAuthenticateOverKrb} - * method. In case when the user doesn't have valid credentials, we don't - * need to retry (from cache or ticket). In such cases, it is prudent to - * throw a runtime exception when we receive a SaslException from the - * underlying authentication implementation, so there is no retry from - * other high level (for eg, HCM or HBaseAdmin). - *

- * - * @param currRetries retry count - * @param ex exception describing fail - * @param user which is trying to connect - * @throws java.io.IOException if IO fail - * @throws InterruptedException if thread is interrupted - */ - private void handleSaslConnectionFailure(final int currRetries, final Throwable ex, - final UserGroupInformation user) throws IOException, InterruptedException { - user.doAs(new PrivilegedExceptionAction() { - public Void run() throws IOException, InterruptedException { - if (shouldAuthenticateOverKrb()) { - if (currRetries < MAX_SASL_RETRIES) { - LOG.debug("Exception encountered while connecting to the server : " + ex); - //try re-login - if (UserGroupInformation.isLoginKeytabBased()) { - UserGroupInformation.getLoginUser().reloginFromKeytab(); - } else { - UserGroupInformation.getLoginUser().reloginFromTicketCache(); - } - - // Should reconnect - return null; - } else { - String msg = "Couldn't setup connection for " + - UserGroupInformation.getLoginUser().getUserName() + - " to " + serverPrincipal; - LOG.warn(msg); - throw (IOException) new IOException(msg).initCause(ex); - } - } else { - LOG.warn("Exception encountered while connecting to " + - "the server : " + ex); - } - if (ex instanceof RemoteException) { - throw (RemoteException) ex; - } - if (ex instanceof SaslException) { - String msg = "SASL authentication failed." + - " The most likely cause is missing or invalid credentials." + - " Consider 'kinit'."; - LOG.fatal(msg, ex); - throw new RuntimeException(msg, ex); - } - throw new IOException(ex); - } - }); - } - - public int getConnectionHashCode() { - return ConnectionId.hashCode(ticket, serviceName, address); - } - - @Override - public String toString() { - return this.address.toString() + "/" + this.serviceName + "/" + this.ticket; - } - - /** - * Listens to call writes and fails if write failed - */ - private static final class CallWriteListener implements ChannelFutureListener { - private final AsyncRpcChannel rpcChannel; - private final int id; - - public CallWriteListener(AsyncRpcChannel asyncRpcChannel, int id) { - this.rpcChannel = asyncRpcChannel; - this.id = id; - } - - @Override - public void operationComplete(ChannelFuture future) throws Exception { - if (!future.isSuccess()) { - AsyncCall call = rpcChannel.removePendingCall(id); - if (call != null) { - if (future.cause() instanceof IOException) { - call.setFailed((IOException) future.cause()); - } else { - call.setFailed(new IOException(future.cause())); - } - } - } - } - } + InetSocketAddress getAddress(); } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AsyncRpcChannelImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AsyncRpcChannelImpl.java new file mode 100644 index 0000000..af1ef60 --- /dev/null +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AsyncRpcChannelImpl.java @@ -0,0 +1,758 @@ +/** + * 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.ipc; + +import io.netty.bootstrap.Bootstrap; +import io.netty.buffer.ByteBuf; +import io.netty.buffer.ByteBufOutputStream; +import io.netty.channel.Channel; +import io.netty.channel.ChannelFuture; +import io.netty.channel.ChannelFutureListener; +import io.netty.channel.EventLoop; +import io.netty.handler.codec.LengthFieldBasedFrameDecoder; +import io.netty.util.Timeout; +import io.netty.util.TimerTask; +import io.netty.util.concurrent.GenericFutureListener; + +import java.io.IOException; +import java.net.ConnectException; +import java.net.InetSocketAddress; +import java.net.SocketException; +import java.nio.ByteBuffer; +import java.security.PrivilegedExceptionAction; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Random; +import java.util.concurrent.TimeUnit; + +import javax.security.sasl.SaslException; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.CellScanner; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.client.Future; +import org.apache.hadoop.hbase.exceptions.ConnectionClosingException; +import org.apache.hadoop.hbase.protobuf.ProtobufUtil; +import org.apache.hadoop.hbase.protobuf.generated.AuthenticationProtos; +import org.apache.hadoop.hbase.protobuf.generated.RPCProtos; +import org.apache.hadoop.hbase.protobuf.generated.TracingProtos; +import org.apache.hadoop.hbase.security.AuthMethod; +import org.apache.hadoop.hbase.security.SaslClientHandler; +import org.apache.hadoop.hbase.security.SaslUtil; +import org.apache.hadoop.hbase.security.SecurityInfo; +import org.apache.hadoop.hbase.security.User; +import org.apache.hadoop.hbase.security.token.AuthenticationTokenSelector; +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.ipc.RemoteException; +import org.apache.hadoop.security.SecurityUtil; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.security.token.Token; +import org.apache.hadoop.security.token.TokenIdentifier; +import org.apache.hadoop.security.token.TokenSelector; +import org.apache.htrace.Span; +import org.apache.htrace.Trace; + +import com.google.protobuf.Descriptors; +import com.google.protobuf.Message; + +/** + * Netty RPC channel + */ +@InterfaceAudience.Private +public class AsyncRpcChannelImpl implements AsyncRpcChannel { + private static final Log LOG = LogFactory.getLog(AsyncRpcChannelImpl.class.getName()); + + private static final int MAX_SASL_RETRIES = 5; + + protected final static Map> tokenHandlers = new HashMap<>(); + + static { + tokenHandlers.put(AuthenticationProtos.TokenIdentifier.Kind.HBASE_AUTH_TOKEN, + new AuthenticationTokenSelector()); + } + + final AsyncRpcClient client; + // Contains the channel to work with. + // Only exists when connected + private Channel channel; + + String name; + + final User ticket; + final String serviceName; + final InetSocketAddress address; + + private int ioFailureCounter = 0; + private int connectFailureCounter = 0; + + boolean useSasl; + AuthMethod authMethod; + private int reloginMaxBackoff; + private Token token; + private String serverPrincipal; + + + // NOTE: closed and connected flags below are only changed when a lock on pendingCalls + private final Map pendingCalls = new HashMap(); + private boolean connected = false; + private boolean closed = false; + + private Timeout cleanupTimer; + + private final TimerTask timeoutTask = new TimerTask() { + @Override + public void run(Timeout timeout) throws Exception { + cleanupCalls(); + } + }; + + /** + * Constructor for netty RPC channel + * + * @param bootstrap to construct channel on + * @param client to connect with + * @param ticket of user which uses connection + * @param serviceName name of service to connect to + * @param address to connect to + */ + public AsyncRpcChannelImpl(Bootstrap bootstrap, final AsyncRpcClient client, User ticket, + String serviceName, InetSocketAddress address) { + this.client = client; + + this.ticket = ticket; + this.serviceName = serviceName; + this.address = address; + + this.channel = connect(bootstrap).channel(); + + name = ("IPC Client (" + channel.hashCode() + ") connection to " + + address.toString() + + ((ticket == null) ? + " from an unknown user" : + (" from " + ticket.getName()))); + } + + /** + * Connect to channel + * + * @param bootstrap to connect to + * @return future of connection + */ + private ChannelFuture connect(final Bootstrap bootstrap) { + return bootstrap.remoteAddress(address).connect() + .addListener(new GenericFutureListener() { + @Override + public void operationComplete(final ChannelFuture f) throws Exception { + if (!f.isSuccess()) { + if (f.cause() instanceof SocketException) { + retryOrClose(bootstrap, connectFailureCounter++, f.cause()); + } else { + retryOrClose(bootstrap, ioFailureCounter++, f.cause()); + } + return; + } + channel = f.channel(); + + setupAuthorization(); + + ByteBuf b = channel.alloc().directBuffer(6); + createPreamble(b, authMethod); + channel.writeAndFlush(b).addListener(ChannelFutureListener.CLOSE_ON_FAILURE); + if (useSasl) { + UserGroupInformation ticket = AsyncRpcChannelImpl.this.ticket.getUGI(); + if (authMethod == AuthMethod.KERBEROS) { + if (ticket != null && ticket.getRealUser() != null) { + ticket = ticket.getRealUser(); + } + } + SaslClientHandler saslHandler; + if (ticket == null) { + throw new FatalConnectionException("ticket/user is null"); + } + final UserGroupInformation realTicket = ticket; + saslHandler = ticket.doAs(new PrivilegedExceptionAction() { + @Override + public SaslClientHandler run() throws IOException { + return getSaslHandler(realTicket, bootstrap); + } + }); + if (saslHandler != null) { + // Sasl connect is successful. Let's set up Sasl channel handler + channel.pipeline().addFirst(saslHandler); + } else { + // fall back to simple auth because server told us so. + authMethod = AuthMethod.SIMPLE; + useSasl = false; + } + } else { + startHBaseConnection(f.channel()); + } + } + }); + } + + /** + * Start HBase connection + * + * @param ch channel to start connection on + */ + private void startHBaseConnection(Channel ch) { + ch.pipeline() + .addLast("frameDecoder", new LengthFieldBasedFrameDecoder(Integer.MAX_VALUE, 0, 4, 0, 4)); + ch.pipeline().addLast(new AsyncServerResponseHandler(this)); + try { + writeChannelHeader(ch).addListener(new GenericFutureListener() { + @Override + public void operationComplete(ChannelFuture future) throws Exception { + if (!future.isSuccess()) { + close(future.cause()); + return; + } + List callsToWrite; + synchronized (pendingCalls) { + connected = true; + callsToWrite = new ArrayList(pendingCalls.values()); + } + for (AsyncCall call : callsToWrite) { + writeRequest(call); + } + } + }); + } catch (IOException e) { + close(e); + } + } + + /** + * Get SASL handler + * @param bootstrap to reconnect to + * @return new SASL handler + * @throws java.io.IOException if handler failed to create + */ + private SaslClientHandler getSaslHandler(final UserGroupInformation realTicket, + final Bootstrap bootstrap) throws IOException { + return new SaslClientHandler(realTicket, authMethod, token, serverPrincipal, + client.fallbackAllowed, client.conf.get("hbase.rpc.protection", + SaslUtil.QualityOfProtection.AUTHENTICATION.name().toLowerCase()), + new SaslClientHandler.SaslExceptionHandler() { + @Override + public void handle(int retryCount, Random random, Throwable cause) { + try { + // Handle Sasl failure. Try to potentially get new credentials + handleSaslConnectionFailure(retryCount, cause, realTicket); + + // Try to reconnect + client.newTimeout(new TimerTask() { + @Override + public void run(Timeout timeout) throws Exception { + connect(bootstrap); + } + }, random.nextInt(reloginMaxBackoff) + 1, TimeUnit.MILLISECONDS); + } catch (IOException | InterruptedException e) { + close(e); + } + } + }, new SaslClientHandler.SaslSuccessfulConnectHandler() { + @Override + public void onSuccess(Channel channel) { + startHBaseConnection(channel); + } + }); + } + + /** + * Retry to connect or close + * + * @param bootstrap to connect with + * @param connectCounter amount of tries + * @param e exception of fail + */ + private void retryOrClose(final Bootstrap bootstrap, int connectCounter, Throwable e) { + if (connectCounter < client.maxRetries) { + client.newTimeout(new TimerTask() { + @Override public void run(Timeout timeout) throws Exception { + connect(bootstrap); + } + }, client.failureSleep, TimeUnit.MILLISECONDS); + } else { + client.failedServers.addToFailedServers(address); + close(e); + } + } + + /** + * Calls method on channel + * @param method to call + * @param request to send + * @param cellScanner with cells to send + * @param responsePrototype to construct response with + * @param rpcTimeout timeout for request + * @param priority for request + * @return Promise for the response Message + */ + @Override + public Future callMethod( + final Descriptors.MethodDescriptor method, + final Message request,final CellScanner cellScanner, + R responsePrototype, MessageConverter messageConverter, IOExceptionConverter + exceptionConverter, long rpcTimeout, int priority) { + final AsyncCall call = + new AsyncCall<>(this, client.callIdCnt.getAndIncrement(), method, request, + cellScanner, responsePrototype, messageConverter, exceptionConverter, rpcTimeout, priority); + + synchronized (pendingCalls) { + if (closed) { + call.setFailure(new ConnectException()); + return call; + } + pendingCalls.put(call.id, call); + // Add timeout for cleanup if none is present + if (cleanupTimer == null && call.getRpcTimeout() > 0) { + cleanupTimer = + client.newTimeout(timeoutTask, call.getRpcTimeout(), + TimeUnit.MILLISECONDS); + } + if (!connected) { + return call; + } + } + writeRequest(call); + return call; + } + + @Override + public EventLoop getEventExecutor() { + return this.channel.eventLoop(); + } + + AsyncCall removePendingCall(int id) { + synchronized (pendingCalls) { + return pendingCalls.remove(id); + } + } + + /** + * Write the channel header + * + * @param channel to write to + * @return future of write + * @throws java.io.IOException on failure to write + */ + private ChannelFuture writeChannelHeader(Channel channel) throws IOException { + RPCProtos.ConnectionHeader.Builder headerBuilder = + RPCProtos.ConnectionHeader.newBuilder().setServiceName(serviceName); + + RPCProtos.UserInformation userInfoPB = buildUserInfo(ticket.getUGI(), authMethod); + if (userInfoPB != null) { + headerBuilder.setUserInfo(userInfoPB); + } + + if (client.codec != null) { + headerBuilder.setCellBlockCodecClass(client.codec.getClass().getCanonicalName()); + } + if (client.compressor != null) { + headerBuilder.setCellBlockCompressorClass(client.compressor.getClass().getCanonicalName()); + } + + headerBuilder.setVersionInfo(ProtobufUtil.getVersionInfo()); + RPCProtos.ConnectionHeader header = headerBuilder.build(); + + + int totalSize = IPCUtil.getTotalSizeWhenWrittenDelimited(header); + + ByteBuf b = channel.alloc().directBuffer(totalSize); + + b.writeInt(header.getSerializedSize()); + b.writeBytes(header.toByteArray()); + + return channel.writeAndFlush(b); + } + + /** + * Write request to channel + * + * @param call to write + */ + private void writeRequest(final AsyncCall call) { + try { + final RPCProtos.RequestHeader.Builder requestHeaderBuilder = RPCProtos.RequestHeader + .newBuilder(); + requestHeaderBuilder.setCallId(call.id) + .setMethodName(call.method.getName()).setRequestParam(call.param != null); + + if (Trace.isTracing()) { + Span s = Trace.currentSpan(); + requestHeaderBuilder.setTraceInfo(TracingProtos.RPCTInfo.newBuilder(). + setParentId(s.getSpanId()).setTraceId(s.getTraceId())); + } + + ByteBuffer cellBlock = client.buildCellBlock(call.cellScanner()); + if (cellBlock != null) { + final RPCProtos.CellBlockMeta.Builder cellBlockBuilder = RPCProtos.CellBlockMeta + .newBuilder(); + cellBlockBuilder.setLength(cellBlock.limit()); + requestHeaderBuilder.setCellBlockMeta(cellBlockBuilder.build()); + } + // Only pass priority if there one. Let zero be same as no priority. + if (call.getPriority() != 0) { + requestHeaderBuilder.setPriority(call.getPriority()); + } + + RPCProtos.RequestHeader rh = requestHeaderBuilder.build(); + + int totalSize = IPCUtil.getTotalSizeWhenWrittenDelimited(rh, call.param); + if (cellBlock != null) { + totalSize += cellBlock.remaining(); + } + + ByteBuf b = channel.alloc().directBuffer(4 + totalSize); + try(ByteBufOutputStream out = new ByteBufOutputStream(b)) { + IPCUtil.write(out, rh, call.param, cellBlock); + } + + channel.writeAndFlush(b).addListener(new CallWriteListener(this, call.id)); + } catch (IOException e) { + close(e); + } + } + + /** + * Set up server authorization + * + * @throws java.io.IOException if auth setup failed + */ + private void setupAuthorization() throws IOException { + SecurityInfo securityInfo = SecurityInfo.getInfo(serviceName); + this.useSasl = client.userProvider.isHBaseSecurityEnabled(); + + this.token = null; + if (useSasl && securityInfo != null) { + AuthenticationProtos.TokenIdentifier.Kind tokenKind = securityInfo.getTokenKind(); + if (tokenKind != null) { + TokenSelector tokenSelector = tokenHandlers.get(tokenKind); + if (tokenSelector != null) { + token = tokenSelector + .selectToken(new Text(client.clusterId), ticket.getUGI().getTokens()); + } else if (LOG.isDebugEnabled()) { + LOG.debug("No token selector found for type " + tokenKind); + } + } + String serverKey = securityInfo.getServerPrincipal(); + if (serverKey == null) { + throw new IOException("Can't obtain server Kerberos config key from SecurityInfo"); + } + this.serverPrincipal = SecurityUtil.getServerPrincipal(client.conf.get(serverKey), + address.getAddress().getCanonicalHostName().toLowerCase()); + if (LOG.isDebugEnabled()) { + LOG.debug("RPC Server Kerberos principal name for service=" + serviceName + " is " + + serverPrincipal); + } + } + + if (!useSasl) { + authMethod = AuthMethod.SIMPLE; + } else if (token != null) { + authMethod = AuthMethod.DIGEST; + } else { + authMethod = AuthMethod.KERBEROS; + } + + if (LOG.isDebugEnabled()) { + LOG.debug("Use " + authMethod + " authentication for service " + serviceName + + ", sasl=" + useSasl); + } + reloginMaxBackoff = client.conf.getInt("hbase.security.relogin.maxbackoff", 5000); + } + + /** + * Build the user information + * + * @param ugi User Group Information + * @param authMethod Authorization method + * @return UserInformation protobuf + */ + private RPCProtos.UserInformation buildUserInfo(UserGroupInformation ugi, AuthMethod authMethod) { + if (ugi == null || authMethod == AuthMethod.DIGEST) { + // Don't send user for token auth + return null; + } + RPCProtos.UserInformation.Builder userInfoPB = RPCProtos.UserInformation.newBuilder(); + if (authMethod == AuthMethod.KERBEROS) { + // Send effective user for Kerberos auth + userInfoPB.setEffectiveUser(ugi.getUserName()); + } else if (authMethod == AuthMethod.SIMPLE) { + //Send both effective user and real user for simple auth + userInfoPB.setEffectiveUser(ugi.getUserName()); + if (ugi.getRealUser() != null) { + userInfoPB.setRealUser(ugi.getRealUser().getUserName()); + } + } + return userInfoPB.build(); + } + + /** + * Create connection preamble + * + * @param byteBuf to write to + * @param authMethod to write + */ + private void createPreamble(ByteBuf byteBuf, AuthMethod authMethod) { + byteBuf.writeBytes(HConstants.RPC_HEADER); + byteBuf.writeByte(HConstants.RPC_CURRENT_VERSION); + byteBuf.writeByte(authMethod.code); + } + + /** + * Close connection + * + * @param e exception on close + */ + public void close(final Throwable e) { + client.removeConnection(this); + + // Move closing from the requesting thread to the channel thread + channel.eventLoop().execute(new Runnable() { + @Override + public void run() { + List toCleanup; + synchronized (pendingCalls) { + if (closed) { + return; + } + closed = true; + toCleanup = new ArrayList(pendingCalls.values()); + pendingCalls.clear(); + } + IOException closeException = null; + if (e != null) { + if (e instanceof IOException) { + closeException = (IOException) e; + } else { + closeException = new IOException(e); + } + } + // log the info + if (LOG.isDebugEnabled() && closeException != null) { + LOG.debug(name + ": closing ipc connection to " + address, closeException); + } + if (cleanupTimer != null) { + cleanupTimer.cancel(); + cleanupTimer = null; + } + for (AsyncCall call : toCleanup) { + call.setFailed(closeException != null ? closeException : new ConnectionClosingException( + "Call id=" + call.id + " on server " + address + " aborted: connection is closing")); + } + channel.disconnect().addListener(ChannelFutureListener.CLOSE); + if (LOG.isDebugEnabled()) { + LOG.debug(name + ": closed"); + } + } + }); + } + + /** + * Clean up calls. + */ + private void cleanupCalls() { + List toCleanup = new ArrayList(); + long currentTime = EnvironmentEdgeManager.currentTime(); + long nextCleanupTaskDelay = -1L; + synchronized (pendingCalls) { + for (Iterator iter = pendingCalls.values().iterator(); iter.hasNext();) { + AsyncCall call = iter.next(); + long timeout = call.getRpcTimeout(); + if (timeout > 0) { + if (currentTime - call.getStartTime() >= timeout) { + iter.remove(); + toCleanup.add(call); + } else { + if (nextCleanupTaskDelay < 0 || timeout < nextCleanupTaskDelay) { + nextCleanupTaskDelay = timeout; + } + } + } + } + if (nextCleanupTaskDelay > 0) { + cleanupTimer = + client.newTimeout(timeoutTask, nextCleanupTaskDelay, + TimeUnit.MILLISECONDS); + } else { + cleanupTimer = null; + } + } + for (AsyncCall call : toCleanup) { + call.setFailed(new CallTimeoutException("Call id=" + call.id + ", waitTime=" + + (currentTime - call.getStartTime()) + ", rpcTimeout=" + call.getRpcTimeout())); + } + } + + /** + * Check if the connection is alive + * + * @return true if alive + */ + public boolean isAlive() { + return channel.isOpen(); + } + + @Override + public InetSocketAddress getAddress() { + return this.address; + } + + /** + * Check if user should authenticate over Kerberos + * + * @return true if should be authenticated over Kerberos + * @throws java.io.IOException on failure of check + */ + private synchronized boolean shouldAuthenticateOverKrb() throws IOException { + UserGroupInformation loginUser = UserGroupInformation.getLoginUser(); + UserGroupInformation currentUser = UserGroupInformation.getCurrentUser(); + UserGroupInformation realUser = currentUser.getRealUser(); + return authMethod == AuthMethod.KERBEROS && + loginUser != null && + //Make sure user logged in using Kerberos either keytab or TGT + loginUser.hasKerberosCredentials() && + // relogin only in case it is the login user (e.g. JT) + // or superuser (like oozie). + (loginUser.equals(currentUser) || loginUser.equals(realUser)); + } + + /** + * If multiple clients with the same principal try to connect + * to the same server at the same time, the server assumes a + * replay attack is in progress. This is a feature of kerberos. + * In order to work around this, what is done is that the client + * backs off randomly and tries to initiate the connection + * again. + * The other problem is to do with ticket expiry. To handle that, + * a relogin is attempted. + *

+ * The retry logic is governed by the {@link #shouldAuthenticateOverKrb} + * method. In case when the user doesn't have valid credentials, we don't + * need to retry (from cache or ticket). In such cases, it is prudent to + * throw a runtime exception when we receive a SaslException from the + * underlying authentication implementation, so there is no retry from + * other high level (for eg, HCM or HBaseAdmin). + *

+ * + * @param currRetries retry count + * @param ex exception describing fail + * @param user which is trying to connect + * @throws java.io.IOException if IO fail + * @throws InterruptedException if thread is interrupted + */ + private void handleSaslConnectionFailure(final int currRetries, final Throwable ex, + final UserGroupInformation user) throws IOException, InterruptedException { + user.doAs(new PrivilegedExceptionAction() { + public Void run() throws IOException, InterruptedException { + if (shouldAuthenticateOverKrb()) { + if (currRetries < MAX_SASL_RETRIES) { + LOG.debug("Exception encountered while connecting to the server : " + ex); + //try re-login + if (UserGroupInformation.isLoginKeytabBased()) { + UserGroupInformation.getLoginUser().reloginFromKeytab(); + } else { + UserGroupInformation.getLoginUser().reloginFromTicketCache(); + } + + // Should reconnect + return null; + } else { + String msg = "Couldn't setup connection for " + + UserGroupInformation.getLoginUser().getUserName() + + " to " + serverPrincipal; + LOG.warn(msg); + throw (IOException) new IOException(msg).initCause(ex); + } + } else { + LOG.warn("Exception encountered while connecting to " + + "the server : " + ex); + } + if (ex instanceof RemoteException) { + throw (RemoteException) ex; + } + if (ex instanceof SaslException) { + String msg = "SASL authentication failed." + + " The most likely cause is missing or invalid credentials." + + " Consider 'kinit'."; + LOG.fatal(msg, ex); + throw new RuntimeException(msg, ex); + } + throw new IOException(ex); + } + }); + } + + @Override + public int hashCode() { + return ConnectionId.hashCode(ticket, serviceName, address); + } + + @Override + public boolean equals(Object o) { + if (this == o) + return true; + if (o == null || getClass() != o.getClass()) + return false; + + AsyncRpcChannelImpl that = (AsyncRpcChannelImpl) o; + + return hashCode() == that.hashCode(); + } + + @Override + public String toString() { + return this.address.toString() + "/" + this.serviceName + "/" + this.ticket; + } + /** + * Listens to call writes and fails if write failed + */ + private static final class CallWriteListener implements ChannelFutureListener { + private final AsyncRpcChannelImpl rpcChannel; + private final int id; + + public CallWriteListener(AsyncRpcChannelImpl asyncRpcChannel, int id) { + this.rpcChannel = asyncRpcChannel; + this.id = id; + } + + @Override + public void operationComplete(ChannelFuture future) throws Exception { + if (!future.isSuccess()) { + AsyncCall call = rpcChannel.removePendingCall(id); + if (call != null) { + if (future.cause() instanceof IOException) { + call.setFailed((IOException) future.cause()); + } else { + call.setFailed(new IOException(future.cause())); + } + } + } + } + } +} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AsyncRpcClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AsyncRpcClient.java index 2e4d0a6..ecbcd65 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AsyncRpcClient.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AsyncRpcClient.java @@ -21,6 +21,7 @@ import io.netty.bootstrap.Bootstrap; import io.netty.channel.Channel; import io.netty.channel.ChannelInitializer; import io.netty.channel.ChannelOption; +import io.netty.channel.EventLoop; import io.netty.channel.EventLoopGroup; import io.netty.channel.epoll.EpollEventLoopGroup; import io.netty.channel.epoll.EpollSocketChannel; @@ -30,9 +31,6 @@ import io.netty.channel.socket.nio.NioSocketChannel; import io.netty.util.HashedWheelTimer; import io.netty.util.Timeout; import io.netty.util.TimerTask; -import io.netty.util.concurrent.Future; -import io.netty.util.concurrent.GenericFutureListener; -import io.netty.util.concurrent.Promise; import java.io.IOException; import java.net.InetSocketAddress; @@ -52,6 +50,8 @@ import org.apache.hadoop.hbase.HBaseInterfaceAudience; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.client.ResponseFutureListener; +import org.apache.hadoop.hbase.client.Future; import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.util.JVM; import org.apache.hadoop.hbase.util.Pair; @@ -103,7 +103,7 @@ public class AsyncRpcClient extends AbstractRpcClient { @VisibleForTesting static Pair> GLOBAL_EVENT_LOOP_GROUP; - private synchronized static Pair> + protected synchronized static Pair> getGlobalEventLoopGroup(Configuration conf) { if (GLOBAL_EVENT_LOOP_GROUP == null) { GLOBAL_EVENT_LOOP_GROUP = createEventLoopGroup(conf); @@ -223,9 +223,21 @@ public class AsyncRpcClient extends AbstractRpcClient { if (pcrc == null) { pcrc = new PayloadCarryingRpcController(); } - final AsyncRpcChannel connection = createRpcChannel(md.getService().getName(), addr, ticket); + final AsyncRpcChannel connection = + createRpcChannel(md.getService().getName(), addr, ticket); + + final Future promise = connection.callMethod(md, param, pcrc.cellScanner(), returnType, + getMessageConverterWithRpcController(pcrc), null, pcrc.getCallTimeout(), pcrc.getPriority + ()); + + pcrc.notifyOnCancel(new RpcCallback() { + @Override + public void run(Object parameter) { + // Will automatically fail the promise with CancellationException + promise.cancel(true); + } + }); - Promise promise = connection.callMethod(md, pcrc, param, returnType); long timeout = pcrc.hasCallTimeout() ? pcrc.getCallTimeout() : 0; try { Message response = timeout > 0 ? promise.get(timeout, TimeUnit.MILLISECONDS) : promise.get(); @@ -251,38 +263,52 @@ public class AsyncRpcClient extends AbstractRpcClient { try { connection = createRpcChannel(md.getService().getName(), addr, ticket); - connection.callMethod(md, pcrc, param, returnType).addListener( - new GenericFutureListener>() { - @Override - public void operationComplete(Future future) throws Exception { - if(!future.isSuccess()){ - Throwable cause = future.cause(); - if (cause instanceof IOException) { - pcrc.setFailed((IOException) cause); - }else{ - pcrc.setFailed(new IOException(cause)); - } - }else{ - try { - done.run(future.get()); - }catch (ExecutionException e){ - Throwable cause = e.getCause(); - if (cause instanceof IOException) { - pcrc.setFailed((IOException) cause); - }else{ - pcrc.setFailed(new IOException(cause)); - } - }catch (InterruptedException e){ - pcrc.setFailed(new IOException(e)); - } - } - } - }); + connection.callMethod(md, param, pcrc.cellScanner(), returnType, + getMessageConverterWithRpcController(pcrc), null, + pcrc.getCallTimeout(), pcrc.getPriority()) + .addListener(new ResponseFutureListener() { + @Override + public void operationComplete(Future future) throws Exception { + if (!future.isSuccess()) { + Throwable cause = future.cause(); + if (cause instanceof IOException) { + pcrc.setFailed((IOException) cause); + } else { + pcrc.setFailed(new IOException(cause)); + } + } else { + try { + done.run(future.get()); + } catch (ExecutionException e) { + Throwable cause = e.getCause(); + if (cause instanceof IOException) { + pcrc.setFailed((IOException) cause); + } else { + pcrc.setFailed(new IOException(cause)); + } + } catch (InterruptedException e) { + pcrc.setFailed(new IOException(e)); + } + } + } + }); } catch (StoppedRpcClientException|FailedServerException e) { pcrc.setFailed(e); } } + private MessageConverter getMessageConverterWithRpcController( + final PayloadCarryingRpcController pcrc) { + return new + MessageConverter() { + @Override + public Message convert(Message msg, CellScanner cellScanner) { + pcrc.setCellScanner(cellScanner); + return msg; + } + }; + } + private boolean closed = false; /** @@ -308,6 +334,11 @@ public class AsyncRpcClient extends AbstractRpcClient { } } + @Override + public EventLoop getEventExecutor() { + return this.bootstrap.group().next(); + } + /** * Create a cell scanner * @@ -330,6 +361,13 @@ public class AsyncRpcClient extends AbstractRpcClient { return ipcUtil.buildCellBlock(this.codec, this.compressor, cells); } + @Override + public AsyncRpcChannel createRpcChannel(String serviceName, ServerName sn, User user) + throws StoppedRpcClientException, FailedServerException { + return this.createRpcChannel(serviceName, new InetSocketAddress(sn.getHostname(), sn.getPort()), + user); + } + /** * Creates an RPC client * @@ -340,7 +378,7 @@ public class AsyncRpcClient extends AbstractRpcClient { * @throws StoppedRpcClientException when Rpc client is stopped * @throws FailedServerException if server failed */ - private AsyncRpcChannel createRpcChannel(String serviceName, InetSocketAddress location, + public AsyncRpcChannel createRpcChannel(String serviceName, InetSocketAddress location, User ticket) throws StoppedRpcClientException, FailedServerException { // Check if server is failed if (this.failedServers.isFailedServer(location)) { @@ -361,7 +399,7 @@ public class AsyncRpcClient extends AbstractRpcClient { } rpcChannel = connections.get(hashCode); if (rpcChannel == null) { - rpcChannel = new AsyncRpcChannel(this.bootstrap, this, ticket, serviceName, location); + rpcChannel = new AsyncRpcChannelImpl(this.bootstrap, this, ticket, serviceName, location); connections.put(hashCode, rpcChannel); } } @@ -384,8 +422,8 @@ public class AsyncRpcClient extends AbstractRpcClient { synchronized (connections) { for (AsyncRpcChannel rpcChannel : connections.values()) { if (rpcChannel.isAlive() && - rpcChannel.address.getPort() == sn.getPort() && - rpcChannel.address.getHostName().contentEquals(sn.getHostname())) { + rpcChannel.getAddress().getPort() == sn.getPort() && + rpcChannel.getAddress().getHostName().contentEquals(sn.getHostname())) { LOG.info("The server on " + sn.toString() + " is dead - stopping the connection " + rpcChannel.toString()); rpcChannel.close(null); @@ -396,9 +434,10 @@ public class AsyncRpcClient extends AbstractRpcClient { /** * Remove connection from pool + * @param connection to remove */ public void removeConnection(AsyncRpcChannel connection) { - int connectionHashCode = connection.getConnectionHashCode(); + int connectionHashCode = connection.hashCode(); synchronized (connections) { // we use address as cache key, so we should check here to prevent removing the // wrong connection @@ -413,17 +452,7 @@ public class AsyncRpcClient extends AbstractRpcClient { } } - /** - * Creates a "channel" that can be used by a protobuf service. Useful setting up - * protobuf stubs. - * - * @param sn server name describing location of server - * @param user which is to use the connection - * @param rpcTimeout default rpc operation timeout - * - * @return A rpc channel that goes via this rpc client instance. - * @throws IOException when channel could not be created - */ + @Override public RpcChannel createRpcChannel(final ServerName sn, final User user, int rpcTimeout) { return new RpcChannelImplementation(this, sn, user, rpcTimeout); } @@ -467,6 +496,13 @@ public class AsyncRpcClient extends AbstractRpcClient { } } + /** + * Get a new timeout on this RPC client + * @param task to run at timeout + * @param delay for the timeout + * @param unit time unit for the timeout + * @return Timeout + */ Timeout newTimeout(TimerTask task, long delay, TimeUnit unit) { return WHEEL_TIMER.newTimeout(task, delay, unit); } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AsyncServerResponseHandler.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AsyncServerResponseHandler.java index 1404e6f..698e620 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AsyncServerResponseHandler.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AsyncServerResponseHandler.java @@ -40,14 +40,14 @@ import com.google.protobuf.Message; public class AsyncServerResponseHandler extends ChannelInboundHandlerAdapter { private static final Log LOG = LogFactory.getLog(AsyncServerResponseHandler.class.getName()); - private final AsyncRpcChannel channel; + private final AsyncRpcChannelImpl channel; /** * Constructor * * @param channel on which this response handler operates */ - public AsyncServerResponseHandler(AsyncRpcChannel channel) { + public AsyncServerResponseHandler(AsyncRpcChannelImpl channel) { this.channel = channel; } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/CoprocessorRpcChannel.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/CoprocessorRpcChannel.java index e60fbd6..8b4efd3 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/CoprocessorRpcChannel.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/CoprocessorRpcChannel.java @@ -35,47 +35,13 @@ import com.google.protobuf.RpcController; import com.google.protobuf.ServiceException; /** - * Base class which provides clients with an RPC connection to + * Base interface which provides clients with an RPC connection to * call coprocessor endpoint {@link com.google.protobuf.Service}s. * Note that clients should not use this class directly, except through - * {@link org.apache.hadoop.hbase.client.HTableInterface#coprocessorService(byte[])}. + * {@link org.apache.hadoop.hbase.client.Table#coprocessorService(byte[])}. */ @InterfaceAudience.Public @InterfaceStability.Evolving -public abstract class CoprocessorRpcChannel implements RpcChannel, BlockingRpcChannel { - private static final Log LOG = LogFactory.getLog(CoprocessorRpcChannel.class); +public interface CoprocessorRpcChannel extends RpcChannel, BlockingRpcChannel { - @Override - @InterfaceAudience.Private - public void callMethod(Descriptors.MethodDescriptor method, - RpcController controller, - Message request, Message responsePrototype, - RpcCallback callback) { - Message response = null; - try { - response = callExecService(method, request, responsePrototype); - } catch (IOException ioe) { - LOG.warn("Call failed on IOException", ioe); - ResponseConverter.setControllerException(controller, ioe); - } - if (callback != null) { - callback.run(response); - } - } - - @Override - @InterfaceAudience.Private - public Message callBlockingMethod(Descriptors.MethodDescriptor method, - RpcController controller, - Message request, Message responsePrototype) - throws ServiceException { - try { - return callExecService(method, request, responsePrototype); - } catch (IOException ioe) { - throw new ServiceException("Error calling method "+method.getFullName(), ioe); - } - } - - protected abstract Message callExecService(Descriptors.MethodDescriptor method, - Message request, Message responsePrototype) throws IOException; -} +} \ No newline at end of file diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/IOExceptionConverter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/IOExceptionConverter.java new file mode 100644 index 0000000..b7c4bb2 --- /dev/null +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/IOExceptionConverter.java @@ -0,0 +1,15 @@ +package org.apache.hadoop.hbase.ipc; + +import java.io.IOException; + +/** + * Converts exceptions to other exceptions + */ +public interface IOExceptionConverter { + /** + * Converts given IOException + * @param e exception to convert + * @return converted IOException + */ + IOException convert(IOException e); +} \ No newline at end of file diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/MasterCoprocessorRpcChannel.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/MasterCoprocessorRpcChannel.java index 622b5f4..3a0d944 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/MasterCoprocessorRpcChannel.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/MasterCoprocessorRpcChannel.java @@ -42,7 +42,7 @@ import com.google.protobuf.Message; * @see org.apache.hadoop.hbase.client.HBaseAdmin#coprocessorService() */ @InterfaceAudience.Private -public class MasterCoprocessorRpcChannel extends CoprocessorRpcChannel{ +public class MasterCoprocessorRpcChannel extends SyncCoprocessorRpcChannel{ private static final Log LOG = LogFactory.getLog(MasterCoprocessorRpcChannel.class); private final HConnection connection; diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/MessageConverter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/MessageConverter.java new file mode 100644 index 0000000..a8c7d61 --- /dev/null +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/MessageConverter.java @@ -0,0 +1,49 @@ +/** + * 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.ipc; + +import org.apache.hadoop.hbase.CellScanner; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.protobuf.generated.ClientProtos; +import sun.plugin2.message.Message; + +import java.io.IOException; + +/** + * Interface to convert Messages to specific types + * @param Message Type to convert + * @param Output Type + */ +@InterfaceAudience.Private +public interface MessageConverter { + /** + * Converts Message to Output + * @param msg to convert + * @param cellScanner to use for conversion + * @return Output + * @throws IOException if message could not be converted to response + */ + O convert(M msg, CellScanner cellScanner) throws IOException; + + MessageConverter NO_CONVERTER = new MessageConverter() { + @Override + public Message convert(Message msg, CellScanner cellScanner) throws IOException { + return null; + } + }; +} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/Promise.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/Promise.java new file mode 100644 index 0000000..30486a5 --- /dev/null +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/Promise.java @@ -0,0 +1,39 @@ +/** + * 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.ipc; + +import io.netty.channel.EventLoop; +import io.netty.util.concurrent.DefaultPromise; +import io.netty.util.concurrent.EventExecutor; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.client.Future; + +/** + * Abstract response promise + * @param Type of result contained in Promise + */ +@InterfaceAudience.Private +public class Promise extends DefaultPromise implements Future { + /** + * Constructor + * @param eventLoop to handle events on + */ + public Promise(EventExecutor eventLoop) { + super(eventLoop); + } +} \ No newline at end of file diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RegionCoprocessorRpcChannel.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RegionCoprocessorRpcChannel.java index 092e24d..09394be 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RegionCoprocessorRpcChannel.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RegionCoprocessorRpcChannel.java @@ -18,17 +18,19 @@ package org.apache.hadoop.hbase.ipc; -import java.io.IOException; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.CellScanner; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.classification.InterfaceAudience; -import org.apache.hadoop.hbase.client.HConnection; -import org.apache.hadoop.hbase.client.RegionServerCallable; +import org.apache.hadoop.hbase.client.AsyncRegionServerCallable; +import org.apache.hadoop.hbase.client.AsyncTableImpl; +import org.apache.hadoop.hbase.client.ClusterConnection; +import org.apache.hadoop.hbase.client.Future; import org.apache.hadoop.hbase.client.RpcRetryingCallerFactory; -import org.apache.hadoop.hbase.protobuf.ProtobufUtil; +import org.apache.hadoop.hbase.protobuf.RequestConverter; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResponse; import org.apache.hadoop.hbase.util.ByteStringer; @@ -37,6 +39,12 @@ import org.apache.hadoop.hbase.util.Bytes; import com.google.protobuf.Descriptors; import com.google.protobuf.Message; +import java.io.IOException; + +import static org.apache.hadoop.hbase.client.AsyncTableImpl.getPriority; +import static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier + .RegionSpecifierType.REGION_NAME; + /** * Provides clients with an RPC connection to call coprocessor endpoint {@link com.google.protobuf.Service}s * against a given table region. An instance of this class may be obtained @@ -46,18 +54,24 @@ import com.google.protobuf.Message; * @see org.apache.hadoop.hbase.client.HTable#coprocessorService(byte[]) */ @InterfaceAudience.Private -public class RegionCoprocessorRpcChannel extends CoprocessorRpcChannel{ +public class RegionCoprocessorRpcChannel extends AsyncCoprocessorRpcChannel{ private static final Log LOG = LogFactory.getLog(RegionCoprocessorRpcChannel.class); - private final HConnection connection; + private final ClusterConnection connection; private final TableName table; private final byte[] row; private byte[] lastRegion; - private int operationTimeout; + private final int operationTimeout; - private RpcRetryingCallerFactory rpcFactory; + private final RpcRetryingCallerFactory rpcFactory; - public RegionCoprocessorRpcChannel(HConnection conn, TableName table, byte[] row) { + /** + * Constructor + * @param conn connection to use + * @param table to connect to + * @param row to locate region with + */ + public RegionCoprocessorRpcChannel(ClusterConnection conn, TableName table, byte[] row) { this.connection = conn; this.table = table; this.row = row; @@ -68,9 +82,8 @@ public class RegionCoprocessorRpcChannel extends CoprocessorRpcChannel{ } @Override - protected Message callExecService(Descriptors.MethodDescriptor method, - Message request, Message responsePrototype) - throws IOException { + public Future callMethodWithPromise( + Descriptors.MethodDescriptor method, Message request, final R responsePrototype) { if (LOG.isTraceEnabled()) { LOG.trace("Call: "+method.getName()+", "+request.toString()); } @@ -79,36 +92,56 @@ public class RegionCoprocessorRpcChannel extends CoprocessorRpcChannel{ throw new IllegalArgumentException("Missing row property for remote region location"); } + final MessageConverter converter = + new MessageConverter(){ + @Override + @SuppressWarnings("unchecked") + public R convert(CoprocessorServiceResponse result, CellScanner cellScanner) throws IOException { + R response; + if (result.getValue().hasValue()) { + response = (R) responsePrototype.newBuilderForType() + .mergeFrom(result.getValue().getValue()).build(); + } else { + response = (R) responsePrototype.getDefaultInstanceForType(); + } + lastRegion = result.getRegion().getValue().toByteArray(); + + if (LOG.isTraceEnabled()) { + LOG.trace("Result is region=" + Bytes.toStringBinary(lastRegion) + ", value=" + response); + } + return response; + } + }; + final ClientProtos.CoprocessorServiceCall call = ClientProtos.CoprocessorServiceCall.newBuilder() .setRow(ByteStringer.wrap(row)) .setServiceName(method.getService().getFullName()) .setMethodName(method.getName()) .setRequest(request.toByteString()).build(); - RegionServerCallable callable = - new RegionServerCallable(connection, table, row) { - public CoprocessorServiceResponse call(int callTimeout) throws Exception { + AsyncRegionServerCallable callable = + new AsyncRegionServerCallable(connection, table, row) { + public Future call(int callTimeout) { byte[] regionName = getLocation().getRegionInfo().getRegionName(); - return ProtobufUtil.execService(getStub(), call, regionName); + + ClientProtos.CoprocessorServiceRequest + request = ClientProtos.CoprocessorServiceRequest.newBuilder() + .setCall(call).setRegion( + RequestConverter.buildRegionSpecifier(REGION_NAME, regionName)).build(); + + return getChannel().callMethod(AsyncTableImpl.EXEC_SERVICE_DESCRIPTOR, + request, null, ClientProtos.CoprocessorServiceResponse.getDefaultInstance(), + converter, null, callTimeout, getPriority(tableName)); } }; - CoprocessorServiceResponse result = rpcFactory. newCaller() - .callWithRetries(callable, operationTimeout); - Message response = null; - if (result.getValue().hasValue()) { - response = responsePrototype.newBuilderForType() - .mergeFrom(result.getValue().getValue()).build(); - } else { - response = responsePrototype.getDefaultInstanceForType(); - } - lastRegion = result.getRegion().getValue().toByteArray(); - if (LOG.isTraceEnabled()) { - LOG.trace("Result is region=" + Bytes.toStringBinary(lastRegion) + ", value=" + response); - } - return response; + return rpcFactory. newCaller().callAsyncWithRetries(callable, operationTimeout); } + /** + * Get last region this RpcChannel communicated with + * @return region name as byte array + */ public byte[] getLastRegion() { return lastRegion; } -} +} \ No newline at end of file diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RegionServerCoprocessorRpcChannel.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RegionServerCoprocessorRpcChannel.java index 4d9b9b2..07a92bf 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RegionServerCoprocessorRpcChannel.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RegionServerCoprocessorRpcChannel.java @@ -36,7 +36,7 @@ import com.google.protobuf.Message; * @see org.apache.hadoop.hbase.client.HBaseAdmin#coprocessorService(ServerName) */ @InterfaceAudience.Private -public class RegionServerCoprocessorRpcChannel extends CoprocessorRpcChannel { +public class RegionServerCoprocessorRpcChannel extends SyncCoprocessorRpcChannel { private static final Log LOG = LogFactory.getLog(RegionServerCoprocessorRpcChannel.class); private final ClusterConnection connection; private final ServerName serverName; diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClient.java index cf689f5..0572f84 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClient.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClient.java @@ -18,6 +18,9 @@ package org.apache.hadoop.hbase.ipc; import com.google.protobuf.BlockingRpcChannel; +import com.google.protobuf.RpcChannel; +import io.netty.channel.EventLoop; +import io.netty.util.concurrent.EventExecutor; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.security.User; @@ -29,29 +32,29 @@ import java.io.IOException; * Interface for RpcClient implementations so ConnectionManager can handle it. */ @InterfaceAudience.Private public interface RpcClient extends Closeable { - public final static String FAILED_SERVER_EXPIRY_KEY = "hbase.ipc.client.failed.servers.expiry"; - public final static int FAILED_SERVER_EXPIRY_DEFAULT = 2000; - public final static String IDLE_TIME = "hbase.ipc.client.connection.minIdleTimeBeforeClose"; - public static final String IPC_CLIENT_FALLBACK_TO_SIMPLE_AUTH_ALLOWED_KEY = + String FAILED_SERVER_EXPIRY_KEY = "hbase.ipc.client.failed.servers.expiry"; + int FAILED_SERVER_EXPIRY_DEFAULT = 2000; + String IDLE_TIME = "hbase.ipc.client.connection.minIdleTimeBeforeClose"; + String IPC_CLIENT_FALLBACK_TO_SIMPLE_AUTH_ALLOWED_KEY = "hbase.ipc.client.fallback-to-simple-auth-allowed"; - public static final boolean IPC_CLIENT_FALLBACK_TO_SIMPLE_AUTH_ALLOWED_DEFAULT = false; - public static final String SPECIFIC_WRITE_THREAD = "hbase.ipc.client.specificThreadForWriting"; - public static final String DEFAULT_CODEC_CLASS = "hbase.client.default.rpc.codec"; + boolean IPC_CLIENT_FALLBACK_TO_SIMPLE_AUTH_ALLOWED_DEFAULT = false; + String SPECIFIC_WRITE_THREAD = "hbase.ipc.client.specificThreadForWriting"; + String DEFAULT_CODEC_CLASS = "hbase.client.default.rpc.codec"; - public final static String SOCKET_TIMEOUT_CONNECT = "hbase.ipc.client.socket.timeout.connect"; + String SOCKET_TIMEOUT_CONNECT = "hbase.ipc.client.socket.timeout.connect"; /** * How long we wait when we wait for an answer. It's not the operation time, it's the time * we wait when we start to receive an answer, when the remote write starts to send the data. */ - public final static String SOCKET_TIMEOUT_READ = "hbase.ipc.client.socket.timeout.read"; - public final static String SOCKET_TIMEOUT_WRITE = "hbase.ipc.client.socket.timeout.write"; - public final static int DEFAULT_SOCKET_TIMEOUT_CONNECT = 10000; // 10 seconds - public final static int DEFAULT_SOCKET_TIMEOUT_READ = 20000; // 20 seconds - public final static int DEFAULT_SOCKET_TIMEOUT_WRITE = 60000; // 60 seconds + String SOCKET_TIMEOUT_READ = "hbase.ipc.client.socket.timeout.read"; + String SOCKET_TIMEOUT_WRITE = "hbase.ipc.client.socket.timeout.write"; + int DEFAULT_SOCKET_TIMEOUT_CONNECT = 10000; // 10 seconds + int DEFAULT_SOCKET_TIMEOUT_READ = 20000; // 20 seconds + int DEFAULT_SOCKET_TIMEOUT_WRITE = 60000; // 60 seconds // Used by the server, for compatibility with old clients. // The client in 0.99+ does not ping the server. - final static int PING_CALL_ID = -1; + int PING_CALL_ID = -1; /** * Creates a "channel" that can be used by a blocking protobuf service. Useful setting up @@ -64,10 +67,34 @@ import java.io.IOException; * @return A blocking rpc channel that goes via this rpc client instance. * @throws IOException when channel could not be created */ - public BlockingRpcChannel createBlockingRpcChannel(ServerName sn, User user, + BlockingRpcChannel createBlockingRpcChannel(ServerName sn, User user, int rpcTimeout) throws IOException; /** + * Create or fetch AsyncRpcChannel + * @param serviceName to connect to + * @param sn ServerName of the channel to create + * @param user for the service + * @return An async RPC channel fitting given parameters + * @throws FailedServerException if server failed + * @throws StoppedRpcClientException if the RPC client has stopped + */ + AsyncRpcChannel createRpcChannel(String serviceName, ServerName sn, User user) + throws StoppedRpcClientException, FailedServerException; + + /** + * Creates a "channel" that can be used by a protobuf service. Useful setting up + * protobuf stubs. + * + * @param sn server name describing location of server + * @param user which is to use the connection + * @param rpcTimeout default rpc operation timeout + * + * @return A rpc channel that goes via this rpc client instance. + */ + RpcChannel createRpcChannel(final ServerName sn, final User user, int rpcTimeout); + + /** * Interrupt the connections to the given server. This should be called if the server * is known as actually dead. This will not prevent current operation to be retried, and, * depending on their own behavior, they may retry on the same server. This can be a feature, @@ -76,11 +103,17 @@ import java.io.IOException; * safe exception. * @param sn server location to cancel connections of */ - public void cancelConnections(ServerName sn); + void cancelConnections(ServerName sn); /** * Stop all threads related to this client. No further calls may be made * using this client. */ - @Override public void close(); + @Override void close(); + + /** + * Get an event loop to operate on + * @return EventLoop + */ + EventExecutor getEventExecutor(); } \ No newline at end of file diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClientImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClientImpl.java index 9a5fc14..0cb10a2 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClientImpl.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClientImpl.java @@ -1,5 +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 @@ -16,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.ipc; import com.google.protobuf.Descriptors.MethodDescriptor; @@ -24,6 +22,9 @@ import com.google.protobuf.Message; import com.google.protobuf.Message.Builder; import com.google.protobuf.RpcCallback; +import com.google.protobuf.RpcChannel; +import com.google.protobuf.RpcController; +import io.netty.util.concurrent.EventExecutor; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; @@ -32,6 +33,7 @@ import org.apache.hadoop.hbase.DoNotRetryIOException; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.client.Future; import org.apache.hadoop.hbase.codec.Codec; import org.apache.hadoop.hbase.exceptions.ConnectionClosingException; import org.apache.hadoop.hbase.protobuf.ProtobufUtil; @@ -1129,6 +1131,11 @@ public class RpcClientImpl extends AbstractRpcClient { } } + @Override + public EventExecutor getEventExecutor() { + return AsyncRpcClient.getGlobalEventLoopGroup(this.conf).getFirst().next(); + } + /** Make a call, passing param, to the IPC server running at * address which is servicing the protocol protocol, * with the ticket credentials, returning the value. @@ -1148,27 +1155,51 @@ public class RpcClientImpl extends AbstractRpcClient { if (pcrc == null) { pcrc = new PayloadCarryingRpcController(); } + + Call call = this.call(md, param, returnType, pcrc, ticket, addr); + + return new Pair<>(call.response, call.cells); + } + + + /** Make a call, passing param, to the IPC server running at + * address which is servicing the protocol protocol, + * with the ticket credentials, returning the value. + * Throws exceptions if there are network problems or if the remote code + * threw an exception. + * @param ticket Be careful which ticket you pass. A new user will mean a new Connection. + * {@link UserProvider#getCurrent()} makes a new instance of User each time so will be a + * new Connection each time. + * @return A Call + * @throws InterruptedException + * @throws IOException + */ + private Call call(MethodDescriptor method, Message request, + R responsePrototype, PayloadCarryingRpcController pcrc, User ticket, + InetSocketAddress addr) + throws IOException, InterruptedException { + CellScanner cells = pcrc.cellScanner(); - final Call call = new Call(this.callIdCnt.getAndIncrement(), md, param, cells, returnType, - pcrc.getCallTimeout()); + final Call call = new Call(callIdCnt.getAndIncrement(), method, request, cells, + responsePrototype, pcrc.getCallTimeout()); final Connection connection = getConnection(ticket, call, addr); final CallFuture cts; if (connection.callSender != null) { cts = connection.callSender.sendCall(call, pcrc.getPriority(), Trace.currentSpan()); - pcrc.notifyOnCancel(new RpcCallback() { - @Override - public void run(Object parameter) { - connection.callSender.remove(cts); - } - }); - if (pcrc.isCanceled()) { - // To finish if the call was cancelled before we set the notification (race condition) - call.callComplete(); - return new Pair(call.response, call.cells); + pcrc.notifyOnCancel(new RpcCallback() { + @Override + public void run(Object parameter) { + connection.callSender.remove(cts); } + }); + if (pcrc.isCanceled()) { + // To finish if the call was cancelled before we set the notification (race condition) + call.callComplete(); + throw new InterruptedException(); + } } else { cts = null; connection.tracedWriteRequest(call, pcrc.getPriority(), Trace.currentSpan()); @@ -1176,7 +1207,8 @@ public class RpcClientImpl extends AbstractRpcClient { while (!call.done) { if (call.checkAndSetTimeout()) { - if (cts != null) connection.callSender.remove(cts); + if (cts != null) + connection.callSender.remove(cts); break; } if (connection.shouldCloseConnection.get()) { @@ -1185,12 +1217,14 @@ public class RpcClientImpl extends AbstractRpcClient { } try { synchronized (call) { - if (call.done) break; + if (call.done) + break; call.wait(Math.min(call.remainingTime(), 1000) + 1); } } catch (InterruptedException e) { call.setException(new InterruptedIOException()); - if (cts != null) connection.callSender.remove(cts); + if (cts != null) + connection.callSender.remove(cts); throw e; } } @@ -1204,7 +1238,7 @@ public class RpcClientImpl extends AbstractRpcClient { throw wrapException(addr, call.error); } - return new Pair(call.response, call.cells); + return call; } @@ -1238,6 +1272,17 @@ public class RpcClientImpl extends AbstractRpcClient { } } + @Override + public org.apache.hadoop.hbase.ipc.AsyncRpcChannel createRpcChannel(String serviceName, + ServerName sn, User user) throws StoppedRpcClientException, FailedServerException { + return new AsyncRpcChannel(sn, user); + } + + @Override + public RpcChannel createRpcChannel(ServerName sn, User user, int rpcTimeout) { + return new RpcChannelImplementation(sn, user, rpcTimeout); + } + /** * Interrupt the connections to the given ip:port server. This should be called if the server * is known as actually dead. This will not prevent current operation to be retried, and, @@ -1282,4 +1327,124 @@ public class RpcClientImpl extends AbstractRpcClient { return connection; } -} + + /** + * Simulated async call + */ + private class RpcChannelImplementation implements RpcChannel { + private final InetSocketAddress isa; + private final User ticket; + private final int channelOperationTimeout; + private final EventExecutor executor; + + /** + * @param channelOperationTimeout - the default timeout when no timeout is given + */ + protected RpcChannelImplementation( + final ServerName sn, final User ticket, int channelOperationTimeout) { + this.isa = new InetSocketAddress(sn.getHostname(), sn.getPort()); + this.ticket = ticket; + this.channelOperationTimeout = channelOperationTimeout; + + this.executor = RpcClientImpl.this.getEventExecutor(); + } + + @Override + public void callMethod(final MethodDescriptor method, RpcController controller, + final Message request, final Message responsePrototype, final RpcCallback done) { + final PayloadCarryingRpcController pcrc = configurePayloadCarryingRpcController( + controller, + channelOperationTimeout); + + executor.execute(new Runnable() { + @Override + public void run() { + try { + Call call = call(method, request, responsePrototype, pcrc, ticket, isa); + done.run(call.response); + } catch (IOException e) { + pcrc.setFailed(e); + } catch (InterruptedException e) { + pcrc.startCancel(); + } + } + }); + } + } + + /** + * Wraps the call in an async channel. + */ + private class AsyncRpcChannel implements org.apache.hadoop.hbase.ipc.AsyncRpcChannel { + private final EventExecutor executor; + private final InetSocketAddress isa; + + private final User ticket; + + /** + * Constructor + * @param sn servername to connect to + * @param user to connect with + */ + public AsyncRpcChannel(ServerName sn, User user) { + this.isa = new InetSocketAddress(sn.getHostname(), sn.getPort()); + this.executor = RpcClientImpl.this.getEventExecutor(); + this.ticket = user; + } + + @Override + @SuppressWarnings("unchecked") + public Future callMethod(final MethodDescriptor method, + final Message request, CellScanner cellScanner, final R responsePrototype, + final MessageConverter messageConverter, + final IOExceptionConverter exceptionConverter, long rpcTimeout, int priority) { + final PayloadCarryingRpcController pcrc = new PayloadCarryingRpcController(cellScanner); + pcrc.setPriority(priority); + pcrc.setCallTimeout((int) rpcTimeout); + + final Promise promise = new Promise<>(executor); + + executor.execute(new Runnable() { + @Override + public void run() { + try { + Call call = call(method, request, responsePrototype, pcrc, ticket, isa); + + promise.setSuccess( + messageConverter.convert((R) call.response, call.cells) + ); + } catch (InterruptedException e) { + promise.cancel(true); + } catch (IOException e) { + if(exceptionConverter != null) { + e = exceptionConverter.convert(e); + } + promise.setFailure(e); + } + } + }); + + return promise; + } + + @Override + public EventExecutor getEventExecutor() { + return this.executor; + } + + @Override + public void close(Throwable cause) { + this.executor.shutdownGracefully(); + } + + @Override + public boolean isAlive() { + return !this.executor.isShuttingDown() && !this.executor.isShutdown(); + } + + @Override + public InetSocketAddress getAddress() { + return isa; + } + } +} \ No newline at end of file diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/SyncCoprocessorRpcChannel.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/SyncCoprocessorRpcChannel.java new file mode 100644 index 0000000..93a7611 --- /dev/null +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/SyncCoprocessorRpcChannel.java @@ -0,0 +1,85 @@ +/* + * 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.ipc; + +import com.google.protobuf.Descriptors; +import com.google.protobuf.Message; +import com.google.protobuf.RpcCallback; +import com.google.protobuf.RpcController; +import com.google.protobuf.ServiceException; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.classification.InterfaceStability; +import org.apache.hadoop.hbase.protobuf.ResponseConverter; + +import java.io.IOException; + +/** + * Base class which provides clients with an RPC connection to + * call coprocessor endpoint {@link com.google.protobuf.Service}s. + * Note that clients should not use this class directly, except through + * {@link org.apache.hadoop.hbase.client.Table#coprocessorService(byte[])}. + */ +@InterfaceAudience.Public +@InterfaceStability.Evolving +public abstract class SyncCoprocessorRpcChannel implements CoprocessorRpcChannel { + private static final Log LOG = LogFactory.getLog(SyncCoprocessorRpcChannel.class); + + @Override + @InterfaceAudience.Private + public void callMethod(Descriptors.MethodDescriptor method, + RpcController controller, + Message request, Message responsePrototype, + RpcCallback callback) { + Message response = null; + try { + response = callExecService(method, request, responsePrototype); + } catch (IOException ioe) { + LOG.warn("Call failed on IOException", ioe); + ResponseConverter.setControllerException(controller, ioe); + } + if (callback != null) { + callback.run(response); + } + } + + @Override + @InterfaceAudience.Private + public Message callBlockingMethod(Descriptors.MethodDescriptor method, + RpcController controller, + Message request, Message responsePrototype) + throws ServiceException { + try { + return callExecService(method, request, responsePrototype); + } catch (IOException ioe) { + throw new ServiceException("Error calling method "+method.getFullName(), ioe); + } + } + + /** + * Calls the Exec service in a sync way + * @param method to call + * @param request to send + * @param responsePrototype for response + * @return response as Message + * @throws IOException if connection fails + */ + protected abstract Message callExecService(Descriptors.MethodDescriptor method, + Message request, Message responsePrototype) throws IOException; +} 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 cda6bd7..b1d5b37 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 @@ -1601,20 +1601,6 @@ public final class ProtobufUtil { } } - public static CoprocessorServiceResponse execService(final ClientService.BlockingInterface client, - final CoprocessorServiceCall call, final byte[] regionName) throws IOException { - CoprocessorServiceRequest request = CoprocessorServiceRequest.newBuilder() - .setCall(call).setRegion( - RequestConverter.buildRegionSpecifier(REGION_NAME, regionName)).build(); - try { - CoprocessorServiceResponse response = - client.execService(null, request); - return response; - } catch (ServiceException se) { - throw getRemoteException(se); - } - } - public static CoprocessorServiceResponse execService( final MasterService.BlockingInterface client, final CoprocessorServiceCall call) throws IOException { diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientScanner.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientScanner.java index 3f406df..69cc8b1 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientScanner.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientScanner.java @@ -23,7 +23,6 @@ import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; import java.io.IOException; -import java.util.LinkedList; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; @@ -84,9 +83,9 @@ public class TestClientScanner { public MockClientScanner(final Configuration conf, final Scan scan, final TableName tableName, ClusterConnection connection, RpcRetryingCallerFactory rpcFactory, - RpcControllerFactory controllerFactory, ExecutorService pool, int primaryOperationTimeout) + ExecutorService pool, int primaryOperationTimeout) throws IOException { - super(conf, scan, tableName, connection, rpcFactory, controllerFactory, pool, + super(conf, scan, tableName, connection, rpcFactory, pool, primaryOperationTimeout); } @@ -110,12 +109,11 @@ public class TestClientScanner { int nbRows) { scan.setStartRow(localStartKey); ScannerCallable s = - new ScannerCallable(getConnection(), getTable(), scan, this.scanMetrics, - this.rpcControllerFactory); + new ScannerCallable(getConnection(), getTable(), scan, this.scanMetrics); s.setCaching(nbRows); ScannerCallableWithReplicas sr = new ScannerCallableWithReplicas(getTable(), getConnection(), - s, pool, primaryOperationTimeout, scan, - getRetries(), scannerTimeout, caching, conf, caller); + s, primaryOperationTimeout, scan, + scannerTimeout, caller); return sr; } @@ -144,22 +142,22 @@ public class TestClientScanner { RpcRetryingCaller caller = Mockito.mock(RpcRetryingCaller.class); Mockito.when(rpcFactory. newCaller()).thenReturn(caller); - Mockito.when(caller.callWithoutRetries(Mockito.any(RetryingCallable.class), - Mockito.anyInt())).thenAnswer(new Answer() { + Mockito.when(caller.callAsyncWithRetries(Mockito.any(AsyncRetryingCallable.class), + Mockito.anyInt())).thenAnswer(new Answer>() { private int count = 0; @Override - public Result[] answer(InvocationOnMock invocation) throws Throwable { + public Future answer(InvocationOnMock invocation) throws Throwable { ScannerCallableWithReplicas callable = invocation.getArgumentAt(0, ScannerCallableWithReplicas.class); switch (count) { case 0: // initialize case 2: // close count++; - return null; + return new SuccessfulFuture(clusterConn.getEventExecutor(), null); case 1: count++; callable.setHasMoreResultsContext(false); - return results; + return new SuccessfulFuture(clusterConn.getEventExecutor(), results); default: throw new RuntimeException("Expected only 2 invocations"); } @@ -171,7 +169,7 @@ public class TestClientScanner { scan.setMaxResultSize(1000*1000); try (MockClientScanner scanner = new MockClientScanner(conf, scan, TableName.valueOf("table"), - clusterConn, rpcFactory, controllerFactory, pool, Integer.MAX_VALUE)) { + clusterConn, rpcFactory, pool, Integer.MAX_VALUE)) { scanner.setRpcFinished(true); @@ -180,8 +178,8 @@ public class TestClientScanner { scanner.loadCache(); // One more call due to initializeScannerInConstruction() - inOrder.verify(caller, Mockito.times(2)).callWithoutRetries( - Mockito.any(RetryingCallable.class), Mockito.anyInt()); + inOrder.verify(caller, Mockito.times(2)).callAsyncWithRetries( + Mockito.any(AsyncRetryingCallable.class), Mockito.anyInt()); assertEquals(1, scanner.cache.size()); Result r = scanner.cache.poll(); @@ -204,23 +202,23 @@ public class TestClientScanner { RpcRetryingCaller caller = Mockito.mock(RpcRetryingCaller.class); Mockito.when(rpcFactory. newCaller()).thenReturn(caller); - Mockito.when(caller.callWithoutRetries(Mockito.any(RetryingCallable.class), - Mockito.anyInt())).thenAnswer(new Answer() { + Mockito.when(caller.callAsyncWithRetries(Mockito.any(AsyncRetryingCallable.class), + Mockito.anyInt())).thenAnswer(new Answer>() { private int count = 0; @Override - public Result[] answer(InvocationOnMock invocation) throws Throwable { + public Future answer(InvocationOnMock invocation) throws Throwable { ScannerCallableWithReplicas callable = invocation.getArgumentAt(0, ScannerCallableWithReplicas.class); switch (count) { case 0: // initialize case 2: // close count++; - return null; + return new SuccessfulFuture(clusterConn.getEventExecutor(), null); case 1: count++; callable.setHasMoreResultsContext(true); callable.setServerHasMoreResults(false); - return results; + return new SuccessfulFuture(clusterConn.getEventExecutor(), results); default: throw new RuntimeException("Expected only 2 invocations"); } @@ -235,18 +233,18 @@ public class TestClientScanner { scan.setMaxResultSize(1); try (MockClientScanner scanner = new MockClientScanner(conf, scan, TableName.valueOf("table"), - clusterConn, rpcFactory, controllerFactory, pool, Integer.MAX_VALUE)) { + clusterConn, rpcFactory, pool, Integer.MAX_VALUE)) { // Due to initializeScannerInConstruction() - Mockito.verify(caller).callWithoutRetries(Mockito.any(RetryingCallable.class), + Mockito.verify(caller).callAsyncWithRetries(Mockito.any(AsyncRetryingCallable.class), Mockito.anyInt()); InOrder inOrder = Mockito.inOrder(caller); scanner.loadCache(); - inOrder.verify(caller, Mockito.times(2)).callWithoutRetries( - Mockito.any(RetryingCallable.class), Mockito.anyInt()); + inOrder.verify(caller, Mockito.times(2)).callAsyncWithRetries( + Mockito.any(AsyncRetryingCallable.class), Mockito.anyInt()); assertEquals(1, scanner.cache.size()); Result r = scanner.cache.poll(); @@ -271,23 +269,23 @@ public class TestClientScanner { RpcRetryingCaller caller = Mockito.mock(RpcRetryingCaller.class); Mockito.when(rpcFactory. newCaller()).thenReturn(caller); - Mockito.when(caller.callWithoutRetries(Mockito.any(RetryingCallable.class), - Mockito.anyInt())).thenAnswer(new Answer() { + Mockito.when(caller.callAsyncWithRetries(Mockito.any(AsyncRetryingCallable.class), + Mockito.anyInt())).thenAnswer(new Answer>() { private int count = 0; @Override - public Result[] answer(InvocationOnMock invocation) throws Throwable { + public Future answer(InvocationOnMock invocation) throws Throwable { ScannerCallableWithReplicas callable = invocation.getArgumentAt(0, ScannerCallableWithReplicas.class); switch (count) { case 0: // initialize case 2: // close count++; - return null; + return new SuccessfulFuture(clusterConn.getEventExecutor(), null); case 1: count++; callable.setHasMoreResultsContext(true); callable.setServerHasMoreResults(false); - return results; + return new SuccessfulFuture(clusterConn.getEventExecutor(), results); default: throw new RuntimeException("Expected only 2 invocations"); } @@ -302,10 +300,10 @@ public class TestClientScanner { scan.setMaxResultSize(1000*1000); try (MockClientScanner scanner = new MockClientScanner(conf, scan, TableName.valueOf("table"), - clusterConn, rpcFactory, controllerFactory, pool, Integer.MAX_VALUE)) { + clusterConn, rpcFactory, pool, Integer.MAX_VALUE)) { // Due to initializeScannerInConstruction() - Mockito.verify(caller).callWithoutRetries(Mockito.any(RetryingCallable.class), + Mockito.verify(caller).callAsyncWithRetries(Mockito.any(AsyncRetryingCallable.class), Mockito.anyInt()); InOrder inOrder = Mockito.inOrder(caller); @@ -314,8 +312,8 @@ public class TestClientScanner { // Ensures that possiblyNextScanner isn't called at the end which would trigger // another call to callWithoutRetries - inOrder.verify(caller, Mockito.times(2)).callWithoutRetries( - Mockito.any(RetryingCallable.class), Mockito.anyInt()); + inOrder.verify(caller, Mockito.times(2)).callAsyncWithRetries( + Mockito.any(AsyncRetryingCallable.class), Mockito.anyInt()); assertEquals(3, scanner.cache.size()); Result r = scanner.cache.poll(); @@ -352,23 +350,23 @@ public class TestClientScanner { RpcRetryingCaller caller = Mockito.mock(RpcRetryingCaller.class); Mockito.when(rpcFactory. newCaller()).thenReturn(caller); - Mockito.when(caller.callWithoutRetries(Mockito.any(RetryingCallable.class), - Mockito.anyInt())).thenAnswer(new Answer() { + Mockito.when(caller.callAsyncWithRetries(Mockito.any(AsyncRegionServerCallable.class), + Mockito.anyInt())).thenAnswer(new Answer>() { private int count = 0; @Override - public Result[] answer(InvocationOnMock invocation) throws Throwable { + public Future answer(InvocationOnMock invocation) throws Throwable { ScannerCallableWithReplicas callable = invocation.getArgumentAt(0, ScannerCallableWithReplicas.class); switch (count) { case 0: // initialize case 2: // close count++; - return null; + return new SuccessfulFuture(clusterConn.getEventExecutor(), null); case 1: count++; callable.setHasMoreResultsContext(true); callable.setServerHasMoreResults(false); - return results; + return new SuccessfulFuture(clusterConn.getEventExecutor(), results); default: throw new RuntimeException("Expected only 2 invocations"); } @@ -382,10 +380,10 @@ public class TestClientScanner { scan.setMaxResultSize(1000*1000); try (MockClientScanner scanner = new MockClientScanner(conf, scan, TableName.valueOf("table"), - clusterConn, rpcFactory, controllerFactory, pool, Integer.MAX_VALUE)) { + clusterConn, rpcFactory, pool, Integer.MAX_VALUE)) { // Due to initializeScannerInConstruction() - Mockito.verify(caller).callWithoutRetries(Mockito.any(RetryingCallable.class), + Mockito.verify(caller).callAsyncWithRetries(Mockito.any(AsyncRetryingCallable.class), Mockito.anyInt()); scanner.setRpcFinished(true); @@ -394,8 +392,8 @@ public class TestClientScanner { scanner.loadCache(); - inOrder.verify(caller, Mockito.times(2)).callWithoutRetries( - Mockito.any(RetryingCallable.class), Mockito.anyInt()); + inOrder.verify(caller, Mockito.times(2)).callAsyncWithRetries( + Mockito.any(AsyncRetryingCallable.class), Mockito.anyInt()); assertEquals(1, scanner.cache.size()); Result r = scanner.cache.poll(); @@ -424,29 +422,29 @@ public class TestClientScanner { RpcRetryingCaller caller = Mockito.mock(RpcRetryingCaller.class); Mockito.when(rpcFactory. newCaller()).thenReturn(caller); - Mockito.when(caller.callWithoutRetries(Mockito.any(RetryingCallable.class), - Mockito.anyInt())).thenAnswer(new Answer() { + Mockito.when(caller.callAsyncWithRetries(Mockito.any(AsyncRetryingCallable.class), + Mockito.anyInt())).thenAnswer(new Answer>() { private int count = 0; @Override - public Result[] answer(InvocationOnMock invocation) throws Throwable { + public Future answer(InvocationOnMock invocation) throws Throwable { ScannerCallableWithReplicas callable = invocation.getArgumentAt(0, ScannerCallableWithReplicas.class); switch (count) { case 0: // initialize case 3: // close count++; - return null; + return new SuccessfulFuture(clusterConn.getEventExecutor(), null); case 1: count++; callable.setHasMoreResultsContext(true); callable.setServerHasMoreResults(true); - return results1; + return new SuccessfulFuture(clusterConn.getEventExecutor(), results1); case 2: count++; // The server reports back false WRT more results callable.setHasMoreResultsContext(true); callable.setServerHasMoreResults(false); - return results2; + return new SuccessfulFuture(clusterConn.getEventExecutor(), results2); default: throw new RuntimeException("Expected only 2 invocations"); } @@ -458,18 +456,18 @@ public class TestClientScanner { scan.setMaxResultSize(1000*1000); try (MockClientScanner scanner = new MockClientScanner(conf, scan, TableName.valueOf("table"), - clusterConn, rpcFactory, controllerFactory, pool, Integer.MAX_VALUE)) { + clusterConn, rpcFactory, pool, Integer.MAX_VALUE)) { // Due to initializeScannerInConstruction() - Mockito.verify(caller).callWithoutRetries(Mockito.any(RetryingCallable.class), + Mockito.verify(caller).callAsyncWithRetries(Mockito.any(AsyncRetryingCallable.class), Mockito.anyInt()); InOrder inOrder = Mockito.inOrder(caller); scanner.loadCache(); - inOrder.verify(caller, Mockito.times(2)).callWithoutRetries( - Mockito.any(RetryingCallable.class), Mockito.anyInt()); + inOrder.verify(caller, Mockito.times(2)).callAsyncWithRetries( + Mockito.any(AsyncRetryingCallable.class), Mockito.anyInt()); assertEquals(1, scanner.cache.size()); Result r = scanner.cache.poll(); @@ -485,8 +483,8 @@ public class TestClientScanner { scanner.loadCache(); - inOrder.verify(caller, Mockito.times(3)).callWithoutRetries( - Mockito.any(RetryingCallable.class), Mockito.anyInt()); + inOrder.verify(caller, Mockito.times(3)).callAsyncWithRetries( + Mockito.any(AsyncRetryingCallable.class), Mockito.anyInt()); r = scanner.cache.poll(); assertNotNull(r); diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientSmallReversedScanner.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientSmallReversedScanner.java index 082090e..82df336 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientSmallReversedScanner.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientSmallReversedScanner.java @@ -104,8 +104,7 @@ public class TestClientSmallReversedScanner { @Override public ScannerCallableWithReplicas getCallable(ClusterConnection connection, TableName table, Scan scan, ScanMetrics scanMetrics, byte[] localStartKey, int cacheNum, - RpcControllerFactory controllerFactory, ExecutorService pool, - int primaryOperationTimeout, int retries, int scannerTimeout, Configuration conf, + int primaryOperationTimeout, int scannerTimeout, RpcRetryingCaller caller) { return callableWithReplicas; } @@ -134,18 +133,18 @@ public class TestClientSmallReversedScanner { SmallScannerCallableFactory factory = getFactory(callableWithReplicas); try (ClientSmallReversedScanner csrs = new ClientSmallReversedScanner(conf, scan, - TableName.valueOf("table"), clusterConn, rpcFactory, controllerFactory, pool, + TableName.valueOf("table"), clusterConn, rpcFactory, pool, Integer.MAX_VALUE)) { csrs.setScannerCallableFactory(factory); // Return some data the first time, less the second, and none after that - Mockito.when(caller.callWithoutRetries(callableWithReplicas, csrs.getScannerTimeout())) - .thenAnswer(new Answer() { + Mockito.when(caller.callAsyncWithoutRetries(callableWithReplicas, csrs.getScannerTimeout())) + .thenAnswer(new Answer>() { int count = 0; @Override - public Result[] answer(InvocationOnMock invocation) { + public Future answer(InvocationOnMock invocation) { Result[] results; if (0 == count) { results = new Result[] {Result.create(new Cell[] {kv3}), @@ -156,7 +155,7 @@ public class TestClientSmallReversedScanner { results = new Result[0]; } count++; - return results; + return new SuccessfulFuture<>(clusterConn.getEventExecutor(), results); } }); @@ -203,18 +202,17 @@ public class TestClientSmallReversedScanner { SmallScannerCallableFactory factory = getFactory(callableWithReplicas); try (ClientSmallReversedScanner csrs = new ClientSmallReversedScanner(conf, scan, - TableName.valueOf("table"), clusterConn, rpcFactory, controllerFactory, pool, - Integer.MAX_VALUE)) { + TableName.valueOf("table"), clusterConn, rpcFactory, pool, Integer.MAX_VALUE)) { csrs.setScannerCallableFactory(factory); // Return some data the first time, less the second, and none after that - Mockito.when(caller.callWithoutRetries(callableWithReplicas, csrs.getScannerTimeout())) - .thenAnswer(new Answer() { + Mockito.when(caller.callAsyncWithoutRetries(callableWithReplicas, csrs.getScannerTimeout())) + .thenAnswer(new Answer>() { int count = 0; @Override - public Result[] answer(InvocationOnMock invocation) { + public Future answer(InvocationOnMock invocation) { Result[] results; if (0 == count) { results = new Result[] {Result.create(new Cell[] {kv3}), @@ -226,7 +224,7 @@ public class TestClientSmallReversedScanner { throw new RuntimeException("Should not fetch a third batch from the server"); } count++; - return results; + return new SuccessfulFuture<>(clusterConn.getEventExecutor(), results); } }); @@ -279,14 +277,13 @@ public class TestClientSmallReversedScanner { SmallScannerCallableFactory factory = getFactory(callableWithReplicas); try (ClientSmallReversedScanner csrs = new ClientSmallReversedScanner(conf, scan, - TableName.valueOf("table"), clusterConn, rpcFactory, controllerFactory, pool, - Integer.MAX_VALUE)) { + TableName.valueOf("table"), clusterConn, rpcFactory, pool, Integer.MAX_VALUE)) { csrs.setScannerCallableFactory(factory); // Return some data the first time, less the second, and none after that - Mockito.when(caller.callWithoutRetries(callableWithReplicas, csrs.getScannerTimeout())) - .thenReturn(new Result[0]); + Mockito.when(caller.callAsyncWithoutRetries(callableWithReplicas, csrs.getScannerTimeout())) + .thenReturn(new SuccessfulFuture<>(null, new Result[0])); // Server doesn't return the context Mockito.when(callableWithReplicas.hasMoreResultsContext()).thenReturn(false); @@ -315,14 +312,13 @@ public class TestClientSmallReversedScanner { SmallScannerCallableFactory factory = getFactory(callableWithReplicas); try (ClientSmallReversedScanner csrs = new ClientSmallReversedScanner(conf, scan, - TableName.valueOf("table"), clusterConn, rpcFactory, controllerFactory, pool, - Integer.MAX_VALUE)) { + TableName.valueOf("table"), clusterConn, rpcFactory, pool, Integer.MAX_VALUE)) { csrs.setScannerCallableFactory(factory); // Return some data the first time, less the second, and none after that - Mockito.when(caller.callWithoutRetries(callableWithReplicas, csrs.getScannerTimeout())) - .thenReturn(new Result[0]); + Mockito.when(caller.callAsyncWithoutRetries(callableWithReplicas, csrs.getScannerTimeout())) + .thenReturn(new SuccessfulFuture<>(null, new Result[0])); // Server doesn't return the context Mockito.when(callableWithReplicas.hasMoreResultsContext()).thenReturn(true); diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientSmallScanner.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientSmallScanner.java index 318fbe7..604bf66 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientSmallScanner.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientSmallScanner.java @@ -103,8 +103,7 @@ public class TestClientSmallScanner { @Override public ScannerCallableWithReplicas getCallable(ClusterConnection connection, TableName table, Scan scan, ScanMetrics scanMetrics, byte[] localStartKey, int cacheNum, - RpcControllerFactory controllerFactory, ExecutorService pool, - int primaryOperationTimeout, int retries, int scannerTimeout, Configuration conf, + int primaryOperationTimeout, int scannerTimeout, RpcRetryingCaller caller) { return callableWithReplicas; } @@ -138,12 +137,12 @@ public class TestClientSmallScanner { css.setScannerCallableFactory(factory); // Return some data the first time, less the second, and none after that - Mockito.when(caller.callWithoutRetries(callableWithReplicas, css.getScannerTimeout())) - .thenAnswer(new Answer() { + Mockito.when(caller.callAsyncWithoutRetries(callableWithReplicas, css.getScannerTimeout())) + .thenAnswer(new Answer>() { int count = 0; @Override - public Result[] answer(InvocationOnMock invocation) { + public Future answer(InvocationOnMock invocation) { Result[] results; if (0 == count) { results = new Result[] {Result.create(new Cell[] {kv1}), @@ -154,7 +153,7 @@ public class TestClientSmallScanner { results = new Result[0]; } count++; - return results; + return new SuccessfulFuture<>(clusterConn.getEventExecutor(), results); } }); @@ -204,12 +203,12 @@ public class TestClientSmallScanner { css.setScannerCallableFactory(factory); // Return some data the first time, less the second, and none after that - Mockito.when(caller.callWithoutRetries(callableWithReplicas, css.getScannerTimeout())) - .thenAnswer(new Answer() { + Mockito.when(caller.callAsyncWithoutRetries(callableWithReplicas, css.getScannerTimeout())) + .thenAnswer(new Answer>() { int count = 0; @Override - public Result[] answer(InvocationOnMock invocation) { + public Future answer(InvocationOnMock invocation) { Result[] results; if (0 == count) { results = new Result[] {Result.create(new Cell[] {kv1}), @@ -221,7 +220,7 @@ public class TestClientSmallScanner { throw new RuntimeException("Should not fetch a third batch from the server"); } count++; - return results; + return new SuccessfulFuture<>(clusterConn.getEventExecutor(), results); } }); @@ -277,8 +276,8 @@ public class TestClientSmallScanner { css.setScannerCallableFactory(factory); // Return some data the first time, less the second, and none after that - Mockito.when(caller.callWithoutRetries(callableWithReplicas, css.getScannerTimeout())) - .thenReturn(new Result[0]); + Mockito.when(caller.callAsyncWithoutRetries(callableWithReplicas, css.getScannerTimeout())) + .thenReturn(new SuccessfulFuture<>(null, new Result[0])); // Server doesn't return the context Mockito.when(callableWithReplicas.hasMoreResultsContext()).thenReturn(false); @@ -312,8 +311,8 @@ public class TestClientSmallScanner { css.setScannerCallableFactory(factory); // Return some data the first time, less the second, and none after that - Mockito.when(caller.callWithoutRetries(callableWithReplicas, css.getScannerTimeout())) - .thenReturn(new Result[0]); + Mockito.when(caller.callAsyncWithoutRetries(callableWithReplicas, css.getScannerTimeout())) + .thenReturn(new SuccessfulFuture<>(null, new Result[0])); // Server doesn't return the context Mockito.when(callableWithReplicas.hasMoreResultsContext()).thenReturn(true); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java index 3dc1387..6dc3124 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java @@ -28,7 +28,6 @@ import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; import java.io.IOException; -import java.lang.reflect.Method; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -48,7 +47,6 @@ import org.apache.commons.lang.ArrayUtils; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.Abortable; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.DoNotRetryIOException; @@ -91,7 +89,6 @@ import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.MutationType; import org.apache.hadoop.hbase.protobuf.generated.MultiRowMutationProtos.MultiRowMutationService; import org.apache.hadoop.hbase.protobuf.generated.MultiRowMutationProtos.MutateRowsRequest; -import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegionServer; import org.apache.hadoop.hbase.regionserver.NoSuchColumnFamilyException; import org.apache.hadoop.hbase.regionserver.Region; @@ -101,7 +98,6 @@ import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.Pair; -import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.apache.log4j.AppenderSkeleton; import org.apache.log4j.Logger; import org.apache.log4j.spi.LoggingEvent; @@ -4569,7 +4565,7 @@ public class TestFromClientSide { assertIncrementKey(kvs[0], ROW, FAMILY, QUALIFIERS[0], 2); assertIncrementKey(kvs[1], ROW, FAMILY, QUALIFIERS[1], 2); assertIncrementKey(kvs[2], ROW, FAMILY, QUALIFIERS[2], 2); - + ht.close(); } @@ -5621,7 +5617,7 @@ public class TestFromClientSide { table.put(put); } - // nomal scan + // normal scan ResultScanner scanner = table.getScanner(new Scan()); int count = 0; for (Result r : scanner) { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSideAsync.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSideAsync.java new file mode 100644 index 0000000..8c83e9d --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSideAsync.java @@ -0,0 +1,593 @@ +/** + * + * 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.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.HColumnDescriptor; +import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.filter.CompareFilter; +import org.apache.hadoop.hbase.regionserver.BloomType; +import org.apache.hadoop.hbase.testclassification.ClientTests; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.junit.After; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +import java.io.IOException; +import java.util.Arrays; +import java.util.concurrent.ExecutionException; + +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +/** + * Run tests that use the HBase clients; {@link HTable}. + * Sets up the HBase mini cluster once at start and runs through all client tests. + * Each creates a table named for the method and does its stuff against that. + */ +@Category({LargeTests.class, ClientTests.class}) +@SuppressWarnings ("deprecation") +public class TestFromClientSideAsync { + private static final Log LOG = LogFactory.getLog(TestFromClientSideAsync.class); + protected final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); + private static final byte [] ROW = Bytes.toBytes("testRow"); + private static final byte [] FAMILY = Bytes.toBytes("testFamily"); + private static final byte [] QUALIFIER = Bytes.toBytes("testQualifier"); + private static final byte [] VALUE = Bytes.toBytes("testValue"); + protected static int SLAVES = 3; + + /** + * @throws Exception + */ + @BeforeClass + public static void setUpBeforeClass() throws Exception { + // Uncomment the following lines if more verbosity is needed for + // debugging (see HBASE-12285 for details). + //((Log4JLogger)RpcServer.LOG).getLogger().setLevel(Level.ALL); + //((Log4JLogger)RpcClient.LOG).getLogger().setLevel(Level.ALL); + //((Log4JLogger)ScannerCallable.LOG).getLogger().setLevel(Level.ALL); + // We need more than one region server in this test + TEST_UTIL.startMiniCluster(SLAVES); + } + + /** + * @throws Exception + */ + @AfterClass + public static void tearDownAfterClass() throws Exception { + TEST_UTIL.shutdownMiniCluster(); + } + + /** + * @throws Exception + */ + @Before + public void setUp() throws Exception { + // Nothing to do. + } + + /** + * @throws Exception + */ + @After + public void tearDown() throws Exception { + // Nothing to do. + } + + @Test + public void testGet_NonExistentRow() throws IOException, ExecutionException, + InterruptedException { + AsyncTable table = this.createAsyncTable(TableName.valueOf("testGet_NonExistentRow"), null); + + Put put = new Put(ROW); + put.add(FAMILY, QUALIFIER, VALUE); + table.mutate(put).get(); + LOG.info("Row put"); + + Get get = new Get(ROW); + get.addFamily(FAMILY); + Result r = table.get(get).get(); + assertFalse(r.isEmpty()); + LOG.info("Row retrieved successfully"); + + byte [] missingrow = Bytes.toBytes("missingrow"); + get = new Get(missingrow); + get.addFamily(FAMILY); + r = table.get(get).get(); + assertTrue(r.isEmpty()); + LOG.info("Row missing as it should be"); + } + + @Test + public void testMultiGet() throws IOException, ExecutionException, + InterruptedException { + AsyncTable table = this.createAsyncTable(TableName.valueOf("testMultiGet"), null); + + Put put = new Put(ROW); + put.add(FAMILY, QUALIFIER, VALUE); + table.mutate(put).get(); + LOG.info("Row put"); + + final byte[] ROW2 = Bytes.toBytes("testRow2"); + + put = new Put(ROW2); + put.add(FAMILY, QUALIFIER, VALUE); + table.mutate(put).get(); + LOG.info("Row put"); + + Get get = new Get(ROW); + get.addFamily(FAMILY); + + Get get2 = new Get(ROW2); + get2.addFamily(FAMILY); + + Result[] r = table.get(Arrays.asList(get, get2)).get(); + assertFalse(r[0].isEmpty()); + assertFalse(r[1].isEmpty()); + LOG.info("Row retrieved successfully"); + + byte [] missingrow = Bytes.toBytes("missingrow"); + get = new Get(missingrow); + get.addFamily(FAMILY); + r = table.get(Arrays.asList(get, get2)).get(); + assertTrue(r[0].isEmpty()); + LOG.info("Row missing as it should be"); + assertFalse(r[1].isEmpty()); + } + + @Test + public void testPut() throws IOException, ExecutionException, InterruptedException { + final byte [] CONTENTS_FAMILY = Bytes.toBytes("contents"); + final byte [] SMALL_FAMILY = Bytes.toBytes("smallfam"); + final byte [] row1 = Bytes.toBytes("row1"); + final byte [] row2 = Bytes.toBytes("row2"); + final byte [] value = Bytes.toBytes("abcd"); + AsyncTable table = this.createAsyncTable(TableName.valueOf("testPut"), + new byte[][] { CONTENTS_FAMILY, SMALL_FAMILY }); + Put put = new Put(row1); + put.add(CONTENTS_FAMILY, null, value); + table.mutate(put).get(); + + put = new Put(row2); + put.add(CONTENTS_FAMILY, null, value); + + assertEquals(put.size(), 1); + assertEquals(put.getFamilyCellMap().get(CONTENTS_FAMILY).size(), 1); + + // KeyValue v1 expectation. Cast for now until we go all Cell all the time. TODO + KeyValue kv = (KeyValue)put.getFamilyCellMap().get(CONTENTS_FAMILY).get(0); + + assertTrue(Bytes.equals(kv.getFamily(), CONTENTS_FAMILY)); + // will it return null or an empty byte array? + assertTrue(Bytes.equals(kv.getQualifier(), new byte[0])); + + assertTrue(Bytes.equals(kv.getValue(), value)); + + table.mutate(put).get(); + + Table syncTable = TEST_UTIL.getConnection().getTable(table.getName()); + + Scan scan = new Scan(); + scan.addColumn(CONTENTS_FAMILY, null); + ResultScanner scanner = syncTable.getScanner(scan); + for (Result r : scanner) { + for(Cell key : r.rawCells()) { + System.out.println(Bytes.toString(r.getRow()) + ": " + key.toString()); + } + } + } + + @Test + public void testIncrement() throws Exception { + LOG.info("Starting testIncrement"); + + AsyncTable ht = this.createAsyncTable(TableName.valueOf("testIncrement"), null); + + byte [][] ROWS = new byte [][] { + Bytes.toBytes("a"), Bytes.toBytes("b"), Bytes.toBytes("c"), + Bytes.toBytes("d"), Bytes.toBytes("e"), Bytes.toBytes("f"), + Bytes.toBytes("g"), Bytes.toBytes("h"), Bytes.toBytes("i") + }; + byte [][] QUALIFIERS = new byte [][] { + Bytes.toBytes("a"), Bytes.toBytes("b"), Bytes.toBytes("c"), + Bytes.toBytes("d"), Bytes.toBytes("e"), Bytes.toBytes("f"), + Bytes.toBytes("g"), Bytes.toBytes("h"), Bytes.toBytes("i") + }; + + // Do some simple single-column increments + + Increment inc = new Increment(ROW); + inc.addColumn(FAMILY, QUALIFIERS[0], 1); + inc.addColumn(FAMILY, QUALIFIERS[1], 2); + inc.addColumn(FAMILY, QUALIFIERS[2], 3); + inc.addColumn(FAMILY, QUALIFIERS[3], 4); + ht.mutate(inc).get(); + + // Now increment things incremented with old and do some new + inc = new Increment(ROW); + inc.addColumn(FAMILY, QUALIFIERS[1], 1); + inc.addColumn(FAMILY, QUALIFIERS[3], 1); + inc.addColumn(FAMILY, QUALIFIERS[4], 1); + ht.mutate(inc).get(); + + // Verify expected results + Result r = ht.get(new Get(ROW)).get(); + Cell [] kvs = r.rawCells(); + assertEquals(5, kvs.length); + assertIncrementKey(kvs[0], ROW, FAMILY, QUALIFIERS[0], 1); + assertIncrementKey(kvs[1], ROW, FAMILY, QUALIFIERS[1], 3); + assertIncrementKey(kvs[2], ROW, FAMILY, QUALIFIERS[2], 3); + assertIncrementKey(kvs[3], ROW, FAMILY, QUALIFIERS[3], 5); + assertIncrementKey(kvs[4], ROW, FAMILY, QUALIFIERS[4], 1); + + // Now try multiple columns by different amounts + inc = new Increment(ROWS[0]); + for (int i=0;i iter = scanner.iterator(); iter.next(); - Assert.assertTrue(((ClientScanner)scanner).isAnyRPCcancelled()); + Assert.assertTrue(((ClientScanner) scanner).isAnyRPCcancelled()); SlowMeCopro.slowDownNext.set(false); SlowMeCopro.countOfNext.set(0); } finally { @@ -687,7 +687,7 @@ public class TestReplicasClient { // we won't get it SlowMeCopro.sleepTime.set(5000); scanWithReplicas(reversed, small, Consistency.STRONG, caching, maxResultSize, start, NUMROWS, - NUMCOLS, false, false); + NUMCOLS, false, false); SlowMeCopro.sleepTime.set(0); flushRegion(hriPrimary); @@ -696,22 +696,22 @@ public class TestReplicasClient { //Now set the flag to get a response even if stale SlowMeCopro.sleepTime.set(5000); - scanWithReplicas(reversed, small, Consistency.TIMELINE, caching, maxResultSize, - start, NUMROWS, NUMCOLS, true, false); + scanWithReplicas(reversed, small, Consistency.TIMELINE, caching, maxResultSize, start, + NUMROWS, NUMCOLS, true, false); SlowMeCopro.sleepTime.set(0); // now make some 'next' calls slow SlowMeCopro.slowDownNext.set(true); SlowMeCopro.countOfNext.set(0); scanWithReplicas(reversed, small, Consistency.TIMELINE, caching, maxResultSize, start, - NUMROWS, NUMCOLS, true, true); + NUMROWS, NUMCOLS, true, true); SlowMeCopro.slowDownNext.set(false); SlowMeCopro.countOfNext.set(0); // Make sure we do not get stale data.. SlowMeCopro.sleepTime.set(5000); - scanWithReplicas(reversed, small, Consistency.STRONG, caching, maxResultSize, - start, NUMROWS, NUMCOLS, false, false); + scanWithReplicas(reversed, small, Consistency.STRONG, caching, maxResultSize, start, NUMROWS, + NUMCOLS, false, false); SlowMeCopro.sleepTime.set(0); // While the next calls are slow, set maxResultSize to 1 so that some partial results will be -- 2.3.2 (Apple Git-55)