/** * Forcefully shuts down the connection to this tablet server and fails all the outstanding RPCs. * Only use when shutting down a client. * * @return deferred object to use to track the shutting down of this connection */ public Deferred<Void> shutdown() { // First, check whether we have RPCs in flight and cancel them. for (Iterator<KuduRpc<?>> ite = rpcs_inflight.values().iterator(); ite.hasNext(); ) { KuduRpc<?> rpc = ite.next(); rpc.errback(new ConnectionResetException(null)); ite.remove(); } // Same for the pending RPCs. synchronized (this) { if (pending_rpcs != null) { for (Iterator<KuduRpc<?>> ite = pending_rpcs.iterator(); ite.hasNext(); ) { ite.next().errback(new ConnectionResetException(null)); ite.remove(); } } } final Channel chancopy = chan; if (chancopy == null) { return Deferred.fromResult(null); } if (chancopy.isConnected()) { Channels.disconnect(chancopy); // ... this is going to set it to null. // At this point, all in-flight RPCs are going to be failed. } if (chancopy.isBound()) { Channels.unbind(chancopy); } // It's OK to call close() on a Channel if it's already closed. final ChannelFuture future = Channels.close(chancopy); // Now wrap the ChannelFuture in a Deferred. final Deferred<Void> d = new Deferred<Void>(); // Opportunistically check if it's already completed successfully. if (future.isSuccess()) { d.callback(null); } else { // If we get here, either the future failed (yeah, that sounds weird) // or the future hasn't completed yet (heh). future.addListener( new ChannelFutureListener() { public void operationComplete(final ChannelFuture future) { if (future.isSuccess()) { d.callback(null); return; } final Throwable t = future.getCause(); if (t instanceof Exception) { d.callback(t); } else { // Wrap the Throwable because Deferred doesn't handle Throwables, // it only uses Exception. d.callback( new NonRecoverableException("Failed to shutdown: " + TabletClient.this, t)); } } }); } return d; }
/** * Retry the given RPC. * * @param rpc an RPC to retry or fail * @param exception an exception to propagate with the RPC */ private void failOrRetryRpc(final KuduRpc<?> rpc, final ConnectionResetException exception) { AsyncKuduClient.RemoteTablet tablet = rpc.getTablet(); // Note As of the time of writing (03/11/16), a null tablet doesn't make sense, if we see a null // tablet it's because we didn't set it properly before calling sendRpc(). if (tablet == null) { // Can't retry, dunno where this RPC should go. rpc.errback(exception); } else { kuduClient.handleRetryableError(rpc, exception); } }
private ChannelBuffer header() { RpcHeader.ConnectionContextPB.Builder builder = RpcHeader.ConnectionContextPB.newBuilder(); RpcHeader.UserInformationPB.Builder userBuilder = RpcHeader.UserInformationPB.newBuilder(); userBuilder.setEffectiveUser(SecureRpcHelper.USER_AND_PASSWORD); // TODO set real user userBuilder.setRealUser(SecureRpcHelper.USER_AND_PASSWORD); builder.setUserInfo(userBuilder.build()); RpcHeader.ConnectionContextPB pb = builder.build(); RpcHeader.RequestHeader header = RpcHeader.RequestHeader.newBuilder().setCallId(CONNECTION_CTX_CALL_ID).build(); return KuduRpc.toChannelBuffer(header, pb); }
/** * The reason we are suppressing the unchecked conversions is because the KuduRpc is coming from a * collection that has RPCs with different generics, and there's no way to get "decoded" casted * correctly. The best we can do is to rely on the RPC to decode correctly, and to not pass an * Exception in the callback. */ @Override @SuppressWarnings("unchecked") protected Object decode( ChannelHandlerContext ctx, Channel chan, ChannelBuffer buf, VoidEnum voidEnum) { final long start = System.nanoTime(); final int rdx = buf.readerIndex(); LOG.debug("------------------>> ENTERING DECODE >>------------------"); try { buf = secureRpcHelper.handleResponse(buf, chan); } catch (SaslException e) { String message = getPeerUuidLoggingString() + "Couldn't complete the SASL handshake"; LOG.error(message); throw new NonRecoverableException(message, e); } if (buf == null) { return null; } CallResponse response = new CallResponse(buf); RpcHeader.ResponseHeader header = response.getHeader(); if (!header.hasCallId()) { final int size = response.getTotalResponseSize(); final String msg = getPeerUuidLoggingString() + "RPC response (size: " + size + ") doesn't" + " have a call ID: " + header + ", buf=" + Bytes.pretty(buf); LOG.error(msg); throw new NonRecoverableException(msg); } final int rpcid = header.getCallId(); @SuppressWarnings("rawtypes") final KuduRpc rpc = rpcs_inflight.get(rpcid); if (rpc == null) { final String msg = getPeerUuidLoggingString() + "Invalid rpcid: " + rpcid + " found in " + buf + '=' + Bytes.pretty(buf); LOG.error(msg); // The problem here is that we don't know which Deferred corresponds to // this RPC, since we don't have a valid ID. So we're hopeless, we'll // never be able to recover because responses are not framed, we don't // know where the next response will start... We have to give up here // and throw this outside of our Netty handler, so Netty will call our // exception handler where we'll close this channel, which will cause // all RPCs in flight to be failed. throw new NonRecoverableException(msg); } Pair<Object, Object> decoded = null; Exception exception = null; KuduException retryableHeaderException = null; if (header.hasIsError() && header.getIsError()) { RpcHeader.ErrorStatusPB.Builder errorBuilder = RpcHeader.ErrorStatusPB.newBuilder(); KuduRpc.readProtobuf(response.getPBMessage(), errorBuilder); RpcHeader.ErrorStatusPB error = errorBuilder.build(); if (error.getCode().equals(RpcHeader.ErrorStatusPB.RpcErrorCodePB.ERROR_SERVER_TOO_BUSY)) { // We can't return right away, we still need to remove ourselves from 'rpcs_inflight', so we // populate 'retryableHeaderException'. retryableHeaderException = new TabletServerErrorException(uuid, error); } else { String message = getPeerUuidLoggingString() + "Tablet server sent error " + error.getMessage(); exception = new NonRecoverableException(message); LOG.error(message); // can be useful } } else { try { decoded = rpc.deserialize(response, this.uuid); } catch (Exception ex) { exception = ex; } } if (LOG.isDebugEnabled()) { LOG.debug( getPeerUuidLoggingString() + "rpcid=" + rpcid + ", response size=" + (buf.readerIndex() - rdx) + " bytes" + ", " + actualReadableBytes() + " readable bytes left" + ", rpc=" + rpc); } { final KuduRpc<?> removed = rpcs_inflight.remove(rpcid); if (removed == null) { // The RPC we were decoding was cleaned up already, give up. throw new NonRecoverableException("RPC not found"); } } // This check is specifically for the ERROR_SERVER_TOO_BUSY case above. if (retryableHeaderException != null) { kuduClient.handleRetryableError(rpc, retryableHeaderException); return null; } // We can get this Message from within the RPC's expected type, // so convert it into an exception and nullify decoded so that we use the errback route. // Have to do it for both TS and Master errors. if (decoded != null) { if (decoded.getSecond() instanceof Tserver.TabletServerErrorPB) { Tserver.TabletServerErrorPB error = (Tserver.TabletServerErrorPB) decoded.getSecond(); exception = dispatchTSErrorOrReturnException(rpc, error); if (exception == null) { // It was taken care of. return null; } else { // We're going to errback. decoded = null; } } else if (decoded.getSecond() instanceof Master.MasterErrorPB) { Master.MasterErrorPB error = (Master.MasterErrorPB) decoded.getSecond(); exception = dispatchMasterErrorOrReturnException(rpc, error); if (exception == null) { // Exception was taken care of. return null; } else { decoded = null; } } } try { if (decoded != null) { assert !(decoded.getFirst() instanceof Exception); if (kuduClient.isStatisticsEnabled()) { rpc.updateStatistics(kuduClient.getStatistics(), decoded.getFirst()); } rpc.callback(decoded.getFirst()); } else { if (kuduClient.isStatisticsEnabled()) { rpc.updateStatistics(kuduClient.getStatistics(), null); } rpc.errback(exception); } } catch (Exception e) { LOG.debug( getPeerUuidLoggingString() + "Unexpected exception while handling RPC #" + rpcid + ", rpc=" + rpc + ", buf=" + Bytes.pretty(buf), e); } if (LOG.isDebugEnabled()) { LOG.debug( "------------------<< LEAVING DECODE <<------------------" + " time elapsed: " + ((System.nanoTime() - start) / 1000) + "us"); } return null; // Stop processing here. The Deferred does everything else. }
private <R> ChannelBuffer encode(final KuduRpc<R> rpc) { final int rpcid = this.rpcid.incrementAndGet(); ChannelBuffer payload; final String service = rpc.serviceName(); final String method = rpc.method(); try { final RpcHeader.RequestHeader.Builder headerBuilder = RpcHeader.RequestHeader.newBuilder() .setCallId(rpcid) .addAllRequiredFeatureFlags(rpc.getRequiredFeatures()) .setRemoteMethod( RpcHeader.RemoteMethodPB.newBuilder() .setServiceName(service) .setMethodName(method)); // If any timeout is set, find the lowest non-zero one, since this will be the deadline that // the server must respect. if (rpc.deadlineTracker.hasDeadline() || socketReadTimeoutMs > 0) { long millisBeforeDeadline = Long.MAX_VALUE; if (rpc.deadlineTracker.hasDeadline()) { millisBeforeDeadline = rpc.deadlineTracker.getMillisBeforeDeadline(); } long localRpcTimeoutMs = Long.MAX_VALUE; if (socketReadTimeoutMs > 0) { localRpcTimeoutMs = socketReadTimeoutMs; } headerBuilder.setTimeoutMillis((int) Math.min(millisBeforeDeadline, localRpcTimeoutMs)); } payload = rpc.serialize(headerBuilder.build()); } catch (Exception e) { LOG.error("Uncaught exception while serializing RPC: " + rpc, e); rpc.errback(e); // Make the RPC fail with the exception. return null; } final KuduRpc<?> oldrpc = rpcs_inflight.put(rpcid, rpc); if (oldrpc != null) { final String wtf = getPeerUuidLoggingString() + "WTF? There was already an RPC in flight with" + " rpcid=" + rpcid + ": " + oldrpc + ". This happened when sending out: " + rpc; LOG.error(wtf); // Make it fail. This isn't an expected failure mode. oldrpc.errback(new NonRecoverableException(wtf)); } if (LOG.isDebugEnabled()) { LOG.debug( getPeerUuidLoggingString() + chan + " Sending RPC #" + rpcid + ", payload=" + payload + ' ' + Bytes.pretty(payload)); } payload = secureRpcHelper.wrap(payload); return payload; }
<R> void sendRpc(KuduRpc<R> rpc) { if (!rpc.deadlineTracker.hasDeadline()) { LOG.warn(getPeerUuidLoggingString() + " sending an rpc without a timeout " + rpc); } if (chan != null) { if (!rpc.getRequiredFeatures().isEmpty() && !secureRpcHelper .getServerFeatures() .contains(RpcHeader.RpcFeatureFlag.APPLICATION_FEATURE_FLAGS)) { rpc.errback( new NonRecoverableException( "the server does not support the APPLICATION_FEATURE_FLAGS RPC feature")); } final ChannelBuffer serialized = encode(rpc); if (serialized == null) { // Error during encoding. return; // Stop here. RPC has been failed already. } final Channel chan = this.chan; // Volatile read. if (chan != null) { // Double check if we disconnected during encode(). Channels.write(chan, serialized); } else { // The RPC was already added to rpcs_inflight so we don't need to fall down in the next big // block of code, cleanup() will take care of it. if (LOG.isDebugEnabled()) { LOG.debug( getPeerUuidLoggingString() + " connection was closed before sending rpcid {}, rpc=", rpcid, rpc); } } return; } boolean tryagain = false; boolean copyOfDead; synchronized (this) { copyOfDead = this.dead; // Check if we got connected while entering this synchronized block. if (chan != null) { tryagain = true; } else if (!copyOfDead) { if (pending_rpcs == null) { pending_rpcs = new ArrayList<KuduRpc<?>>(); } pending_rpcs.add(rpc); } } if (copyOfDead) { failOrRetryRpc(rpc, new ConnectionResetException(null)); return; } else if (tryagain) { // This recursion will not lead to a loop because we only get here if we // connected while entering the synchronized block above. So when trying // a second time, we will either succeed to send the RPC if we're still // connected, or fail through to the code below if we got disconnected // in the mean time. sendRpc(rpc); return; } }