/** * Calls {@link #getMockedConnection(Configuration)} and then mocks a few more of the popular * {@link ClusterConnection} methods so they do 'normal' operation (see return doc below for * list). Be sure to shutdown the connection when done by calling {@link Connection#close()} else * it will stick around; this is probably not what you want. * * @param conf Configuration to use * @param admin An AdminProtocol; can be null but is usually itself a mock. * @param client A ClientProtocol; can be null but is usually itself a mock. * @param sn ServerName to include in the region location returned by this <code>connection</code> * @param hri HRegionInfo to include in the location returned when getRegionLocator is called on * the mocked connection * @return Mock up a connection that returns a {@link Configuration} when {@link * ClusterConnection#getConfiguration()} is called, a 'location' when {@link * ClusterConnection#getRegionLocation(org.apache.hadoop.hbase.TableName, byte[], boolean)} is * called, and that returns the passed {@link AdminProtos.AdminService.BlockingInterface} * instance when {@link ClusterConnection#getAdmin(ServerName)} is called, returns the passed * {@link ClientProtos.ClientService.BlockingInterface} instance when {@link * ClusterConnection#getClient(ServerName)} is called (Be sure to call {@link * Connection#close()} when done with this mocked Connection. * @throws IOException */ public static ClusterConnection getMockedConnectionAndDecorate( final Configuration conf, final AdminProtos.AdminService.BlockingInterface admin, final ClientProtos.ClientService.BlockingInterface client, final ServerName sn, final HRegionInfo hri) throws IOException { ConnectionImplementation c = Mockito.mock(ConnectionImplementation.class); Mockito.when(c.getConfiguration()).thenReturn(conf); Mockito.doNothing().when(c).close(); // Make it so we return a particular location when asked. final HRegionLocation loc = new HRegionLocation(hri, sn); Mockito.when( c.getRegionLocation( (TableName) Mockito.any(), (byte[]) Mockito.any(), Mockito.anyBoolean())) .thenReturn(loc); Mockito.when(c.locateRegion((TableName) Mockito.any(), (byte[]) Mockito.any())).thenReturn(loc); Mockito.when( c.locateRegion( (TableName) Mockito.any(), (byte[]) Mockito.any(), Mockito.anyBoolean(), Mockito.anyBoolean(), Mockito.anyInt())) .thenReturn(new RegionLocations(loc)); if (admin != null) { // If a call to getAdmin, return this implementation. Mockito.when(c.getAdmin(Mockito.any(ServerName.class))).thenReturn(admin); } if (client != null) { // If a call to getClient, return this client. Mockito.when(c.getClient(Mockito.any(ServerName.class))).thenReturn(client); } NonceGenerator ng = Mockito.mock(NonceGenerator.class); Mockito.when(c.getNonceGenerator()).thenReturn(ng); Mockito.when(c.getAsyncProcess()) .thenReturn( new AsyncProcess( c, conf, null, RpcRetryingCallerFactory.instantiate(conf), false, RpcControllerFactory.instantiate(conf), conf.getInt(HConstants.HBASE_RPC_TIMEOUT_KEY, HConstants.DEFAULT_HBASE_RPC_TIMEOUT), conf.getInt( HConstants.HBASE_CLIENT_OPERATION_TIMEOUT, HConstants.DEFAULT_HBASE_CLIENT_OPERATION_TIMEOUT))); Mockito.when(c.getNewRpcRetryingCallerFactory(conf)) .thenReturn( RpcRetryingCallerFactory.instantiate( conf, RetryingCallerInterceptorFactory.NO_OP_INTERCEPTOR, null)); Mockito.when(c.getRpcControllerFactory()).thenReturn(Mockito.mock(RpcControllerFactory.class)); Table t = Mockito.mock(Table.class); Mockito.when(c.getTable((TableName) Mockito.any())).thenReturn(t); ResultScanner rs = Mockito.mock(ResultScanner.class); Mockito.when(t.getScanner((Scan) Mockito.any())).thenReturn(rs); return c; }
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 = RpcRetryingCallerFactory.instantiate(ScannerCallableWithReplicas.this.conf) .<Result[]>newCaller(); } }
public FlushWorker( Configuration conf, ClusterConnection conn, HRegionLocation addr, HTableMultiplexer htableMultiplexer, int perRegionServerBufferQueueSize, ExecutorService pool, ScheduledExecutorService executor) { this.addr = addr; this.multiplexer = htableMultiplexer; this.queue = new LinkedBlockingQueue<>(perRegionServerBufferQueueSize); RpcRetryingCallerFactory rpcCallerFactory = RpcRetryingCallerFactory.instantiate(conf); RpcControllerFactory rpcControllerFactory = RpcControllerFactory.instantiate(conf); this.ap = new AsyncProcess(conn, conf, pool, rpcCallerFactory, false, rpcControllerFactory); this.executor = executor; this.maxRetryInQueue = conf.getInt(TABLE_MULTIPLEXER_MAX_RETRIES_IN_QUEUE, 10000); }
@Test(timeout = 30000) public void testBulkLoad() throws IOException { // Create table then get the single region for our new table. LOG.debug("Creating test table"); HTableDescriptor hdt = HTU.createTableDescriptor("testBulkLoad"); hdt.setRegionReplication(NB_SERVERS); hdt.addCoprocessor(SlowMeCopro.class.getName()); Table table = HTU.createTable(hdt, new byte[][] {f}, HTU.getConfiguration()); // create hfiles to load. LOG.debug("Creating test data"); Path dir = HTU.getDataTestDirOnTestFS("testBulkLoad"); final int numRows = 10; final byte[] qual = Bytes.toBytes("qual"); final byte[] val = Bytes.toBytes("val"); final List<Pair<byte[], String>> famPaths = new ArrayList<Pair<byte[], String>>(); for (HColumnDescriptor col : hdt.getColumnFamilies()) { Path hfile = new Path(dir, col.getNameAsString()); TestHRegionServerBulkLoad.createHFile( HTU.getTestFileSystem(), hfile, col.getName(), qual, val, numRows); famPaths.add(new Pair<byte[], String>(col.getName(), hfile.toString())); } // bulk load HFiles LOG.debug("Loading test data"); @SuppressWarnings("deprecation") final HConnection conn = HTU.getHBaseAdmin().getConnection(); RegionServerCallable<Void> callable = new RegionServerCallable<Void>( conn, hdt.getTableName(), TestHRegionServerBulkLoad.rowkey(0)) { @Override public Void call(int timeout) throws Exception { LOG.debug( "Going to connect to server " + getLocation() + " for row " + Bytes.toStringBinary(getRow())); byte[] regionName = getLocation().getRegionInfo().getRegionName(); BulkLoadHFileRequest request = RequestConverter.buildBulkLoadHFileRequest(famPaths, regionName, true); getStub().bulkLoadHFile(null, request); return null; } }; RpcRetryingCallerFactory factory = new RpcRetryingCallerFactory(HTU.getConfiguration()); RpcRetryingCaller<Void> caller = factory.<Void>newCaller(); caller.callWithRetries(callable, 10000); // verify we can read them from the primary LOG.debug("Verifying data load"); for (int i = 0; i < numRows; i++) { byte[] row = TestHRegionServerBulkLoad.rowkey(i); Get g = new Get(row); Result r = table.get(g); Assert.assertFalse(r.isStale()); } // verify we can read them from the replica LOG.debug("Verifying replica queries"); try { SlowMeCopro.cdl.set(new CountDownLatch(1)); for (int i = 0; i < numRows; i++) { byte[] row = TestHRegionServerBulkLoad.rowkey(i); Get g = new Get(row); g.setConsistency(Consistency.TIMELINE); Result r = table.get(g); Assert.assertTrue(r.isStale()); } SlowMeCopro.cdl.get().countDown(); } finally { SlowMeCopro.cdl.get().countDown(); SlowMeCopro.sleepTime.set(0); } HTU.getHBaseAdmin().disableTable(hdt.getTableName()); HTU.deleteTable(hdt.getTableName()); }
@Override public Result[] call(int timeout) throws IOException { // 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 // the closed flag. Then an RPC is required to actually close the scanner. if (currentScannerCallable != null && currentScannerCallable.closed) { // For closing we target that exact scanner (and not do replica fallback like in // the case of normal reads) if (LOG.isTraceEnabled()) { LOG.trace("Closing scanner id=" + currentScannerCallable.scannerId); } Result[] 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<Pair<Result[], ScannerCallable>> cs = new ResultBoundedCompletionService<Pair<Result[], ScannerCallable>>( RpcRetryingCallerFactory.instantiate(ScannerCallableWithReplicas.this.conf), pool, rl.size() * 5); AtomicBoolean done = new AtomicBoolean(false); replicaSwitched.set(false); // submit call for the primary replica. addCallsForCurrentReplica(cs, rl); try { // wait for the timeout to see whether the primary responds back Future<Pair<Result[], ScannerCallable>> f = cs.poll(timeBeforeReplicas, TimeUnit.MICROSECONDS); // Yes, microseconds if (f != null) { Pair<Result[], ScannerCallable> r = f.get(timeout, TimeUnit.MILLISECONDS); if (r != null && r.getSecond() != null) { updateCurrentlyServingReplica(r.getSecond(), r.getFirst(), done, pool); } return r == null ? null : r.getFirst(); // great we got a response } } catch (ExecutionException e) { RpcRetryingCallerWithReadReplicas.throwEnrichedException(e, retries); } catch (CancellationException e) { throw new InterruptedIOException(e.getMessage()); } catch (InterruptedException e) { throw new InterruptedIOException(e.getMessage()); } catch (TimeoutException 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 addCallsForOtherReplicas(cs, rl, 0, rl.size() - 1); try { Future<Pair<Result[], ScannerCallable>> f = cs.poll(timeout, TimeUnit.MILLISECONDS); if (f != null) { Pair<Result[], ScannerCallable> r = f.get(timeout, TimeUnit.MILLISECONDS); 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) { RpcRetryingCallerWithReadReplicas.throwEnrichedException(e, retries); } catch (CancellationException e) { throw new InterruptedIOException(e.getMessage()); } catch (InterruptedException e) { throw new InterruptedIOException(e.getMessage()); } catch (TimeoutException 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(); } return null; // unreachable }