You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by ar...@apache.org on 2017/07/21 13:30:49 UTC

[1/4] drill git commit: DRILL-5634: Add Crypto Functions

Repository: drill
Updated Branches:
  refs/heads/master 8d659ff70 -> 07346c782


DRILL-5634: Add Crypto Functions

closes #865


Project: http://git-wip-us.apache.org/repos/asf/drill/repo
Commit: http://git-wip-us.apache.org/repos/asf/drill/commit/a3f9dff0
Tree: http://git-wip-us.apache.org/repos/asf/drill/tree/a3f9dff0
Diff: http://git-wip-us.apache.org/repos/asf/drill/diff/a3f9dff0

Branch: refs/heads/master
Commit: a3f9dff0a00f5ed8b87c0802ea7cc069c6407bcd
Parents: 8d659ff
Author: cgivre <cg...@gmail.com>
Authored: Sun Jul 2 23:13:02 2017 -0400
Committer: Arina Ielchiieva <ar...@gmail.com>
Committed: Fri Jul 21 15:48:06 2017 +0300

----------------------------------------------------------------------
 exec/java-exec/pom.xml                          |   5 +
 .../exec/expr/fn/impl/CryptoFunctions.java      | 388 +++++++++++++++++++
 .../drill/exec/fn/impl/TestCryptoFunctions.java |  92 +++++
 exec/jdbc-all/pom.xml                           |   4 +
 4 files changed, 489 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/a3f9dff0/exec/java-exec/pom.xml
----------------------------------------------------------------------
diff --git a/exec/java-exec/pom.xml b/exec/java-exec/pom.xml
index cd287aa..c275303 100644
--- a/exec/java-exec/pom.xml
+++ b/exec/java-exec/pom.xml
@@ -94,6 +94,11 @@
       <version>2.2</version>
     </dependency>
     <dependency>
+      <groupId>commons-codec</groupId>
+      <artifactId>commons-codec</artifactId>
+      <version>1.10</version>
+    </dependency>
+    <dependency>
       <groupId>com.thoughtworks.paranamer</groupId>
       <artifactId>paranamer</artifactId>
       <version>2.5.6</version>

