/** * Decodes the response of an RPC and triggers its {@link Deferred}. * * <p>This method is used by FrameDecoder when the channel gets disconnected. The buffer for that * channel is passed to this method in case there's anything left in it. * * @param ctx Unused. * @param chan The channel on which the response came. * @param buf The buffer containing the raw RPC response. * @return {@code null}, always. */ @Override protected Object decodeLast( final ChannelHandlerContext ctx, final Channel chan, final ChannelBuffer buf, final VoidEnum unused) { // When we disconnect, decodeLast is called instead of decode. // We simply check whether there's any data left in the buffer, in which // case we attempt to process it. But if there's no data left, then we // don't even bother calling decode() as it'll complain that the buffer // doesn't contain enough data, which unnecessarily pollutes the logs. if (buf.readable()) { try { return decode(ctx, chan, buf, unused); } finally { if (buf.readable()) { LOG.error( getPeerUuidLoggingString() + "After decoding the last message on " + chan + ", there was still some undecoded bytes in the channel's" + " buffer (which are going to be lost): " + buf + '=' + Bytes.pretty(buf)); } } } else { return null; } }
/** * 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; }