[jira] [Commented] (DRILL-4335) Apache Drill should support network encryption

2017-04-25 Thread ASF GitHub Bot (JIRA)

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

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_r113375142
  
--- Diff: 
exec/rpc/src/main/java/org/apache/drill/exec/rpc/AbstractRemoteConnection.java 
---
@@ -224,4 +240,104 @@ public void close() {
 }
   }
 
+  /**
+   * Helps to add all the required security handler's after negotiation 
for encryption is completed.
+   * Handler's before encryption is negotiated are:
+   * 
+   *  PROTOCOL_DECODER {@link ProtobufLengthDecoder} 
+   *  MESSAGE_DECODER {@link RpcDecoder}  
+   *  PROTOCOL_ENCODER {@link RpcEncoder} 
+   *  HANDSHAKE_HANDLER {@link 
org.apache.drill.exec.rpc.BasicClient.ClientHandshakeHandler} OR
+   *{@link 
org.apache.drill.exec.rpc.BasicServer.ServerHandshakeHandler}  
+   *  optional - IDLE_STATE_HANDLER {@link 
org.apache.drill.exec.rpc.BasicClient.IdlePingHandler} OR
+   *   - TIMEOUT_HANDLER {@link 
org.apache.drill.exec.rpc.BasicServer.LoggingReadTimeoutHandler}  
+   *  MESSAGE_HANDLER {@link 
org.apache.drill.exec.rpc.RpcBus.InboundHandler} 
+   *  EXCEPTION_HANDLER {@link RpcExceptionHandler} 
+   * 
+   * Handler's after encryption is negotiated are:
+   * 
+   *  LENGTH_DECODER_HANDLER {@link LengthFieldBasedFrameDecoder}
+   *  SASL_DECRYPTION_HANDLER {@link SaslDecryptionHandler}
+   *  PROTOCOL_DECODER {@link ProtobufLengthDecoder}
+   *  MESSAGE_DECODER {@link RpcDecoder}
+   *  SASL_ENCRYPTION_HANDLER {@link SaslEncryptionHandler}
+   *  CHUNK_CREATION_HANDLER {@link ChunkCreationHandler}
+   *  PROTOCOL_ENCODER {@link RpcEncoder}
+   *  HANDSHAKE_HANDLER {@link 
org.apache.drill.exec.rpc.BasicClient.ClientHandshakeHandler} OR
+   *{@link 
org.apache.drill.exec.rpc.BasicServer.ServerHandshakeHandler}
+   *  optional - IDLE_STATE_HANDLER {@link 
org.apache.drill.exec.rpc.BasicClient.IdlePingHandler} OR
+   *   - TIMEOUT_HANDLER {@link 
org.apache.drill.exec.rpc.BasicServer.LoggingReadTimeoutHandler}
+   *  MESSAGE_HANDLER {@link 
org.apache.drill.exec.rpc.RpcBus.InboundHandler}
+   *  EXCEPTION_HANDLER {@link RpcExceptionHandler}
+   * 
+   * 
+   *  If encryption is enabled ChunkCreationHandler is always added to 
divide the Rpc message into chunks of
--- End diff --

This is required since Encoded Message size cannot be bigger than 16MB, 
this handler will take care of that. Keeping a separate handler for chunking 
purpose also helps to keep the encryption and chunking logic separate. As far 
as performance degradation is concerned I don't think there should be any 
notable difference. 


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


[jira] [Commented] (DRILL-4335) Apache Drill should support network encryption

2017-04-25 Thread ASF GitHub Bot (JIRA)

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

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_r113348583
  
--- 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.
+ * 
+ * Example:
+ * Input ByteBuf  --> [B1,B2] - 2 component ByteBuf of 16K byte each.
+ * 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).
+ * 
+ */
+class SaslEncryptionHandler extends MessageToMessageEncoder {
+
+  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 
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 

[jira] [Commented] (DRILL-4335) Apache Drill should support network encryption

2017-04-25 Thread ASF GitHub Bot (JIRA)

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

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_r113324923
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/rpc/security/ServerAuthenticationHandler.java
 ---
@@ -251,25 +255,67 @@ void process(SaslResponseContext context) 
throws Exception {
   private static , T extends EnumLite>
   void handleSuccess(final SaslResponseContext context, final 
SaslMessage.Builder challenge,
  final SaslServer saslServer) throws IOException {
-context.connection.changeHandlerTo(context.requestHandler);
-context.connection.finalizeSaslSession();
-context.sender.send(new Response(context.saslResponseType, 
challenge.build()));
 
-// setup security layers here..
+final S connection = context.connection;
+connection.changeHandlerTo(context.requestHandler);
+connection.finalizeSaslSession();
+
+// Check the negotiated property before sending the response back to 
client
+try {
+  final String negotiatedQOP = 
saslServer.getNegotiatedProperty(Sasl.QOP).toString();
--- End diff --

Could `.toString()` throw NPE, since null QOP is same as "auth"?


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


[jira] [Commented] (DRILL-4335) Apache Drill should support network encryption

2017-04-25 Thread ASF GitHub Bot (JIRA)

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

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_r113324280
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/rpc/security/ServerAuthenticationHandler.java
 ---
@@ -251,25 +255,67 @@ void process(SaslResponseContext context) 
throws Exception {
   private static , T extends EnumLite>
   void handleSuccess(final SaslResponseContext context, final 
SaslMessage.Builder challenge,
  final SaslServer saslServer) throws IOException {
-context.connection.changeHandlerTo(context.requestHandler);
-context.connection.finalizeSaslSession();
-context.sender.send(new Response(context.saslResponseType, 
challenge.build()));
 
-// setup security layers here..
+final S connection = context.connection;
+connection.changeHandlerTo(context.requestHandler);
+connection.finalizeSaslSession();
+
+// Check the negotiated property before sending the response back to 
client
+try {
+  final String negotiatedQOP = 
saslServer.getNegotiatedProperty(Sasl.QOP).toString();
+  final String expectedQOP = (connection.isEncryptionEnabled())
+  ? SaslProperties.QualityOfProtection.PRIVACY.getSaslQop()
+  : SaslProperties.QualityOfProtection.AUTHENTICATION.getSaslQop();
+
+  if (!(negotiatedQOP.equals(expectedQOP))) {
+throw new SaslException(String.format("Mismatch in negotiated QOP 
value: %s and Expected QOP value: %s",
+negotiatedQOP, expectedQOP));
+  }
+
+  // Update the rawWrapSendSize with the negotiated rawSendSize since 
we cannot call encode with more than the
+  // negotiated size of buffer
+  if (connection.isEncryptionEnabled()) {
+final int negotiatedRawSendSize = Integer.parseInt(
+
saslServer.getNegotiatedProperty(Sasl.RAW_SEND_SIZE).toString());
+if (negotiatedRawSendSize <= 0) {
+  throw new SaslException(String.format("Negotiated rawSendSize: 
%d is invalid. Please check the configured " +
+  "value of encryption.sasl.max_wrapped_size. It might be 
configured to a very small value.",
+  negotiatedRawSendSize));
+}
+connection.setWrapSizeLimit(negotiatedRawSendSize);
--- End diff --

On the server side, there is only one EncryptionContext per connection 
type. This will have unintended consequences when there are multiple 
connections!

Since rawSendSize is negotiated, EncryptionContext objects need to live and 
die with a connection. Correct?


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


[jira] [Commented] (DRILL-4335) Apache Drill should support network encryption

2017-04-25 Thread ASF GitHub Bot (JIRA)

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

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_r112789119
  
--- Diff: 
exec/rpc/src/main/java/org/apache/drill/exec/rpc/AbstractRemoteConnection.java 
---
@@ -224,4 +240,104 @@ public void close() {
 }
   }
 
+  /**
+   * Helps to add all the required security handler's after negotiation 
for encryption is completed.
+   * Handler's before encryption is negotiated are:
+   * 
+   *  PROTOCOL_DECODER {@link ProtobufLengthDecoder} 
+   *  MESSAGE_DECODER {@link RpcDecoder}  
+   *  PROTOCOL_ENCODER {@link RpcEncoder} 
+   *  HANDSHAKE_HANDLER {@link 
org.apache.drill.exec.rpc.BasicClient.ClientHandshakeHandler} OR
+   *{@link 
org.apache.drill.exec.rpc.BasicServer.ServerHandshakeHandler}  
+   *  optional - IDLE_STATE_HANDLER {@link 
org.apache.drill.exec.rpc.BasicClient.IdlePingHandler} OR
+   *   - TIMEOUT_HANDLER {@link 
org.apache.drill.exec.rpc.BasicServer.LoggingReadTimeoutHandler}  
+   *  MESSAGE_HANDLER {@link 
org.apache.drill.exec.rpc.RpcBus.InboundHandler} 
+   *  EXCEPTION_HANDLER {@link RpcExceptionHandler} 
+   * 
+   * Handler's after encryption is negotiated are:
+   * 
+   *  LENGTH_DECODER_HANDLER {@link LengthFieldBasedFrameDecoder}
+   *  SASL_DECRYPTION_HANDLER {@link SaslDecryptionHandler}
+   *  PROTOCOL_DECODER {@link ProtobufLengthDecoder}
+   *  MESSAGE_DECODER {@link RpcDecoder}
+   *  SASL_ENCRYPTION_HANDLER {@link SaslEncryptionHandler}
+   *  CHUNK_CREATION_HANDLER {@link ChunkCreationHandler}
+   *  PROTOCOL_ENCODER {@link RpcEncoder}
+   *  HANDSHAKE_HANDLER {@link 
org.apache.drill.exec.rpc.BasicClient.ClientHandshakeHandler} OR
+   *{@link 
org.apache.drill.exec.rpc.BasicServer.ServerHandshakeHandler}
+   *  optional - IDLE_STATE_HANDLER {@link 
org.apache.drill.exec.rpc.BasicClient.IdlePingHandler} OR
+   *   - TIMEOUT_HANDLER {@link 
org.apache.drill.exec.rpc.BasicServer.LoggingReadTimeoutHandler}
+   *  MESSAGE_HANDLER {@link 
org.apache.drill.exec.rpc.RpcBus.InboundHandler}
+   *  EXCEPTION_HANDLER {@link RpcExceptionHandler}
+   * 
+   * 
+   *  If encryption is enabled ChunkCreationHandler is always added to 
divide the Rpc message into chunks of
--- End diff --

But does always adding the ChunckCreationHandler have performance 
implications?


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


[jira] [Commented] (DRILL-4335) Apache Drill should support network encryption

2017-04-25 Thread ASF GitHub Bot (JIRA)

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

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_r113349330
  
--- 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.
+ * 
+ * Example:
+ * Input ByteBuf  --> [B1,B2] - 2 component ByteBuf of 16K byte each.
+ * 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).
+ * 
+ */
+class SaslEncryptionHandler extends MessageToMessageEncoder {
+
+  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 
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 

[jira] [Commented] (DRILL-4335) Apache Drill should support network encryption

2017-04-25 Thread ASF GitHub Bot (JIRA)

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

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_r113347217
  
--- 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.
+ * 
+ * Example:
+ * Input ByteBuf  --> [B1,B2] - 2 component ByteBuf of 16K byte each.
+ * 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).
+ * 
+ */
+class SaslEncryptionHandler extends MessageToMessageEncoder {
+
+  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 
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 

[jira] [Commented] (DRILL-4335) Apache Drill should support network encryption

2017-04-25 Thread ASF GitHub Bot (JIRA)

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

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_r113351684
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/rpc/security/AuthenticationOutcomeListener.java
 ---
@@ -243,4 +249,46 @@ public SaslMessage process(SaslChallengeContext 
context) throws Exception {
   }
 }
   }
+
+  private static void handleSuccess(SaslChallengeContext context) throws 
SaslException {
+final ClientConnection connection = context.connection;
+final SaslClient saslClient = connection.getSaslClient();
+
+try {
+  // Check if connection was marked for being secure then verify for 
negotiated QOP value for
+  // correctness.
+  final String negotiatedQOP = 
saslClient.getNegotiatedProperty(Sasl.QOP).toString();
+  final String expectedQOP = connection.isEncryptionEnabled()
+  ? SaslProperties.QualityOfProtection.PRIVACY.getSaslQop()
+  : SaslProperties.QualityOfProtection.AUTHENTICATION.getSaslQop();
+
+  if (!(negotiatedQOP.equals(expectedQOP))) {
+throw new SaslException(String.format("Mismatch in negotiated QOP 
value: %s and Expected QOP value: %s",
+negotiatedQOP, expectedQOP));
+  }
+
+  // Update the rawWrapChunkSize with the negotiated buffer size since 
we cannot call encode with more than
+  // negotiated size of buffer.
+  if (connection.isEncryptionEnabled()) {
+final int negotiatedRawSendSize = Integer.parseInt(
+
saslClient.getNegotiatedProperty(Sasl.RAW_SEND_SIZE).toString());
+if (negotiatedRawSendSize <= 0) {
+  throw new SaslException(String.format("Negotiated rawSendSize: 
%d is invalid. Please check the configured " +
+  "value of sasl.encryption.encodesize. It might be configured 
to a very small value.",
+  negotiatedRawSendSize));
+}
+connection.setWrapSizeLimit(negotiatedRawSendSize);
--- End diff --

Please fix naming (or maybe I am easily confused).

wrapSizeLimit and negotiatedRawSendSize, contrast with maxRawWrapSize in 
SaslEncryptionHandler. All of which are actually `maxSendBufferSize` per 
documentation in Sasl.RAW_SEND_SIZE.


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


[jira] [Commented] (DRILL-4335) Apache Drill should support network encryption

2017-04-25 Thread ASF GitHub Bot (JIRA)

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

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_r113340308
  
--- Diff: 
exec/rpc/src/main/java/org/apache/drill/exec/rpc/EncryptionContext.java ---
@@ -0,0 +1,37 @@
+/*
+ * 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;
+
+public interface EncryptionContext {
+
+  boolean isEncryptionEnabled();
+
+  void setEncryption(boolean encryptionEnabled);
+
+  void setMaxWrappedSize(int maxWrappedChunkSize);
+
+  int getMaxWrappedSize();
+
+  void setWrapSizeLimit(int wrapSizeLimit);
--- End diff --

+ How about `setMaxSendBufferSize(int size)` as documented in 
`Sasl.RAW_SEND_SIZE`?
+ Add doc pointing to `Sasl.RAW_SEND_SIZE`


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


[jira] [Commented] (DRILL-4335) Apache Drill should support network encryption

2017-04-25 Thread ASF GitHub Bot (JIRA)

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

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_r113346921
  
--- 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.
+ * 
+ * Example:
+ * Input ByteBuf  --> [B1,B2] - 2 component ByteBuf of 16K byte each.
+ * 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).
+ * 
+ */
+class SaslEncryptionHandler extends MessageToMessageEncoder {
+
+  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];
--- End diff --

As mentioned above, this increases memory consumption of idle connections. 
Any measure perf benefits?


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


[jira] [Commented] (DRILL-4335) Apache Drill should support network encryption

2017-04-25 Thread ASF GitHub Bot (JIRA)

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

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_r112788779
  
--- Diff: 
exec/rpc/src/main/java/org/apache/drill/exec/rpc/RemoteConnection.java ---
@@ -51,6 +51,12 @@
 
   SocketAddress getRemoteAddress();
 
+  void addSecurityHandlers();
+
+  void incConnectionCounter();
--- End diff --

Remove this increment and decrement counters from the interface; these 
methods are invoked on implementations. 

Maybe empty protected methods in `AbstractRemoteConnection` with impls in 
concrete classes?


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


[jira] [Commented] (DRILL-4335) Apache Drill should support network encryption

2017-04-25 Thread ASF GitHub Bot (JIRA)

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

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_r113347107
  
--- 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.
+ * 
+ * Example:
+ * Input ByteBuf  --> [B1,B2] - 2 component ByteBuf of 16K byte each.
+ * 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).
+ * 
+ */
+class SaslEncryptionHandler extends MessageToMessageEncoder {
+
+  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 
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 

[jira] [Commented] (DRILL-4335) Apache Drill should support network encryption

2017-04-25 Thread ASF GitHub Bot (JIRA)

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

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_r113353149
  
--- Diff: 
exec/rpc/src/main/java/org/apache/drill/exec/rpc/RpcConstants.java ---
@@ -24,4 +24,29 @@ private RpcConstants(){}
 
   public static final boolean SOME_DEBUGGING = false;
   public static final boolean EXTRA_DEBUGGING = false;
+
+  // RPC Handler names
+  public static final String TIMEOUT_HANDLER = "timeout-handler";
+  public static final String PROTOCOL_DECODER = "protocol-decoder";
+  public static final String PROTOCOL_ENCODER = "protocol-encoder";
+  public static final String MESSAGE_DECODER = "message-decoder";
+  public static final String HANDSHAKE_HANDLER = "handshake-handler";
+  public static final String MESSAGE_HANDLER = "message-handler";
+  public static final String EXCEPTION_HANDLER = "exception-handler";
+  public static final String IDLE_STATE_HANDLER = "idle-state-handler";
+  public static final String SASL_DECRYPTION_HANDLER = 
"sasldecryption-handler";
--- End diff --

`sasl-decryption-handler`
`sasl-encryption-handler`
`chunk-creation-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)


[jira] [Commented] (DRILL-4335) Apache Drill should support network encryption

2017-04-25 Thread ASF GitHub Bot (JIRA)

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

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_r113353494
  
--- 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.
+ * 
+ * Example:
+ * Input - [EBLN1, EB1, EBLN2, EB2] --> ByteBuf with repeated 
combination of encrypted byte length
+ * in network order (EBLNx) and encrypted bytes (EB)
+ * Output - [DB1] --> Decrypted ByteBuf of first chunk.(EB1)
+ * 
+ */
+class SaslDecryptionHandler extends MessageToMessageDecoder {
+
+  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. OXFF)
+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 
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.
+

[jira] [Commented] (DRILL-4335) Apache Drill should support network encryption

2017-04-25 Thread ASF GitHub Bot (JIRA)

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

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_r113323104
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/ControllerImpl.java
 ---
@@ -24,13 +24,15 @@
 import org.apache.drill.exec.exception.DrillbitStartupException;
 import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.rpc.RpcMetrics;
 import org.apache.drill.exec.server.BootStrapContext;
 import org.apache.drill.exec.work.batch.ControlMessageHandler;
 
 import com.google.common.collect.Lists;
 import com.google.protobuf.Message;
 import com.google.protobuf.MessageLite;
 import com.google.protobuf.Parser;
+import org.apache.hadoop.io.ReadaheadPool;
--- End diff --

??


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


[jira] [Commented] (DRILL-4335) Apache Drill should support network encryption

2017-04-25 Thread ASF GitHub Bot (JIRA)

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

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_r113329031
  
--- Diff: 
exec/rpc/src/main/java/org/apache/drill/exec/rpc/EncryptionContext.java ---
@@ -0,0 +1,37 @@
+/*
+ * 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;
+
+public interface EncryptionContext {
+
+  boolean isEncryptionEnabled();
+
+  void setEncryption(boolean encryptionEnabled);
+
+  void setMaxWrappedSize(int maxWrappedChunkSize);
+
+  int getMaxWrappedSize();
+
+  void setWrapSizeLimit(int wrapSizeLimit);
+
+  int getWrapSizeLimit();
+
+  String getEncryptionCtxtString();
--- End diff --

Unnecessary since implementation can override `toString`?


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


[jira] [Commented] (DRILL-4335) Apache Drill should support network encryption

2017-04-25 Thread ASF GitHub Bot (JIRA)

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

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_r113245537
  
--- Diff: 
exec/rpc/src/main/java/org/apache/drill/exec/rpc/EncryptionContext.java ---
@@ -0,0 +1,37 @@
+/*
+ * 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;
+
+public interface EncryptionContext {
--- End diff --

Maybe rename to `EncryptionOptions` or `EncryptionSettings`? This also fits 
well with two implementations based on lifecycle: configured encryption options 
(immutable, defined in config classes) and connection encryption options 
(negotiated).

And instead of connection interface implementing this interface, maybe have 
a `getEncryptionOptions` in the `RemoteConnection` interface. So the usage 
would be: `connection.getEncryptionOptions().isEnabled()` or 
`connection.getEncryptionOptions().getMaxWrappedSize()`.

This is a variant of what you had previously. Sorry for the back and forth 
on this.


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


[jira] [Commented] (DRILL-4335) Apache Drill should support network encryption

2017-04-25 Thread ASF GitHub Bot (JIRA)

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

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_r113334641
  
--- 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.
+ * 
+ * Example:
+ * Input - [EBLN1, EB1, EBLN2, EB2] --> ByteBuf with repeated 
combination of encrypted byte length
+ * in network order (EBLNx) and encrypted bytes (EB)
+ * Output - [DB1] --> Decrypted ByteBuf of first chunk.(EB1)
+ * 
+ */
+class SaslDecryptionHandler extends MessageToMessageDecoder {
+
+  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. OXFF)
+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 
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.
+

[jira] [Commented] (DRILL-4335) Apache Drill should support network encryption

2017-04-25 Thread ASF GitHub Bot (JIRA)

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

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_r113337341
  
--- 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.
+ * 
+ * Example:
+ * Input - [EBLN1, EB1, EBLN2, EB2] --> ByteBuf with repeated 
combination of encrypted byte length
+ * in network order (EBLNx) and encrypted bytes (EB)
+ * Output - [DB1] --> Decrypted ByteBuf of first chunk.(EB1)
+ * 
+ */
+class SaslDecryptionHandler extends MessageToMessageDecoder {
+
+  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. OXFF)
+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 
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.
+

[jira] [Commented] (DRILL-4335) Apache Drill should support network encryption

2017-04-25 Thread ASF GitHub Bot (JIRA)

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

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_r113334954
  
--- 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.
+ * 
+ * Example:
+ * Input - [EBLN1, EB1, EBLN2, EB2] --> ByteBuf with repeated 
combination of encrypted byte length
+ * in network order (EBLNx) and encrypted bytes (EB)
+ * Output - [DB1] --> Decrypted ByteBuf of first chunk.(EB1)
+ * 
+ */
+class SaslDecryptionHandler extends MessageToMessageDecoder {
+
+  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. OXFF)
+encodedMsg = new byte[maxEncodedSize];
--- End diff --

Given this, every connection (idle or not) consumes at least 16 MB. Are 
there any measured perf benefits? If not, please remove this.


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


[jira] [Commented] (DRILL-4335) Apache Drill should support network encryption

2017-04-25 Thread ASF GitHub Bot (JIRA)

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

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_r113346069
  
--- 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.
+ * 
+ * Example:
+ * Input - [EBLN1, EB1, EBLN2, EB2] --> ByteBuf with repeated 
combination of encrypted byte length
+ * in network order (EBLNx) and encrypted bytes (EB)
+ * Output - [DB1] --> Decrypted ByteBuf of first chunk.(EB1)
+ * 
+ */
+class SaslDecryptionHandler extends MessageToMessageDecoder {
+
+  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. OXFF)
+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 
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.
+

[jira] [Commented] (DRILL-4335) Apache Drill should support network encryption

2017-04-25 Thread ASF GitHub Bot (JIRA)

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

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_r113260837
  
--- Diff: 
exec/rpc/src/main/java/org/apache/drill/exec/rpc/ChunkCreationHandler.java ---
@@ -0,0 +1,101 @@
+/*
+ * 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 java.util.List;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static java.lang.Math.min;
+
+/**
+ * Handler that converts an input ByteBuf into chunk size ByteBuf's and 
add it to the
+ * CompositeByteBuf as individual components. This is done irrespective of 
chunk mode is
+ * enabled or not.
+ */
+class ChunkCreationHandler extends MessageToMessageEncoder {
+  private static final org.slf4j.Logger logger = 
org.slf4j.LoggerFactory.getLogger(
+  ChunkCreationHandler.class.getCanonicalName());
+
+  private final int chunkSize;
+
+  ChunkCreationHandler(int chunkSize) {
+checkArgument(chunkSize > 0);
+this.chunkSize = chunkSize;
+  }
+
+  @Override
+  public void handlerAdded(ChannelHandlerContext ctx) throws Exception {
+super.handlerAdded(ctx);
+logger.trace("Added " + RpcConstants.CHUNK_CREATION_HANDLER + " 
handler!");
+  }
+
+  @Override
+  public void handlerRemoved(ChannelHandlerContext ctx) throws Exception {
+super.handlerRemoved(ctx);
+logger.trace("Removed " + RpcConstants.CHUNK_CREATION_HANDLER + " 
handler");
+  }
+
+  @Override
+  protected void encode(ChannelHandlerContext ctx, ByteBuf msg, 
List out) throws Exception {
+
+if (RpcConstants.EXTRA_DEBUGGING) {
+  logger.debug("ChunkCreationHandler called with msg {} of size {} 
with chunkSize {}",
+  msg, msg.readableBytes(), chunkSize);
+}
+
+if (!ctx.channel().isOpen()) {
+  logger.debug("Channel closed, skipping encode inside {}.", 
RpcConstants.CHUNK_CREATION_HANDLER);
+  msg.release();
+  return;
+}
+
+ByteBuf chunkBuf;
+
+// Calculate the number of chunks based on configured chunk size and 
input msg size
+int numChunks = (int) Math.ceil((double) msg.readableBytes() / 
chunkSize);
+
+// Initialize a composite buffer to hold numChunks chunk.
+CompositeByteBuf cbb = new CompositeByteBuf(ctx.alloc(), true, 
numChunks);
--- End diff --

+ Use `ctx.alloc().compositeDirectBuffer(...)`?
+ final


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


[jira] [Commented] (DRILL-4335) Apache Drill should support network encryption

2017-04-25 Thread ASF GitHub Bot (JIRA)

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

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_r113346292
  
--- 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.
+ * 
+ * Example:
+ * Input - [EBLN1, EB1, EBLN2, EB2] --> ByteBuf with repeated 
combination of encrypted byte length
+ * in network order (EBLNx) and encrypted bytes (EB)
+ * Output - [DB1] --> Decrypted ByteBuf of first chunk.(EB1)
+ * 
+ */
+class SaslDecryptionHandler extends MessageToMessageDecoder {
+
+  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. OXFF)
+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 
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.
+

[jira] [Commented] (DRILL-4335) Apache Drill should support network encryption

2017-04-25 Thread ASF GitHub Bot (JIRA)

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

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_r113348380
  
--- 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.
+ * 
+ * Example:
+ * Input ByteBuf  --> [B1,B2] - 2 component ByteBuf of 16K byte each.
+ * 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).
+ * 
+ */
+class SaslEncryptionHandler extends MessageToMessageEncoder {
+
+  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 
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 

[jira] [Commented] (DRILL-4335) Apache Drill should support network encryption

2017-04-25 Thread ASF GitHub Bot (JIRA)

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

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_r113331955
  
--- Diff: exec/rpc/src/main/java/org/apache/drill/exec/rpc/RpcMetrics.java 
---
@@ -0,0 +1,33 @@
+/*
+ * 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 org.apache.drill.exec.memory.BufferAllocator;
+
+/**
+ * Holder interface for all the metrics used in RPC layer
+ */
+public interface RpcMetrics {
--- End diff --

This interface, its hierarchy, and all the plumbing in the server and 
client classes seem unnecessary. Drill only needs one MetricsRegistry (held in 
DrillMetrics), and metrics can be directly modified through that class.

Maybe all rpc metrics can be in one utility class in the java-exec...rpc 
package?


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


[jira] [Commented] (DRILL-4335) Apache Drill should support network encryption

2017-04-25 Thread ASF GitHub Bot (JIRA)

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

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_r113336202
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/rpc/security/AuthenticationOutcomeListener.java
 ---
@@ -143,16 +149,16 @@ public void interrupted(InterruptedException e) {
 completionListener.interrupted(e);
   }
 
-  private static class SaslChallengeContext {
+  private static class SaslChallengeContext {
--- End diff --

Fix raw usage warnings due to this change


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


[jira] [Commented] (DRILL-4335) Apache Drill should support network encryption

2017-04-25 Thread ASF GitHub Bot (JIRA)

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

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_r113353207
  
--- Diff: 
exec/rpc/src/main/java/org/apache/drill/exec/rpc/RpcConstants.java ---
@@ -24,4 +24,29 @@ private RpcConstants(){}
 
   public static final boolean SOME_DEBUGGING = false;
   public static final boolean EXTRA_DEBUGGING = false;
+
+  // RPC Handler names
+  public static final String TIMEOUT_HANDLER = "timeout-handler";
+  public static final String PROTOCOL_DECODER = "protocol-decoder";
+  public static final String PROTOCOL_ENCODER = "protocol-encoder";
+  public static final String MESSAGE_DECODER = "message-decoder";
+  public static final String HANDSHAKE_HANDLER = "handshake-handler";
+  public static final String MESSAGE_HANDLER = "message-handler";
+  public static final String EXCEPTION_HANDLER = "exception-handler";
+  public static final String IDLE_STATE_HANDLER = "idle-state-handler";
+  public static final String SASL_DECRYPTION_HANDLER = 
"sasldecryption-handler";
+  public static final String SASL_ENCRYPTION_HANDLER = 
"saslencryption-handler";
+  public static final String LENGTH_DECODER_HANDLER = "length-decoder";
+  public static final String CHUNK_CREATION_HANDLER = 
"chunkcreation-handler";
+
+
+
+  // SASL RFC /4422 allows only 3 octets to specify the length of 
maximum encoded buffer each side can receive.
+  // Hence the maximum buffer size is capped at 16Mb i.e. 0XFF bytes.
+  public static final int MAX_WRAPPED_SIZE = 0XFF;
+
+  public static final int LENGTH_FIELD_OFFSET = 0;
+  public static final int LENGTH_FIELD_LENGTH = 4;
--- End diff --

Better name for `LENGTH_FIELD_LENGTH`?


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


[jira] [Commented] (DRILL-4335) Apache Drill should support network encryption

2017-04-25 Thread ASF GitHub Bot (JIRA)

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

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_r113340307
  
--- Diff: 
exec/rpc/src/main/java/org/apache/drill/exec/rpc/EncryptionContext.java ---
@@ -0,0 +1,37 @@
+/*
+ * 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;
+
+public interface EncryptionContext {
+
+  boolean isEncryptionEnabled();
+
+  void setEncryption(boolean encryptionEnabled);
+
+  void setMaxWrappedSize(int maxWrappedChunkSize);
--- End diff --

+ How about `setMaxReceiveBufferSize(int size)` as documented in 
`Sasl.MAX_BUFFER`?
+ Add doc pointing to `Sasl.MAX_BUFFER`


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


[jira] [Commented] (DRILL-4335) Apache Drill should support network encryption

2017-04-25 Thread ASF GitHub Bot (JIRA)

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

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_r113348909
  
--- 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.
+ * 
+ * Example:
+ * Input ByteBuf  --> [B1,B2] - 2 component ByteBuf of 16K byte each.
+ * 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).
+ * 
+ */
+class SaslEncryptionHandler extends MessageToMessageEncoder {
+
+  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 
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 

[jira] [Commented] (DRILL-4335) Apache Drill should support network encryption

2017-04-25 Thread ASF GitHub Bot (JIRA)

[ 
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.
+ * 
+ * Example:
+ * Input ByteBuf  --> [B1,B2] - 2 component ByteBuf of 16K byte each.
+ * 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).
+ * 
+ */
+class SaslEncryptionHandler extends MessageToMessageEncoder {
+
+  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 
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 

[jira] [Commented] (DRILL-4335) Apache Drill should support network encryption

2017-04-25 Thread ASF GitHub Bot (JIRA)

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

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_r113350599
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/rpc/security/AuthenticationOutcomeListener.java
 ---
@@ -243,4 +249,46 @@ public SaslMessage process(SaslChallengeContext 
context) throws Exception {
   }
 }
   }
+
+  private static void handleSuccess(SaslChallengeContext context) throws 
SaslException {
+final ClientConnection connection = context.connection;
+final SaslClient saslClient = connection.getSaslClient();
+
+try {
+  // Check if connection was marked for being secure then verify for 
negotiated QOP value for
+  // correctness.
+  final String negotiatedQOP = 
saslClient.getNegotiatedProperty(Sasl.QOP).toString();
--- End diff --

Could .toString() throw NPE, since null QOP is same as "auth"?


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


[jira] [Commented] (DRILL-4335) Apache Drill should support network encryption

2017-04-25 Thread ASF GitHub Bot (JIRA)

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

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_r113328102
  
--- Diff: 
exec/rpc/src/main/java/org/apache/drill/exec/rpc/ChunkCreationHandler.java ---
@@ -0,0 +1,101 @@
+/*
+ * 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 java.util.List;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static java.lang.Math.min;
+
+/**
+ * Handler that converts an input ByteBuf into chunk size ByteBuf's and 
add it to the
+ * CompositeByteBuf as individual components. This is done irrespective of 
chunk mode is
--- End diff --

... **If encryption is enabled, this is added** irrespective of ...


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


[jira] [Commented] (DRILL-4335) Apache Drill should support network encryption

2017-04-25 Thread Sorabh Hamirwasia (JIRA)

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

Sorabh Hamirwasia commented on DRILL-4335:
--

Thanks!. I will add you and other reviewer's in the pull request for C++ 
changes.

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


[jira] [Commented] (DRILL-5445) Assertion Error in Managed External Sort when dealing with repeated maps

2017-04-25 Thread Rahul Challapalli (JIRA)

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

Rahul Challapalli commented on DRILL-5445:
--

Hmm...when I tried for a simpler repro, I ended up getting the below error
{code}
select * from dfs.`/drill/testdata/resource-manager/nested-large.json` d order 
by d.features[0].type;
Error: SYSTEM ERROR: UnsupportedOperationException: Unable to get size for 
minor type [LIST] and mode [REPEATED]

Fragment 0:0

[Error Id: 563ec1ba-2e53-461c-ae29-b1e35807bfdd on qa-node190.qa.lab:31010] 
(state=,code=0)
{code}

Exception from the logs
{code}
Error Id: 563ec1ba-2e53-461c-ae29-b1e35807bfdd on qa-node190.qa.lab:31010]
at 
org.apache.drill.common.exceptions.UserException$Builder.build(UserException.java:544)
 ~[drill-common-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
at 
org.apache.drill.exec.work.fragment.FragmentExecutor.sendFinalState(FragmentExecutor.java:295)
 [drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
at 
org.apache.drill.exec.work.fragment.FragmentExecutor.cleanup(FragmentExecutor.java:160)
 [drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
at 
org.apache.drill.exec.work.fragment.FragmentExecutor.run(FragmentExecutor.java:264)
 [drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
at 
org.apache.drill.common.SelfCleaningRunnable.run(SelfCleaningRunnable.java:38) 
[drill-common-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) 
[na:1.7.0_111]
at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) 
[na:1.7.0_111]
at java.lang.Thread.run(Thread.java:745) [na:1.7.0_111]
Caused by: java.lang.UnsupportedOperationException: Unable to get size for 
minor type [LIST] and mode [REPEATED]
at 
org.apache.drill.exec.expr.BasicTypeHelper.getSize(BasicTypeHelper.java:165) 
~[vector-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
at 
org.apache.drill.exec.physical.impl.spill.RecordBatchSizer$ColumnSize.(RecordBatchSizer.java:74)
 ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
at 
org.apache.drill.exec.physical.impl.spill.RecordBatchSizer.measureColumn(RecordBatchSizer.java:224)
 ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
at 
org.apache.drill.exec.physical.impl.spill.RecordBatchSizer.measureColumn(RecordBatchSizer.java:212)
 ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
at 
org.apache.drill.exec.physical.impl.spill.RecordBatchSizer.(RecordBatchSizer.java:177)
 ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
at 
org.apache.drill.exec.physical.impl.spill.RecordBatchSizer.(RecordBatchSizer.java:165)
 ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
at 
org.apache.drill.exec.physical.impl.xsort.managed.ExternalSortBatch.analyzeIncomingBatch(ExternalSortBatch.java:930)
 ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
at 
org.apache.drill.exec.physical.impl.xsort.managed.ExternalSortBatch.processBatch(ExternalSortBatch.java:821)
 ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
at 
org.apache.drill.exec.physical.impl.xsort.managed.ExternalSortBatch.loadBatch(ExternalSortBatch.java:618)
 ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
at 
org.apache.drill.exec.physical.impl.xsort.managed.ExternalSortBatch.load(ExternalSortBatch.java:660)
 ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
at 
org.apache.drill.exec.physical.impl.xsort.managed.ExternalSortBatch.innerNext(ExternalSortBatch.java:559)
 ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
at 
org.apache.drill.exec.record.AbstractRecordBatch.next(AbstractRecordBatch.java:162)
 ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
at 
org.apache.drill.exec.physical.impl.validate.IteratorValidatorBatchIterator.next(IteratorValidatorBatchIterator.java:215)
 ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
at 
org.apache.drill.exec.record.AbstractRecordBatch.next(AbstractRecordBatch.java:119)
 ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
at 
org.apache.drill.exec.record.AbstractRecordBatch.next(AbstractRecordBatch.java:109)
 ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
at 
org.apache.drill.exec.record.AbstractSingleRecordBatch.innerNext(AbstractSingleRecordBatch.java:51)
 ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
at 
org.apache.drill.exec.physical.impl.svremover.RemovingRecordBatch.innerNext(RemovingRecordBatch.java:93)
 ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
at 
org.apache.drill.exec.record.AbstractRecordBatch.next(AbstractRecordBatch.java:162)
 ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
at 
org.apache.drill.exec.physical.impl.validate.IteratorValidatorBatchIterator.next(I

[jira] [Updated] (DRILL-5445) Assertion Error in Managed External Sort when dealing with repeated maps

2017-04-25 Thread Rahul Challapalli (JIRA)

 [ 
https://issues.apache.org/jira/browse/DRILL-5445?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Rahul Challapalli updated DRILL-5445:
-
Attachment: drillbit.log
27004a3c-c53d-52d1-c7ed-4beb563447f9.sys.drill

Most likely this can be reproduced using a simpler case. I will work towards 
that

> Assertion Error in Managed External Sort when dealing with repeated maps
> 
>
> Key: DRILL-5445
> URL: https://issues.apache.org/jira/browse/DRILL-5445
> Project: Apache Drill
>  Issue Type: Bug
>Affects Versions: 1.10.0
>Reporter: Rahul Challapalli
>Assignee: Paul Rogers
> Attachments: 27004a3c-c53d-52d1-c7ed-4beb563447f9.sys.drill, 
> drillbit.log
>
>
> git.commit.id.abbrev=3e8b01d
> The below query fails with an Assertion Error (I am running with assertions 
> enabled)
> {code}
> ALTER SESSION SET `exec.sort.disable_managed` = false;
> alter session set `planner.width.max_per_node` = 1;
> alter session set `planner.disable_exchanges` = true;
> alter session set `planner.width.max_per_query` = 1;
> alter session set `planner.memory.max_query_memory_per_node` = 152428800;
> select count(*) from (
> select * from (
> select event_info.uid, transaction_info.trans_id, event_info.event.evnt_id
> from (
>  select userinfo.transaction.trans_id trans_id, 
> max(userinfo.event.event_time) max_event_time
>  from (
>  select uid, flatten(events) event, flatten(transactions) transaction 
> from dfs.`/drill/testdata/resource-manager/nested-large.json`
>  ) userinfo
>  where userinfo.transaction.trans_time >= userinfo.event.event_time
>  group by userinfo.transaction.trans_id
> ) transaction_info
> inner join
> (
>  select uid, flatten(events) event
>  from dfs.`/drill/testdata/resource-manager/nested-large.json`
> ) event_info
> on transaction_info.max_event_time = event_info.event.event_time) d order by 
> features[0].type) d1 where d1.uid < -1;
> {code}
> Below is the error from the logs
> {code}
> [Error Id: 26983344-dee3-4a33-8508-ad125f01fee6 on qa-node190.qa.lab:31010]
> at 
> org.apache.drill.common.exceptions.UserException$Builder.build(UserException.java:544)
>  ~[drill-common-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
> at 
> org.apache.drill.exec.work.fragment.FragmentExecutor.sendFinalState(FragmentExecutor.java:295)
>  [drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
> at 
> org.apache.drill.exec.work.fragment.FragmentExecutor.cleanup(FragmentExecutor.java:160)
>  [drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
> at 
> org.apache.drill.exec.work.fragment.FragmentExecutor.run(FragmentExecutor.java:264)
>  [drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
> at 
> org.apache.drill.common.SelfCleaningRunnable.run(SelfCleaningRunnable.java:38)
>  [drill-common-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
> at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
>  [na:1.7.0_111]
> at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
>  [na:1.7.0_111]
> at java.lang.Thread.run(Thread.java:745) [na:1.7.0_111]
> Caused by: java.lang.RuntimeException: java.lang.AssertionError
> at 
> org.apache.drill.common.DeferredException.addThrowable(DeferredException.java:101)
>  ~[drill-common-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
> at 
> org.apache.drill.exec.work.fragment.FragmentExecutor.fail(FragmentExecutor.java:409)
>  [drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
> at 
> org.apache.drill.exec.work.fragment.FragmentExecutor.run(FragmentExecutor.java:250)
>  [drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
> ... 4 common frames omitted
> Caused by: java.lang.AssertionError: null
> at 
> org.apache.drill.exec.vector.complex.RepeatedMapVector.load(RepeatedMapVector.java:444)
>  ~[vector-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
> at 
> org.apache.drill.exec.cache.VectorAccessibleSerializable.readFromStream(VectorAccessibleSerializable.java:118)
>  ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
> at 
> org.apache.drill.exec.physical.impl.xsort.managed.BatchGroup$SpilledRun.getBatch(BatchGroup.java:222)
>  ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
> at 
> org.apache.drill.exec.physical.impl.xsort.managed.BatchGroup$SpilledRun.getNextIndex(BatchGroup.java:196)
>  ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
> at 
> org.apache.drill.exec.test.generated.PriorityQueueCopierGen23.setup(PriorityQueueCopierTemplate.java:60)
>  ~[na:na]
> at 
> org.apache.drill.exec.physical.impl.xsort.managed.CopierHolder.createCopier(CopierHolder.java:116)
>  ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
> at 
> org.apache.drill.exec.physical.impl.xso

[jira] [Created] (DRILL-5445) Assertion Error in Managed External Sort when dealing with repeated maps

2017-04-25 Thread Rahul Challapalli (JIRA)
Rahul Challapalli created DRILL-5445:


 Summary: Assertion Error in Managed External Sort when dealing 
with repeated maps
 Key: DRILL-5445
 URL: https://issues.apache.org/jira/browse/DRILL-5445
 Project: Apache Drill
  Issue Type: Bug
Affects Versions: 1.10.0
Reporter: Rahul Challapalli
Assignee: Paul Rogers


git.commit.id.abbrev=3e8b01d

The below query fails with an Assertion Error (I am running with assertions 
enabled)
{code}
ALTER SESSION SET `exec.sort.disable_managed` = false;
alter session set `planner.width.max_per_node` = 1;
alter session set `planner.disable_exchanges` = true;
alter session set `planner.width.max_per_query` = 1;
alter session set `planner.memory.max_query_memory_per_node` = 152428800;
select count(*) from (
select * from (
select event_info.uid, transaction_info.trans_id, event_info.event.evnt_id
from (
 select userinfo.transaction.trans_id trans_id, max(userinfo.event.event_time) 
max_event_time
 from (
 select uid, flatten(events) event, flatten(transactions) transaction from 
dfs.`/drill/testdata/resource-manager/nested-large.json`
 ) userinfo
 where userinfo.transaction.trans_time >= userinfo.event.event_time
 group by userinfo.transaction.trans_id
) transaction_info
inner join
(
 select uid, flatten(events) event
 from dfs.`/drill/testdata/resource-manager/nested-large.json`
) event_info
on transaction_info.max_event_time = event_info.event.event_time) d order by 
features[0].type) d1 where d1.uid < -1;
{code}

Below is the error from the logs
{code}
[Error Id: 26983344-dee3-4a33-8508-ad125f01fee6 on qa-node190.qa.lab:31010]
at 
org.apache.drill.common.exceptions.UserException$Builder.build(UserException.java:544)
 ~[drill-common-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
at 
org.apache.drill.exec.work.fragment.FragmentExecutor.sendFinalState(FragmentExecutor.java:295)
 [drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
at 
org.apache.drill.exec.work.fragment.FragmentExecutor.cleanup(FragmentExecutor.java:160)
 [drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
at 
org.apache.drill.exec.work.fragment.FragmentExecutor.run(FragmentExecutor.java:264)
 [drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
at 
org.apache.drill.common.SelfCleaningRunnable.run(SelfCleaningRunnable.java:38) 
[drill-common-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) 
[na:1.7.0_111]
at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) 
[na:1.7.0_111]
at java.lang.Thread.run(Thread.java:745) [na:1.7.0_111]
Caused by: java.lang.RuntimeException: java.lang.AssertionError
at 
org.apache.drill.common.DeferredException.addThrowable(DeferredException.java:101)
 ~[drill-common-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
at 
org.apache.drill.exec.work.fragment.FragmentExecutor.fail(FragmentExecutor.java:409)
 [drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
at 
org.apache.drill.exec.work.fragment.FragmentExecutor.run(FragmentExecutor.java:250)
 [drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
... 4 common frames omitted
Caused by: java.lang.AssertionError: null
at 
org.apache.drill.exec.vector.complex.RepeatedMapVector.load(RepeatedMapVector.java:444)
 ~[vector-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
at 
org.apache.drill.exec.cache.VectorAccessibleSerializable.readFromStream(VectorAccessibleSerializable.java:118)
 ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
at 
org.apache.drill.exec.physical.impl.xsort.managed.BatchGroup$SpilledRun.getBatch(BatchGroup.java:222)
 ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
at 
org.apache.drill.exec.physical.impl.xsort.managed.BatchGroup$SpilledRun.getNextIndex(BatchGroup.java:196)
 ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
at 
org.apache.drill.exec.test.generated.PriorityQueueCopierGen23.setup(PriorityQueueCopierTemplate.java:60)
 ~[na:na]
at 
org.apache.drill.exec.physical.impl.xsort.managed.CopierHolder.createCopier(CopierHolder.java:116)
 ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
at 
org.apache.drill.exec.physical.impl.xsort.managed.CopierHolder.access$200(CopierHolder.java:45)
 ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
at 
org.apache.drill.exec.physical.impl.xsort.managed.CopierHolder$BatchMerger.(CopierHolder.java:210)
 ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
at 
org.apache.drill.exec.physical.impl.xsort.managed.CopierHolder$BatchMerger.(CopierHolder.java:171)
 ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
at 
org.apache.drill.exec.physical.impl.xsort.managed.CopierHolder.startFinalMerge(CopierHolder.java:85)
 ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
a

[jira] [Commented] (DRILL-4335) Apache Drill should support network encryption

2017-04-25 Thread Laurent Goujon (JIRA)

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

Laurent Goujon commented on DRILL-4335:
---

unfortunately, not extensively. I'll do it this week!

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


[jira] [Commented] (DRILL-4335) Apache Drill should support network encryption

2017-04-25 Thread Sorabh Hamirwasia (JIRA)

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

Sorabh Hamirwasia commented on DRILL-4335:
--

[~laurentgo] - Hi Laurent, Did you get chance to look into the changes ? It 
would be great if you can help to review the pull request.

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


[jira] [Commented] (DRILL-5444) Document missing string functions

2017-04-25 Thread Khurram Faraaz (JIRA)

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

Khurram Faraaz commented on DRILL-5444:
---

[~arina] Are these undocumented string functions tested ? 

> Document missing string functions
> -
>
> Key: DRILL-5444
> URL: https://issues.apache.org/jira/browse/DRILL-5444
> Project: Apache Drill
>  Issue Type: Improvement
>Reporter: Arina Ielchiieva
>Assignee: Khurram Faraaz
>Priority: Minor
>  Labels: doc-impacting
>
> https://drill.apache.org/docs/string-manipulation/ does not contain full list 
> of Drill string functions.
> For example, reverse, left, right, two of three variations of substr / 
> substring are missing.
> Source - 
> https://github.com/apache/drill/blob/72903d01424139057d4309ce6655e0aecee2573e/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/StringFunctions.java



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Assigned] (DRILL-5444) Document missing string functions

2017-04-25 Thread Khurram Faraaz (JIRA)

 [ 
https://issues.apache.org/jira/browse/DRILL-5444?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Khurram Faraaz reassigned DRILL-5444:
-

Assignee: Khurram Faraaz

> Document missing string functions
> -
>
> Key: DRILL-5444
> URL: https://issues.apache.org/jira/browse/DRILL-5444
> Project: Apache Drill
>  Issue Type: Improvement
>Reporter: Arina Ielchiieva
>Assignee: Khurram Faraaz
>Priority: Minor
>  Labels: doc-impacting
>
> https://drill.apache.org/docs/string-manipulation/ does not contain full list 
> of Drill string functions.
> For example, reverse, left, right, two of three variations of substr / 
> substring are missing.
> Source - 
> https://github.com/apache/drill/blob/72903d01424139057d4309ce6655e0aecee2573e/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/StringFunctions.java



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Commented] (DRILL-5318) Create a sub-operator test framework

2017-04-25 Thread ASF GitHub Bot (JIRA)

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

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

Github user jinfengni commented on the issue:

https://github.com/apache/drill/pull/788
  
It's fine to have example in a follow-up PR.

I was trying to see if I can use new operator/sub-operator test framework 
for the schema change task I'm doing. That requires to work with one or 
multiple RecordBatch (both data and operator).  


> Create a sub-operator test framework
> 
>
> Key: DRILL-5318
> URL: https://issues.apache.org/jira/browse/DRILL-5318
> Project: Apache Drill
>  Issue Type: Improvement
>  Components: Tools, Build & Test
>Affects Versions: 1.11.0
>Reporter: Paul Rogers
>Assignee: Paul Rogers
>  Labels: ready-to-commit
> Fix For: 1.11.0
>
> Attachments: Sub-OperatorTestFramework.pdf
>
>
> Drill provides two unit test frameworks for whole-server, SQL-based testing: 
> the original {{BaseTestQuery}} and the newer {{ClusterFixture}}. Both use the 
> {{TestBuilder}} mechanism to build system-level functional tests that run 
> queries and check results.
> Jason provided an operator-level test framework based, in part on mocks: 
> As Drill operators become more complex, we have a crying need for true 
> unit-level tests at a level below the whole system and below operators. That 
> is, we need to test the individual pieces that, together, form the operator.
> This umbrella ticket includes a number of tasks needed to create the 
> sub-operator framework. Our intention is that, over time, as we find the need 
> to revisit existing operators, or create new ones, we can employ the 
> sub-operator test framework to exercise code at a finer granularity than is 
> possible prior to this framework.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Commented] (DRILL-5318) Create a sub-operator test framework

2017-04-25 Thread ASF GitHub Bot (JIRA)

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

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

Github user paul-rogers commented on the issue:

https://github.com/apache/drill/pull/788
  
One of the key reasons for creating tests is that they show how to use an 
API. In this case, the associated tests show how to use the row set 
abstractions to create a schema, populate a row set and so on. Please also see 
the package-info.java file for Javadoc that provides an overview.

Please let me know if you want to use the test framework. If so, I'd like 
to work with you to figure out what additional documentation and/or examples 
would be useful.

This PR is a dependency on the "main show": a PR with refactored sort code 
and complete sort unit tests. That PR will be submitted this week now that all 
the prerequisites are in master. That will demonstrate now to use the framework 
to create a suite of unit tests. I can point to my personal repo if if it would 
be helpful to preview that code.


> Create a sub-operator test framework
> 
>
> Key: DRILL-5318
> URL: https://issues.apache.org/jira/browse/DRILL-5318
> Project: Apache Drill
>  Issue Type: Improvement
>  Components: Tools, Build & Test
>Affects Versions: 1.11.0
>Reporter: Paul Rogers
>Assignee: Paul Rogers
>  Labels: ready-to-commit
> Fix For: 1.11.0
>
> Attachments: Sub-OperatorTestFramework.pdf
>
>
> Drill provides two unit test frameworks for whole-server, SQL-based testing: 
> the original {{BaseTestQuery}} and the newer {{ClusterFixture}}. Both use the 
> {{TestBuilder}} mechanism to build system-level functional tests that run 
> queries and check results.
> Jason provided an operator-level test framework based, in part on mocks: 
> As Drill operators become more complex, we have a crying need for true 
> unit-level tests at a level below the whole system and below operators. That 
> is, we need to test the individual pieces that, together, form the operator.
> This umbrella ticket includes a number of tasks needed to create the 
> sub-operator framework. Our intention is that, over time, as we find the need 
> to revisit existing operators, or create new ones, we can employ the 
> sub-operator test framework to exercise code at a finer granularity than is 
> possible prior to this framework.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Updated] (DRILL-5419) Calculate return string length for literals & some string functions

2017-04-25 Thread Arina Ielchiieva (JIRA)

 [ 
https://issues.apache.org/jira/browse/DRILL-5419?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Arina Ielchiieva updated DRILL-5419:

Attachment: (was: version_without_cast.JPG)

> Calculate return string length for literals & some string functions
> ---
>
> Key: DRILL-5419
> URL: https://issues.apache.org/jira/browse/DRILL-5419
> Project: Apache Drill
>  Issue Type: Bug
>Affects Versions: 1.9.0
>Reporter: Arina Ielchiieva
>Assignee: Arina Ielchiieva
> Attachments: version_with_cast.JPG
>
>
> Though Drill is schema-less and cannot determine in advance what the length 
> of the column should be but if query has an explicit type/length specified, 
> Drill should return correct column length.
> For example, JDBC / ODBC Driver is ALWAYS returning 64K as the length of a 
> varchar or char even if casts are applied.
> Changes:
> *LITERALS*
> String literals length is the same as actual literal length.
> Example: for 'aaa' return length is 3.
> *CAST*
> Return length is the one indicated in cast expression. This also applies when 
> user has created view where each string columns was casted to varchar with 
> some specific length.
> This length will be returned to the user without need to apply cast one more 
> time. Below mentioned functions can take leverage of underlying varchar 
> length and calculate return length.
> *LOWER, UPPER, INITCAP, REVERSE, FIRST_VALUE, LAST_VALUE* 
> Return length is underlying column length, if column is known, the same 
> length will be returned.
> Example:
> lower(cast(col as varchar(30))) will return 30.
> lower(col) will return max varchar length, since we don't know actual column 
> length.
> *LPAD, RPAD*
> Pads the string to the length specified. Return length is this specified 
> length. 
> *CONCAT, CONCAT OPERATOR (||)*
> Return length is sum of underlying columns length. If length is greater then 
> varchar max length,  varchar max length is returned.
> *SUBSTR, SUBSTRING, LEFT, RIGHT*
> Calculates return length according to each function substring rules, for 
> example, taking into account how many char should be substracted.
> *IF EXPRESSIONS (CASE STATEMENT, COALESCE), UNION OPERATOR*
> When combining string columns with different length, return length is max 
> from source columns.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Created] (DRILL-5444) Document missing string functions

2017-04-25 Thread Arina Ielchiieva (JIRA)
Arina Ielchiieva created DRILL-5444:
---

 Summary: Document missing string functions
 Key: DRILL-5444
 URL: https://issues.apache.org/jira/browse/DRILL-5444
 Project: Apache Drill
  Issue Type: Improvement
Reporter: Arina Ielchiieva
Priority: Minor


https://drill.apache.org/docs/string-manipulation/ does not contain full list 
of Drill string functions.

For example, reverse, left right, two of three variations of substr / substring 
are missing.

Source - 
https://github.com/apache/drill/blob/72903d01424139057d4309ce6655e0aecee2573e/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/StringFunctions.java





--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Updated] (DRILL-5444) Document missing string functions

2017-04-25 Thread Arina Ielchiieva (JIRA)

 [ 
https://issues.apache.org/jira/browse/DRILL-5444?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Arina Ielchiieva updated DRILL-5444:

Description: 
https://drill.apache.org/docs/string-manipulation/ does not contain full list 
of Drill string functions.

For example, reverse, left, right, two of three variations of substr / 
substring are missing.

Source - 
https://github.com/apache/drill/blob/72903d01424139057d4309ce6655e0aecee2573e/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/StringFunctions.java



  was:
https://drill.apache.org/docs/string-manipulation/ does not contain full list 
of Drill string functions.

For example, reverse, left right, two of three variations of substr / substring 
are missing.

Source - 
https://github.com/apache/drill/blob/72903d01424139057d4309ce6655e0aecee2573e/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/StringFunctions.java




> Document missing string functions
> -
>
> Key: DRILL-5444
> URL: https://issues.apache.org/jira/browse/DRILL-5444
> Project: Apache Drill
>  Issue Type: Improvement
>Reporter: Arina Ielchiieva
>Priority: Minor
>  Labels: doc-impacting
>
> https://drill.apache.org/docs/string-manipulation/ does not contain full list 
> of Drill string functions.
> For example, reverse, left, right, two of three variations of substr / 
> substring are missing.
> Source - 
> https://github.com/apache/drill/blob/72903d01424139057d4309ce6655e0aecee2573e/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/StringFunctions.java



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Updated] (DRILL-5419) Calculate return string length for literals & some string functions

2017-04-25 Thread Zelaine Fong (JIRA)

 [ 
https://issues.apache.org/jira/browse/DRILL-5419?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Zelaine Fong updated DRILL-5419:

Reviewer: Jinfeng Ni

Assigned Reviewer to [~jni]

> Calculate return string length for literals & some string functions
> ---
>
> Key: DRILL-5419
> URL: https://issues.apache.org/jira/browse/DRILL-5419
> Project: Apache Drill
>  Issue Type: Bug
>Affects Versions: 1.9.0
>Reporter: Arina Ielchiieva
>Assignee: Arina Ielchiieva
> Attachments: version_with_cast.JPG, version_without_cast.JPG
>
>
> Though Drill is schema-less and cannot determine in advance what the length 
> of the column should be but if query has an explicit type/length specified, 
> Drill should return correct column length.
> For example, JDBC / ODBC Driver is ALWAYS returning 64K as the length of a 
> varchar or char even if casts are applied.
> Changes:
> *LITERALS*
> String literals length is the same as actual literal length.
> Example: for 'aaa' return length is 3.
> *CAST*
> Return length is the one indicated in cast expression. This also applies when 
> user has created view where each string columns was casted to varchar with 
> some specific length.
> This length will be returned to the user without need to apply cast one more 
> time. Below mentioned functions can take leverage of underlying varchar 
> length and calculate return length.
> *LOWER, UPPER, INITCAP, REVERSE, FIRST_VALUE, LAST_VALUE* 
> Return length is underlying column length, if column is known, the same 
> length will be returned.
> Example:
> lower(cast(col as varchar(30))) will return 30.
> lower(col) will return max varchar length, since we don't know actual column 
> length.
> *LPAD, RPAD*
> Pads the string to the length specified. Return length is this specified 
> length. 
> *CONCAT, CONCAT OPERATOR (||)*
> Return length is sum of underlying columns length. If length is greater then 
> varchar max length,  varchar max length is returned.
> *SUBSTR, SUBSTRING, LEFT, RIGHT*
> Calculates return length according to each function substring rules, for 
> example, taking into account how many char should be substracted.
> *IF EXPRESSIONS (CASE STATEMENT, COALESCE), UNION OPERATOR*
> When combining string columns with different length, return length is max 
> from source columns.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Commented] (DRILL-5419) Calculate return string length for literals & some string functions

2017-04-25 Thread ASF GitHub Bot (JIRA)

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

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

GitHub user arina-ielchiieva opened a pull request:

https://github.com/apache/drill/pull/819

DRILL-5419: Calculate return string length for literals & some string…

… functions

1. Revisited calculation logic for string literals and some string 
functions (cast, upper, lower, initcap, reverse, concat, concat operator, 
substring, substr, left, right, rpad, lpad, case statement, coalesce, 
first_value, last_value).
2. Deprecated width and changed it to precision for string types.
3. Synchronized return type length calculation logic between limit 0 and 
regular queries.
4. Performed minor refactoring, added appropriate unit tests.

You can merge this pull request into a Git repository by running:

$ git pull https://github.com/arina-ielchiieva/drill DRILL-5419

Alternatively you can review and apply these changes as the patch at:

https://github.com/apache/drill/pull/819.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

This closes #819


commit c7ec21bc1caa02587f6b748ee6e213b78e916cb6
Author: Arina Ielchiieva 
Date:   2017-04-06T10:44:26Z

DRILL-5419: Calculate return string length for literals & some string 
functions




> Calculate return string length for literals & some string functions
> ---
>
> Key: DRILL-5419
> URL: https://issues.apache.org/jira/browse/DRILL-5419
> Project: Apache Drill
>  Issue Type: Bug
>Affects Versions: 1.9.0
>Reporter: Arina Ielchiieva
>Assignee: Arina Ielchiieva
> Attachments: version_with_cast.JPG, version_without_cast.JPG
>
>
> Though Drill is schema-less and cannot determine in advance what the length 
> of the column should be but if query has an explicit type/length specified, 
> Drill should return correct column length.
> For example, JDBC / ODBC Driver is ALWAYS returning 64K as the length of a 
> varchar or char even if casts are applied.
> Changes:
> *LITERALS*
> String literals length is the same as actual literal length.
> Example: for 'aaa' return length is 3.
> *CAST*
> Return length is the one indicated in cast expression. This also applies when 
> user has created view where each string columns was casted to varchar with 
> some specific length.
> This length will be returned to the user without need to apply cast one more 
> time. Below mentioned functions can take leverage of underlying varchar 
> length and calculate return length.
> *LOWER, UPPER, INITCAP, REVERSE, FIRST_VALUE, LAST_VALUE* 
> Return length is underlying column length, if column is known, the same 
> length will be returned.
> Example:
> lower(cast(col as varchar(30))) will return 30.
> lower(col) will return max varchar length, since we don't know actual column 
> length.
> *LPAD, RPAD*
> Pads the string to the length specified. Return length is this specified 
> length. 
> *CONCAT, CONCAT OPERATOR (||)*
> Return length is sum of underlying columns length. If length is greater then 
> varchar max length,  varchar max length is returned.
> *SUBSTR, SUBSTRING, LEFT, RIGHT*
> Calculates return length according to each function substring rules, for 
> example, taking into account how many char should be substracted.
> *IF EXPRESSIONS (CASE STATEMENT, COALESCE), UNION OPERATOR*
> When combining string columns with different length, return length is max 
> from source columns.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Updated] (DRILL-5419) Calculate return string length for literals & some string functions

2017-04-25 Thread Arina Ielchiieva (JIRA)

 [ 
https://issues.apache.org/jira/browse/DRILL-5419?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Arina Ielchiieva updated DRILL-5419:

Description: 
Though Drill is schema-less and cannot determine in advance what the length of 
the column should be but if query has an explicit type/length specified, Drill 
should return correct column length.

For example, JDBC / ODBC Driver is ALWAYS returning 64K as the length of a 
varchar or char even if casts are applied.

Changes:
*LITERALS*
String literals length is the same as actual literal length.
Example: for 'aaa' return length is 3.

*CAST*
Return length is the one indicated in cast expression. This also applies when 
user has created view where each string columns was casted to varchar with some 
specific length.
This length will be returned to the user without need to apply cast one more 
time. Below mentioned functions can take leverage of underlying varchar length 
and calculate return length.

*LOWER, UPPER, INITCAP, REVERSE, FIRST_VALUE, LAST_VALUE* 
Return length is underlying column length, if column is known, the same length 
will be returned.
Example:
lower(cast(col as varchar(30))) will return 30.
lower(col) will return max varchar length, since we don't know actual column 
length.

*LPAD, RPAD*
Pads the string to the length specified. Return length is this specified 
length. 

*CONCAT, CONCAT OPERATOR (||)*
Return length is sum of underlying columns length. If length is greater then 
varchar max length,  varchar max length is returned.

*SUBSTR, SUBSTRING, LEFT, RIGHT*
Calculates return length according to each function substring rules, for 
example, taking into account how many char should be substracted.

*IF EXPRESSIONS (CASE STATEMENT, COALESCE), UNION OPERATOR*
When combining string columns with different length, return length is max from 
source columns.







  was:
Though Drill is schema-less and cannot determine in advance what the length of 
the column should be but if query has an explicit type/length specified, Drill 
should return correct column length.

For example, JDBC / ODBC Driver is ALWAYS returning 64K as the length of a 
varchar or char even if casts are applied.

Changes:
*LITERALS*
String literals length is the same as actual literal length.
Example: for 'aaa' return length is 3.

*CAST* return length is the one indicated in cast expression. This also applies 
when user has created view where each string columns was casted to varchar with 
some specific length.
This length will be returned to the user without need to apply cast one more 
time. Below mentioned functions can take leverage of underlying varchar length 
and calculate return length.

*LOWER, UPPER, INITCAP, REVERSE, FIRST_VALUE, LAST_VALUE* 
return length is underlying column length, if column is known, the same length 
will be returned.
Example:
lower(cast(col as varchar(30))) will return 30.
lower(col) will return max varchar length, since we don't know actual column 
length.

*LPAD, RPAD*
Pads the string to the length specified. Return length is this specified 
length. 

*CONCAT, CONCAT OPERATOR (||)*
Return length is sum of underlying columns length. If length is greater then 
varchar max length,  varchar max length is returned.

*SUBSTR, SUBSTRING, LEFT, RIGHT*
Calculates return length according to each function substring rules, for 
example, taking into account how many char should be substracted.

*IF EXPRESSIONS (CASE STATEMENT, COALESCE), UNION OPERATOR*
When combining string columns with different length, return length is max from 
source columns.








> Calculate return string length for literals & some string functions
> ---
>
> Key: DRILL-5419
> URL: https://issues.apache.org/jira/browse/DRILL-5419
> Project: Apache Drill
>  Issue Type: Bug
>Affects Versions: 1.9.0
>Reporter: Arina Ielchiieva
>Assignee: Arina Ielchiieva
> Attachments: version_with_cast.JPG, version_without_cast.JPG
>
>
> Though Drill is schema-less and cannot determine in advance what the length 
> of the column should be but if query has an explicit type/length specified, 
> Drill should return correct column length.
> For example, JDBC / ODBC Driver is ALWAYS returning 64K as the length of a 
> varchar or char even if casts are applied.
> Changes:
> *LITERALS*
> String literals length is the same as actual literal length.
> Example: for 'aaa' return length is 3.
> *CAST*
> Return length is the one indicated in cast expression. This also applies when 
> user has created view where each string columns was casted to varchar with 
> some specific length.
> This length will be returned to the user without need to apply cast one more 
> time. Below mentioned functions can take leverage of underlying varchar 
> length and calculate return length.
> *LOWER, UPPER, IN

[jira] [Updated] (DRILL-5419) Calculate return string length for literals & some string functions

2017-04-25 Thread Arina Ielchiieva (JIRA)

 [ 
https://issues.apache.org/jira/browse/DRILL-5419?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Arina Ielchiieva updated DRILL-5419:

Description: 
Though Drill is schema-less and cannot determine in advance what the length of 
the column should be but if query has an explicit type/length specified, Drill 
should return correct column length.

For example, JDBC / ODBC Driver is ALWAYS returning 64K as the length of a 
varchar or char even if casts are applied.

Changes:
*LITERALS*
String literals length is the same as actual literal length.
Example: for 'aaa' return length is 3.

*CAST* return length is the one indicated in cast expression. This also applies 
when user has created view where each string columns was casted to varchar with 
some specific length.
This length will be returned to the user without need to apply cast one more 
time. Below mentioned functions can take leverage of underlying varchar length 
and calculate return length.

*LOWER, UPPER, INITCAP, REVERSE, FIRST_VALUE, LAST_VALUE* 
return length is underlying column length, if column is known, the same length 
will be returned.
Example:
lower(cast(col as varchar(30))) will return 30.
lower(col) will return max varchar length, since we don't know actual column 
length.

*LPAD, RPAD*
Pads the string to the length specified. Return length is this specified 
length. 

*CONCAT, CONCAT OPERATOR (||)*
Return length is sum of underlying columns length. If length is greater then 
varchar max length,  varchar max length is returned.

*SUBSTR, SUBSTRING, LEFT, RIGHT*
Calculates return length according to each function substring rules, for 
example, taking into account how many char should be substracted.

*IF EXPRESSIONS (CASE STATEMENT, COALESCE), UNION OPERATOR*
When combining string columns with different length, return length is max from 
source columns.







  was:
Though Drill is schema-less and cannot determine in advance what the length of 
the column should be but if query has an explicit type/length specified, Drill 
should return correct column length.

For example, JDBC / ODBC Driver is ALWAYS returning 64K as the length of a 
varchar or char even if casts are applied.

Changes:
1. If casts are applied in query, Drill will return length which matches length 
indicated in cast expression.
2. If casts are not applied and varchar length is not defined, Drill will 
return varchar length as undefined (set to 0) (before Drill returned max - 64K).


> Calculate return string length for literals & some string functions
> ---
>
> Key: DRILL-5419
> URL: https://issues.apache.org/jira/browse/DRILL-5419
> Project: Apache Drill
>  Issue Type: Bug
>Affects Versions: 1.9.0
>Reporter: Arina Ielchiieva
>Assignee: Arina Ielchiieva
> Attachments: version_with_cast.JPG, version_without_cast.JPG
>
>
> Though Drill is schema-less and cannot determine in advance what the length 
> of the column should be but if query has an explicit type/length specified, 
> Drill should return correct column length.
> For example, JDBC / ODBC Driver is ALWAYS returning 64K as the length of a 
> varchar or char even if casts are applied.
> Changes:
> *LITERALS*
> String literals length is the same as actual literal length.
> Example: for 'aaa' return length is 3.
> *CAST* return length is the one indicated in cast expression. This also 
> applies when user has created view where each string columns was casted to 
> varchar with some specific length.
> This length will be returned to the user without need to apply cast one more 
> time. Below mentioned functions can take leverage of underlying varchar 
> length and calculate return length.
> *LOWER, UPPER, INITCAP, REVERSE, FIRST_VALUE, LAST_VALUE* 
> return length is underlying column length, if column is known, the same 
> length will be returned.
> Example:
> lower(cast(col as varchar(30))) will return 30.
> lower(col) will return max varchar length, since we don't know actual column 
> length.
> *LPAD, RPAD*
> Pads the string to the length specified. Return length is this specified 
> length. 
> *CONCAT, CONCAT OPERATOR (||)*
> Return length is sum of underlying columns length. If length is greater then 
> varchar max length,  varchar max length is returned.
> *SUBSTR, SUBSTRING, LEFT, RIGHT*
> Calculates return length according to each function substring rules, for 
> example, taking into account how many char should be substracted.
> *IF EXPRESSIONS (CASE STATEMENT, COALESCE), UNION OPERATOR*
> When combining string columns with different length, return length is max 
> from source columns.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Updated] (DRILL-5419) Calculate return string length for literals & some string functions

2017-04-25 Thread Arina Ielchiieva (JIRA)

 [ 
https://issues.apache.org/jira/browse/DRILL-5419?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Arina Ielchiieva updated DRILL-5419:

Summary: Calculate return string length for literals & some string 
functions  (was: Return correct varchar length when cast is applied)

> Calculate return string length for literals & some string functions
> ---
>
> Key: DRILL-5419
> URL: https://issues.apache.org/jira/browse/DRILL-5419
> Project: Apache Drill
>  Issue Type: Bug
>Affects Versions: 1.9.0
>Reporter: Arina Ielchiieva
>Assignee: Arina Ielchiieva
> Attachments: version_with_cast.JPG, version_without_cast.JPG
>
>
> Though Drill is schema-less and cannot determine in advance what the length 
> of the column should be but if query has an explicit type/length specified, 
> Drill should return correct column length.
> For example, JDBC / ODBC Driver is ALWAYS returning 64K as the length of a 
> varchar or char even if casts are applied.
> Changes:
> 1. If casts are applied in query, Drill will return length which matches 
> length indicated in cast expression.
> 2. If casts are not applied and varchar length is not defined, Drill will 
> return varchar length as undefined (set to 0) (before Drill returned max - 
> 64K).



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)