[ 
https://issues.apache.org/jira/browse/DRILL-4335?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15984074#comment-15984074
 ] 

ASF GitHub Bot commented on DRILL-4335:
---------------------------------------

Github user sudheeshkatkam commented on a diff in the pull request:

    https://github.com/apache/drill/pull/773#discussion_r113347315
  
    --- Diff: 
exec/rpc/src/main/java/org/apache/drill/exec/rpc/SaslEncryptionHandler.java ---
    @@ -0,0 +1,181 @@
    +/*
    + * 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.buffer.CompositeByteBuf;
    +import io.netty.channel.ChannelHandlerContext;
    +import io.netty.handler.codec.MessageToMessageEncoder;
    +
    +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 wrap the input Composite ByteBuf components separately and 
append the encrypted length for each
    + * component in the output ByteBuf. If there are multiple components in 
the input ByteBuf then each component will be
    + * encrypted individually and added to output ByteBuf with it's length 
prepended.
    + * <p>
    + * Example:
    + * <li>Input ByteBuf  --> [B1,B2] - 2 component ByteBuf of 16K byte each.
    + * <li>Output ByteBuf --> [[EBLN1, EB1], [EBLN2, EB2]] - List of ByteBuf's 
with each ByteBuf containing
    + *                    Encrypted Byte Length (EBLNx) in network order as 
per SASL RFC and Encrypted Bytes (EBx).
    + * </p>
    + */
    +class SaslEncryptionHandler extends MessageToMessageEncoder<ByteBuf> {
    +
    +  private static final org.slf4j.Logger logger = 
org.slf4j.LoggerFactory.getLogger(
    +      SaslEncryptionHandler.class.getCanonicalName());
    +
    +  private final SaslCodec saslCodec;
    +
    +  private final int maxRawWrapSize;
    +
    +  private byte[] origMsgBuffer;
    +
    +  private final ByteBuffer lengthOctets;
    +
    +  private final OutOfMemoryHandler outOfMemoryHandler;
    +
    +  /**
    +   * We don't provide preference to allocator to use heap buffer instead 
of direct buffer.
    +   * Drill uses it's own buffer allocator which doesn't support heap 
buffer allocation. We use
    +   * Drill buffer allocator in the channel.
    +   */
    +  SaslEncryptionHandler(SaslCodec saslCodec, final int maxRawWrapSize, 
final OutOfMemoryHandler oomHandler) {
    +    this.saslCodec = saslCodec;
    +    this.maxRawWrapSize = maxRawWrapSize;
    +    this.outOfMemoryHandler = oomHandler;
    +
    +    // The maximum size of the component will be maxRawWrapSize. Since 
this is maximum size we can allocate once
    +    // and reuse it for each component encode.
    +    origMsgBuffer = new byte[this.maxRawWrapSize];
    +    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_ENCRYPTION_HANDLER + " 
handler!");
    +  }
    +
    +  @Override
    +  public void handlerRemoved(ChannelHandlerContext ctx) throws Exception {
    +    super.handlerRemoved(ctx);
    +    logger.trace("Removed " + RpcConstants.SASL_ENCRYPTION_HANDLER + " 
handler");
    +  }
    +
    +  public void encode(ChannelHandlerContext ctx, ByteBuf msg, List<Object> 
out) throws IOException {
    +
    +    if (!ctx.channel().isOpen()) {
    +      logger.debug("In " + RpcConstants.SASL_ENCRYPTION_HANDLER + " and 
channel is not open. " +
    +          "So releasing msg memory before encryption.");
    +      msg.release();
    +      return;
    +    }
    +
    +    try {
    +      // If encryption is enabled then this handler will always get 
ByteBuf of type Composite ByteBuf
    +      checkArgument(msg instanceof CompositeByteBuf);
    +
    +      final CompositeByteBuf cbb = (CompositeByteBuf) msg;
    +      int numComponents = cbb.numComponents();
    +      int currentIndex = 0;
    +      byte[] origMsg;
    +      ByteBuf encryptedBuf;
    +      byte[] wrappedMsg;
    +
    +      // Get all the components inside the Composite ByteBuf for encryption
    +      while (numComponents > 0) {
    +        ByteBuf component = cbb.component(currentIndex++);
    +
    +        // Each component ByteBuf size should not be greater than 
maxRawWrapSize since ChunkCreationHandler
    +        // will break the RPC message into chunks of maxRawWrapSize.
    +        if (component.readableBytes() > maxRawWrapSize) {
    +          throw new RpcException(String.format("Component Chunk size: %d 
is greater than the maxRawWrapSize: %d",
    +              component.readableBytes(), maxRawWrapSize));
    +        }
    +
    +        if (component.hasArray()) {
    --- End diff --
    
    same comment as in decrypt handler


> 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)

Reply via email to