http://git-wip-us.apache.org/repos/asf/drill/blob/a3f9dff0/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/CryptoFunctions.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/CryptoFunctions.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/CryptoFunctions.java
new file mode 100644
index 0000000..65e5fb5
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/CryptoFunctions.java
@@ -0,0 +1,388 @@
+/*
+ * 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.expr.fn.impl;
+
+import io.netty.buffer.DrillBuf;
+import org.apache.drill.exec.expr.DrillSimpleFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+import org.apache.drill.exec.expr.annotations.Output;
+import org.apache.drill.exec.expr.annotations.Param;
+import org.apache.drill.exec.expr.annotations.Workspace;
+import org.apache.drill.exec.expr.holders.VarCharHolder;
+
+import javax.crypto.Cipher;
+import javax.inject.Inject;
+
+public class CryptoFunctions {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(CryptoFunctions.class);
+
+  private CryptoFunctions() {
+  }
+
+  /**
+   * This class returns the md2 digest of a given input string.
+   *  Usage is SELECT md2( <input string> ) FROM ...
+   */
+
+  @FunctionTemplate(name = "md2", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.NULL_IF_NULL)
+  public static class MD2Function implements DrillSimpleFunc {
+
+    @Param
+    VarCharHolder rawInput;
+
+    @Output
+    VarCharHolder out;
+
+    @Inject
+    DrillBuf buffer;
+
+    @Override
+    public void setup() {
+    }
+
+    @Override
+    public void eval() {
+
+      String input = org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF8(rawInput.start, rawInput.end, rawInput.buffer);
+      String outputString = org.apache.commons.codec.digest.DigestUtils.md2Hex(input).toLowerCase();
+
+      out.buffer = buffer;
+      out.start = 0;
+      out.end = outputString.getBytes().length;
+      buffer.setBytes(0, outputString.getBytes());
+    }
+
+  }
+
+  /**
+   * This function returns the MD5 digest of a given input string.
+   *  Usage is shown below:
+   *  select md5( 'testing' ) from (VALUES(1));
+   */
+
+  @FunctionTemplate(name = "md5", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.NULL_IF_NULL)
+  public static class MD5Function implements DrillSimpleFunc {
+
+    @Param
+    VarCharHolder rawInput;
+
+    @Output
+    VarCharHolder out;
+
+    @Inject
+    DrillBuf buffer;
+
+    @Override
+    public void setup() {
+    }
+
+    @Override
+    public void eval() {
+
+      String input = org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF8(rawInput.start, rawInput.end, rawInput.buffer);
+      String outputString = org.apache.commons.codec.digest.DigestUtils.md5Hex(input).toLowerCase();
+
+      out.buffer = buffer;
+      out.start = 0;
+      out.end = outputString.getBytes().length;
+      buffer.setBytes(0, outputString.getBytes());
+    }
+
+  }
+
+  /**
+   * sha(<text>) / sha1(<text>): Calculates an SHA-1 160-bit checksum for the string, as described in RFC 3174 (Secure Hash Algorithm).
+   * (https://en.wikipedia.org/wiki/SHA-1) The value is returned as a string of 40 hexadecimal digits, or NULL if the argument was NULL.
+   * Note that sha() and sha1() are aliases for the same function.
+   *
+   * > select sha1( 'testing' ) from (VALUES(1));
+   */
+
+  @FunctionTemplate(names = {"sha", "sha1"}, scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.NULL_IF_NULL)
+  public static class SHA1Function implements DrillSimpleFunc {
+
+    @Param
+    VarCharHolder rawInput;
+
+    @Output
+    VarCharHolder out;
+
+    @Inject
+    DrillBuf buffer;
+
+    @Override
+    public void setup() {
+
+    }
+
+    @Override
+    public void eval() {
+
+      String input = org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF8(rawInput.start, rawInput.end, rawInput.buffer);
+
+      String sha1 = org.apache.commons.codec.digest.DigestUtils.sha1Hex(input);
+
+      out.buffer = buffer;
+      out.start = 0;
+      out.end = sha1.getBytes().length;
+      buffer.setBytes(0, sha1.getBytes());
+    }
+
+  }
+
+  /**
+   * sha2(<text>) / sha256(<text>): Calculates an SHA-2 256-bit checksum for the string.  The value is returned as a string of hexadecimal digits,
+   * or NULL if the argument was NULL. Note that sha2() and sha256() are aliases for the same function.
+   * > select sha2( 'testing' ) from (VALUES(1));
+   */
+
+  @FunctionTemplate(names = {"sha256", "sha2"}, scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.NULL_IF_NULL)
+  public static class SHA256Function implements DrillSimpleFunc {
+
+    @Param
+    VarCharHolder rawInput;
+
+    @Output
+    VarCharHolder out;
+
+    @Inject
+    DrillBuf buffer;
+
+
+    @Override
+    public void setup() {
+
+    }
+
+    @Override
+    public void eval() {
+
+      String input = org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF8(rawInput.start, rawInput.end, rawInput.buffer);
+
+      String sha2 = org.apache.commons.codec.digest.DigestUtils.sha256Hex(input);
+
+      out.buffer = buffer;
+      out.start = 0;
+      out.end = sha2.getBytes().length;
+      buffer.setBytes(0, sha2.getBytes());
+    }
+
+  }
+
+  /**
+   * This function returns the SHA384 digest of a given input string.
+   *  Usage is shown below:
+   *  select sha384( 'testing' ) from (VALUES(1));
+   */
+
+  @FunctionTemplate(name = "sha384", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.NULL_IF_NULL)
+  public static class SHA384Function implements DrillSimpleFunc {
+
+    @Param
+    VarCharHolder rawInput;
+
+    @Output
+    VarCharHolder out;
+
+    @Inject
+    DrillBuf buffer;
+
+    @Override
+    public void setup() {
+
+    }
+
+    @Override
+    public void eval() {
+
+      String input = org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF8(rawInput.start, rawInput.end, rawInput.buffer);
+
+      String sha384 = org.apache.commons.codec.digest.DigestUtils.sha384Hex(input);
+
+      out.buffer = buffer;
+      out.start = 0;
+      out.end = sha384.getBytes().length;
+      buffer.setBytes(0, sha384.getBytes());
+    }
+
+  }
+
+  /**
+   * This function returns the SHA512 digest of a given input string.
+   *  Usage is shown below:
+   *  select sha512( 'testing' ) from (VALUES(1));
+   */
+
+
+  @FunctionTemplate(name = "sha512", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.NULL_IF_NULL)
+  public static class SHA512Function implements DrillSimpleFunc {
+
+    @Param
+    VarCharHolder rawInput;
+
+    @Output
+    VarCharHolder out;
+
+    @Inject
+    DrillBuf buffer;
+
+    @Override
+    public void setup() {
+
+    }
+
+    @Override
+    public void eval() {
+
+      String input = org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF8(rawInput.start, rawInput.end, rawInput.buffer);
+
+      String sha512 = org.apache.commons.codec.digest.DigestUtils.sha512Hex(input);
+
+      out.buffer = buffer;
+      out.start = 0;
+      out.end = sha512.getBytes().length;
+      buffer.setBytes(0, sha512.getBytes());
+    }
+
+  }
+
+  /**
+   * aes_encrypt()/ aes_decrypt(): implement encryption and decryption of data using the official AES (Advanced Encryption Standard) algorithm,
+   * previously known as “Rijndael.” AES_ENCRYPT() encrypts the string str using the key string key_str and returns a
+   * binary string containing the encrypted output.
+   * Usage:  SELECT aes_encrypt( 'encrypted_text', 'my_secret_key' ) AS aes FROM (VALUES(1));
+   */
+
+
+  @FunctionTemplate(name = "aes_encrypt", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.NULL_IF_NULL)
+  public static class AESEncryptFunction implements DrillSimpleFunc {
+
+    @Param
+    VarCharHolder rawInput;
+
+    @Param
+    VarCharHolder rawKey;
+
+    @Output
+    VarCharHolder out;
+
+    @Inject
+    DrillBuf buffer;
+
+    @Workspace
+    Cipher cipher;
+
+    @Override
+    public void setup() {
+      String key = org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF8(rawKey.start, rawKey.end, rawKey.buffer);
+
+      try {
+        byte[] keyByteArray = key.getBytes("UTF-8");
+        java.security.MessageDigest sha = java.security.MessageDigest.getInstance("SHA-1");
+        keyByteArray = sha.digest(keyByteArray);
+        keyByteArray = java.util.Arrays.copyOf(keyByteArray, 16);
+        javax.crypto.spec.SecretKeySpec secretKey = new javax.crypto.spec.SecretKeySpec(keyByteArray, "AES");
+
+        cipher = Cipher.getInstance("AES/ECB/PKCS5Padding");
+        cipher.init(Cipher.ENCRYPT_MODE, secretKey);
+      } catch (Exception e) {
+        //Exceptions are ignored
+      }
+    }
+
+    @Override
+    public void eval() {
+
+      String input = org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF8(rawInput.start, rawInput.end, rawInput.buffer);
+      String encryptedText = "";
+      try {
+        encryptedText = javax.xml.bind.DatatypeConverter.printBase64Binary(cipher.doFinal(input.getBytes("UTF-8")));
+      } catch (Exception e) {
+        //Exceptions are ignored
+      }
+
+      out.buffer = buffer;
+      out.start = 0;
+      out.end = encryptedText.getBytes().length;
+      buffer.setBytes(0, encryptedText.getBytes());
+    }
+
+  }
+
+  /**
+   *  AES_DECRYPT() decrypts the encrypted string crypt_str using the key string key_str and returns the original cleartext string.
+   *  If either function argument is NULL, the function returns NULL.
+   *  Usage:  SELECT aes_decrypt( <encrypted_text>, <key> ) FROM ...
+   */
+
+  @FunctionTemplate(name = "aes_decrypt", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.NULL_IF_NULL)
+  public static class AESDecryptFunction implements DrillSimpleFunc {
+
+    @Param
+    VarCharHolder rawInput;
+
+    @Param
+    VarCharHolder rawKey;
+
+    @Output
+    VarCharHolder out;
+
+    @Inject
+    DrillBuf buffer;
+
+    @Workspace
+    Cipher cipher;
+
+    @Override
+    public void setup() {
+      String key = org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF8(rawKey.start, rawKey.end, rawKey.buffer);
+
+      try {
+        byte[] keyByteArray = key.getBytes("UTF-8");
+        java.security.MessageDigest sha = java.security.MessageDigest.getInstance("SHA-1");
+        keyByteArray = sha.digest(keyByteArray);
+        keyByteArray = java.util.Arrays.copyOf(keyByteArray, 16);
+        javax.crypto.spec.SecretKeySpec secretKey = new javax.crypto.spec.SecretKeySpec(keyByteArray, "AES");
+
+        cipher = Cipher.getInstance("AES/ECB/PKCS5Padding");
+        cipher.init(Cipher.DECRYPT_MODE, secretKey);
+      } catch (Exception e) {
+        //Exceptions are ignored
+      }
+    }
+
+    @Override
+    public void eval() {
+
+      String input = org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF8(rawInput.start, rawInput.end, rawInput.buffer);
+      String decryptedText = "";
+      try {
+        decryptedText = new String(cipher.doFinal(javax.xml.bind.DatatypeConverter.parseBase64Binary(input)));
+      } catch (Exception e) {
+        //Exceptions are ignored
+      }
+
+      out.buffer = buffer;
+      out.start = 0;
+      out.end = decryptedText.getBytes().length;
+      buffer.setBytes(0, decryptedText.getBytes());
+    }
+
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/drill/blob/a3f9dff0/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestCryptoFunctions.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestCryptoFunctions.java b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestCryptoFunctions.java
new file mode 100644
index 0000000..4dc6284
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestCryptoFunctions.java
@@ -0,0 +1,92 @@
+/*
+ * 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.fn.impl;
+
+import org.apache.drill.BaseTestQuery;
+import org.junit.Test;
+
+public class TestCryptoFunctions extends BaseTestQuery {
+
+  @Test
+  public void testMD5() throws Exception {
+    final String query = "select md5('testing') as md5Hash from (values(1))";
+    testBuilder().sqlQuery(query).ordered().baselineColumns("md5Hash").baselineValues("ae2b1fca515949e5d54fb22b8ed95575").go();
+  }
+
+  @Test
+  public void testMD2() throws Exception {
+    final String query = "select md2('testing') as md2Hash from (values(1))";
+    testBuilder().sqlQuery(query).ordered().baselineColumns("md2Hash").baselineValues("fc134df10d6ecafceb5c75861d01b41f").go();
+  }
+
+
+  @Test
+  public void testSHA1() throws Exception {
+    final String query = "select sha('testing') as shaHash from (values(1))";
+    testBuilder()
+      .sqlQuery(query)
+      .ordered()
+      .baselineColumns("shaHash")
+      .baselineValues("dc724af18fbdd4e59189f5fe768a5f8311527050")
+      .go();
+  }
+
+  @Test
+  public void testSHA384() throws Exception {
+    final String query = "select sha384('testing') as shaHash from (values(1))";
+    testBuilder()
+      .sqlQuery(query)
+      .ordered()
+      .baselineColumns("shaHash")
+      .baselineValues("cf4811d74fd40504674fc3273f824fa42f755b9660a2e902b57f1df74873db1a91a037bcee65f1a88ecd1ef57ff254c9")
+      .go();
+  }
+
+  @Test
+  public void testSHA512() throws Exception {
+    final String query = "select sha512('testing') as shaHash from (values(1))";
+    testBuilder()
+      .sqlQuery(query)
+      .ordered()
+      .baselineColumns("shaHash")
+      .baselineValues("521b9ccefbcd14d179e7a1bb877752870a6d620938b28a66a107eac6e6805b9d0989f45b5730508041aa5e710847d439ea74cd312c9355f1f2dae08d40e41d50")
+      .go();
+  }
+
+  @Test
+  public void testAESEncrypt() throws Exception {
+    final String query = "select aes_encrypt('testing', 'secret_key') as encrypted FROM (VALUES(1))";
+    testBuilder()
+      .sqlQuery(query)
+      .ordered()
+      .baselineColumns("encrypted")
+      .baselineValues("ICf+zdOrLitogB8HUDru0w==")
+      .go();
+  }
+
+  @Test
+  public void testAESDecrypt() throws Exception {
+    final String query = "select aes_decrypt('ICf+zdOrLitogB8HUDru0w==', 'secret_key') as decrypt from (values(1))";
+    testBuilder()
+      .sqlQuery(query)
+      .ordered()
+      .baselineColumns("decrypt")
+      .baselineValues("testing")
+      .go();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/drill/blob/a3f9dff0/exec/jdbc-all/pom.xml
----------------------------------------------------------------------
diff --git a/exec/jdbc-all/pom.xml b/exec/jdbc-all/pom.xml
index 238288e..56389be 100644
--- a/exec/jdbc-all/pom.xml
+++ b/exec/jdbc-all/pom.xml
@@ -68,6 +68,10 @@
           <artifactId>bcpkix-jdk15on</artifactId>
         </exclusion>
         <exclusion>
+          <groupId>commons-codec</groupId>
+          <artifactId>commons-codec</artifactId>
+        </exclusion>
+        <exclusion>
           <groupId>org.bouncycastle</groupId>
           <artifactId>bcpkix-jdk15on</artifactId>
         </exclusion>


[2/4] drill git commit: DRILL-4755: Fix IOBE for convert_from/convert_to functions with incorrect encoding type

Posted by ar...@apache.org.
DRILL-4755: Fix IOBE for convert_from/convert_to functions with incorrect encoding type

closes #867


Project: http://git-wip-us.apache.org/repos/asf/drill/repo
Commit: http://git-wip-us.apache.org/repos/asf/drill/commit/34d736e6
Tree: http://git-wip-us.apache.org/repos/asf/drill/tree/34d736e6
Diff: http://git-wip-us.apache.org/repos/asf/drill/diff/34d736e6

Branch: refs/heads/master
Commit: 34d736e6dabea7d3d4bb8fb699e956f72bcdbfb8
Parents: a3f9dff
Author: Volodymyr Vysotskyi <vv...@gmail.com>
Authored: Tue Jul 4 18:19:23 2017 +0000
Committer: Arina Ielchiieva <ar...@gmail.com>
Committed: Fri Jul 21 15:48:07 2017 +0300

----------------------------------------------------------------------
 .../planner/logical/PreProcessLogicalRel.java   |  5 ++--
 .../exec/planner/sql/DrillOperatorTable.java    |  4 ++--
 .../test/TestExecutionExceptionsToClient.java   | 24 +++++++++++++++++++-
 3 files changed, 27 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/34d736e6/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/PreProcessLogicalRel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/PreProcessLogicalRel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/PreProcessLogicalRel.java
index 10c131d..1230498 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/PreProcessLogicalRel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/PreProcessLogicalRel.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -28,7 +28,6 @@ import org.apache.calcite.rex.RexUtil;
 import org.apache.drill.common.exceptions.UserException;
 import org.apache.drill.exec.exception.UnsupportedOperatorCollector;
 import org.apache.drill.exec.planner.StarColumnHelper;
-import org.apache.drill.exec.planner.sql.DrillCalciteSqlWrapper;
 import org.apache.drill.exec.planner.sql.DrillOperatorTable;
 import org.apache.drill.exec.planner.sql.parser.DrillCalciteWrapperUtility;
 import org.apache.drill.exec.util.ApproximateStringMatcher;
@@ -240,7 +239,7 @@ public class PreProcessLogicalRel extends RelShuttleImpl {
         ops.add(op.getName());
       }
       final String bestMatch = ApproximateStringMatcher.getBestMatch(ops, newFunctionName);
-      if (bestMatch != null && bestMatch.length() > 0 && bestMatch.toLowerCase().startsWith("convert")) {
+      if (bestMatch != null && bestMatch.length() > functionName.length() && bestMatch.toLowerCase().startsWith("convert")) {
         final StringBuilder s = new StringBuilder("Did you mean ")
                 .append(bestMatch.substring(functionName.length()))
                 .append("?");

http://git-wip-us.apache.org/repos/asf/drill/blob/34d736e6/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillOperatorTable.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillOperatorTable.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillOperatorTable.java
index 5102ae8..cf858d3 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillOperatorTable.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillOperatorTable.java
@@ -148,9 +148,9 @@ public class DrillOperatorTable extends SqlStdOperatorTable {
     final List<SqlOperator> sqlOperators = Lists.newArrayList();
     sqlOperators.addAll(calciteOperators);
     if(isInferenceEnabled()) {
-      sqlOperators.addAll(calciteOperators);
+      sqlOperators.addAll(drillOperatorsWithInference);
     } else {
-      sqlOperators.addAll(calciteOperators);
+      sqlOperators.addAll(drillOperatorsWithoutInference);
     }
 
     return sqlOperators;

http://git-wip-us.apache.org/repos/asf/drill/blob/34d736e6/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/TestExecutionExceptionsToClient.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/TestExecutionExceptionsToClient.java b/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/TestExecutionExceptionsToClient.java
index 749312f..1eaf6d2 100644
--- a/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/TestExecutionExceptionsToClient.java
+++ b/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/TestExecutionExceptionsToClient.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -19,6 +19,7 @@ package org.apache.drill.jdbc.test;
 
 import static org.junit.Assert.assertThat;
 import static org.hamcrest.CoreMatchers.*;
+import static org.junit.Assert.assertTrue;
 
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -192,4 +193,25 @@ public class TestExecutionExceptionsToClient extends JdbcTestBase {
       throw e;
     }
   }
+
+  @Test(expected = SQLException.class)
+  public void testConvertFromError() throws Exception {
+    final Statement statement = connection.createStatement();
+    try {
+      statement.executeUpdate("select CONVERT_FROM('1','INTEGER') from (values(1))");
+    } catch (SQLException e) {
+      assertThat("Null getCause(); missing expected wrapped exception",
+        e.getCause(), notNullValue());
+
+      assertThat("Unexpectedly wrapped another SQLException",
+        e.getCause(), not(instanceOf(SQLException.class)));
+
+      assertThat("getCause() not UserRemoteException as expected",
+        e.getCause(), instanceOf(UserRemoteException.class));
+
+      assertTrue("No expected current \"UNSUPPORTED_OPERATION ERROR\" and/or \"Did you mean\"",
+        e.getMessage().matches("^UNSUPPORTED_OPERATION ERROR(.|\\n)*Did you mean(.|\\n)*"));
+      throw e;
+    }
+  }
 }


[3/4] drill git commit: DRILL-4511: Add unit tests for "Table does not exist" situation in case of empty directory or incorrect table name

Posted by ar...@apache.org.
DRILL-4511: Add unit tests for "Table does not exist" situation in case of empty directory or incorrect table name

closes #869


Project: http://git-wip-us.apache.org/repos/asf/drill/repo
Commit: http://git-wip-us.apache.org/repos/asf/drill/commit/4e5a0cb9
Tree: http://git-wip-us.apache.org/repos/asf/drill/tree/4e5a0cb9
Diff: http://git-wip-us.apache.org/repos/asf/drill/diff/4e5a0cb9

Branch: refs/heads/master
Commit: 4e5a0cb96f7fc0c725bd4e5b5aac1342170531ae
Parents: 34d736e
Author: Roman Kulyk <ro...@gmail.com>
Authored: Thu Jul 6 18:27:34 2017 +0300
Committer: Arina Ielchiieva <ar...@gmail.com>
Committed: Fri Jul 21 15:48:13 2017 +0300

----------------------------------------------------------------------
 .../store/parquet/TestParquetMetadataCache.java | 28 ++++++++++++++++++++
 1 file changed, 28 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/4e5a0cb9/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetMetadataCache.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetMetadataCache.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetMetadataCache.java
index dff2e86..edb3bd8 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetMetadataCache.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetMetadataCache.java
@@ -207,6 +207,34 @@ public class TestParquetMetadataCache extends PlanTestBase {
         .go();
   }
 
+  @Test //DRILL-4511
+  public void testTableDoesNotExistWithEmptyDirectory() throws Exception {
+    File path = new File(getTempDir("empty_directory"));
+    String pathString = path.toURI().getPath();
+    try {
+      path.mkdir();
+      testBuilder()
+          .sqlQuery("refresh table metadata dfs.`%s`", pathString)
+          .unOrdered()
+          .baselineColumns("ok", "summary")
+          .baselineValues(false, String.format("Table %s does not exist.", pathString))
+          .go();
+    } finally {
+      FileUtils.deleteQuietly(path);
+    }
+  }
+
+  @Test //DRILL-4511
+  public void testTableDoesNotExistWithIncorrectTableName() throws Exception {
+    String tableName = "incorrect_table";
+    testBuilder()
+        .sqlQuery("refresh table metadata dfs.`%s`", tableName)
+        .unOrdered()
+        .baselineColumns("ok", "summary")
+        .baselineValues(false, String.format("Table %s does not exist.", tableName))
+        .go();
+  }
+
   @Test
   public void testNoSupportedError() throws Exception {
     testBuilder()


[4/4] drill git commit: DRILL-5678: Undefined behavior due to un-initialized values in ServerMetaContext

Posted by ar...@apache.org.
DRILL-5678: Undefined behavior due to un-initialized values in ServerMetaContext

closes #880


Project: http://git-wip-us.apache.org/repos/asf/drill/repo
Commit: http://git-wip-us.apache.org/repos/asf/drill/commit/07346c78
Tree: http://git-wip-us.apache.org/repos/asf/drill/tree/07346c78
Diff: http://git-wip-us.apache.org/repos/asf/drill/diff/07346c78

Branch: refs/heads/master
Commit: 07346c782102205091f7e93a7fe566fd9019cbea
Parents: 4e5a0cb
Author: Rob Wu <ro...@gmail.com>
Authored: Tue Jul 18 22:55:52 2017 -0700
Committer: Arina Ielchiieva <ar...@gmail.com>
Committed: Fri Jul 21 15:48:27 2017 +0300

----------------------------------------------------------------------
 contrib/native/client/src/clientlib/drillClientImpl.cpp | 4 ++++
 1 file changed, 4 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/07346c78/contrib/native/client/src/clientlib/drillClientImpl.cpp
----------------------------------------------------------------------
diff --git a/contrib/native/client/src/clientlib/drillClientImpl.cpp b/contrib/native/client/src/clientlib/drillClientImpl.cpp
index c5ef1a2..1ccc29f 100644
--- a/contrib/native/client/src/clientlib/drillClientImpl.cpp
+++ b/contrib/native/client/src/clientlib/drillClientImpl.cpp
@@ -2307,6 +2307,10 @@ namespace { // anonymous
 namespace { // anonymous
 // Helper class to wait on ServerMeta results
 struct ServerMetaContext {
+    ServerMetaContext() : m_done(false), m_status(QRY_FAILURE) 
+    {
+        ; // Do nothing.
+    }
 	bool m_done;
 	status_t m_status;
 	exec::user::ServerMeta m_serverMeta;