[ https://issues.apache.org/jira/browse/DRILL-4335?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15991410#comment-15991410 ]
ASF GitHub Bot commented on DRILL-4335: --------------------------------------- Github user sohami commented on a diff in the pull request: https://github.com/apache/drill/pull/773#discussion_r113836743 --- Diff: exec/rpc/src/main/java/org/apache/drill/exec/rpc/SaslDecryptionHandler.java --- @@ -0,0 +1,162 @@ +/* + * 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.drill.exec.rpc; + +import io.netty.buffer.ByteBuf; +import io.netty.channel.ChannelHandlerContext; +import io.netty.handler.codec.MessageToMessageDecoder; + +import org.apache.drill.exec.exception.OutOfMemoryException; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.util.List; + +import static com.google.common.base.Preconditions.checkArgument; + +/** + * Handler to Decrypt the input ByteBuf. It expects input to be in format where it has length of the bytes to + * decode in network order and actual encrypted bytes. The handler reads the length and then reads the + * required bytes to pass it to unwrap function for decryption. The decrypted buffer is copied to a new + * ByteBuf and added to out list. + * <p> + * Example: + * <li>Input - [EBLN1, EB1, EBLN2, EB2] --> ByteBuf with repeated combination of encrypted byte length + * in network order (EBLNx) and encrypted bytes (EB) + * <li>Output - [DB1] --> Decrypted ByteBuf of first chunk.(EB1) + * </p> + */ +class SaslDecryptionHandler extends MessageToMessageDecoder<ByteBuf> { + + private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger( + SaslDecryptionHandler.class.getCanonicalName()); + + private final SaslCodec saslCodec; + + private final int maxEncodedSize; + + private final OutOfMemoryHandler outOfMemoryHandler; + + private final byte[] encodedMsg; + + private final ByteBuffer lengthOctets; + + SaslDecryptionHandler(SaslCodec saslCodec, int maxEncodedSize, OutOfMemoryHandler oomHandler) { + this.saslCodec = saslCodec; + this.outOfMemoryHandler = oomHandler; + this.maxEncodedSize = maxEncodedSize; + + // Allocate the byte array of maxEncodedSize to reuse for each encoded packet received on this connection + // Maximum value of maxEncodedSize can be 16MB (i.e. OXFFFFFF) + encodedMsg = new byte[maxEncodedSize]; + lengthOctets = ByteBuffer.allocate(RpcConstants.LENGTH_FIELD_LENGTH); + lengthOctets.order(ByteOrder.BIG_ENDIAN); + } + + @Override + public void handlerAdded(ChannelHandlerContext ctx) throws Exception { + super.handlerAdded(ctx); + logger.trace("Added " + RpcConstants.SASL_DECRYPTION_HANDLER + " handler"); + } + + @Override + public void handlerRemoved(ChannelHandlerContext ctx) throws Exception { + super.handlerRemoved(ctx); + logger.trace("Removed " + RpcConstants.SASL_DECRYPTION_HANDLER + " handler"); + } + + public void decode(ChannelHandlerContext ctx, ByteBuf msg, List<Object> out) throws IOException { + + if (!ctx.channel().isOpen()) { + logger.trace("Channel closed before decoding the message of {} bytes", msg.readableBytes()); + msg.skipBytes(msg.readableBytes()); + return; + } + + try { + if(logger.isTraceEnabled()) { + logger.trace("Trying to decrypt the encrypted message of size: {} with maxEncodedSize", msg.readableBytes()); + } + + final byte[] wrappedMsg; + + // All the encrypted blocks are prefixed with it's length in network byte order (or BigEndian format). Netty's + // default Byte order of ByteBuf is Little Endian, so we cannot just do msg.getInt() as that will read the 4 + // octets in little endian format. + // + // We will read the length of one complete encrypted chunk and decode that. + msg.getBytes(msg.readerIndex(), lengthOctets.array(), 0, RpcConstants.LENGTH_FIELD_LENGTH); + int wrappedMsgLength = lengthOctets.getInt(0); + msg.skipBytes(RpcConstants.LENGTH_FIELD_LENGTH); + + // Since lengthBasedFrameDecoder will ensure we have enough bytes it's good to have this check here. + checkArgument(msg.readableBytes() >= wrappedMsgLength); --- End diff -- `checkState` and `checkArgument` both does the same thing. Used CheckArgument since `msg` is an argument to decode method. > Apache Drill should support network encryption > ---------------------------------------------- > > Key: DRILL-4335 > URL: https://issues.apache.org/jira/browse/DRILL-4335 > Project: Apache Drill > Issue Type: New Feature > Reporter: Keys Botzum > Assignee: Sorabh Hamirwasia > Labels: security > Attachments: ApacheDrillEncryptionUsingSASLDesign.pdf > > > This is clearly related to Drill-291 but wanted to make explicit that this > needs to include network level encryption and not just authentication. This > is particularly important for the client connection to Drill which will often > be sending passwords in the clear until there is encryption. -- This message was sent by Atlassian JIRA (v6.3.15#6346)