You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by su...@apache.org on 2016/11/05 01:01:29 UTC

[3/4] drill git commit: DRILL-4674: Allow casting to boolean the same literals as in Postgres

DRILL-4674: Allow casting to boolean the same literals as in Postgres

closes #610


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

Branch: refs/heads/master
Commit: 0672d49b6d731566d3ad8b1594336e6842e65e98
Parents: 3fd6938
Author: Arina Ielchiieva <ar...@gmail.com>
Authored: Tue Oct 11 19:30:14 2016 +0300
Committer: Sudheesh Katkam <sk...@maprtech.com>
Committed: Fri Nov 4 17:17:06 2016 -0700

----------------------------------------------------------------------
 .../org/apache/drill/exec/expr/BooleanType.java |  79 +++++++++++
 .../exec/expr/fn/impl/SimpleCastFunctions.java  |  19 +--
 .../drill/exec/sql/TestSimpleCastFunctions.java | 135 ++++++++++++++++++-
 3 files changed, 214 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/0672d49b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/BooleanType.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/BooleanType.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/BooleanType.java
new file mode 100644
index 0000000..6ffde9f
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/BooleanType.java
@@ -0,0 +1,79 @@
+/**
+ * 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;
+
+import org.apache.drill.common.map.CaseInsensitiveMap;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Enum that contains two boolean types: TRUE and FALSE.
+ * Each has numeric representation and list of allowed literals.
+ * List of literals if formed according to
+ * {@link <a href="https://www.postgresql.org/docs/9.6/static/datatype-boolean.html">Postgre Documentation</a>}
+ */
+public enum BooleanType {
+  TRUE(1, Arrays.asList("t", "true", "y", "yes", "on", "1")),
+  FALSE(0, Arrays.asList("f", "false", "n", "no", "off", "0"));
+
+  private final int numericValue;
+  private final List<String> literals;
+
+  BooleanType(int numericValue, List<String> literals) {
+    this.numericValue = numericValue;
+    this.literals = literals;
+  }
+
+  public int getNumericValue() {
+    return numericValue;
+  }
+
+  public List<String> getLiterals() {
+    return literals;
+  }
+
+  /** Contains all literals that are allowed to represent boolean type. */
+  private static final Map<String, BooleanType> allLiterals = CaseInsensitiveMap.newHashMap();
+  static {
+    for (BooleanType booleanType : BooleanType.values()) {
+      for (String literal : booleanType.getLiterals()) {
+        allLiterals.put(literal, booleanType);
+      }
+    }
+  }
+
+  /**
+   * Finds boolean by passed literal.
+   * Leading or trailing whitespace is ignored, and case does not matter.
+   *
+   * @param literal boolean string representation
+   * @return boolean type
+   * @throws IllegalArgumentException if boolean type is not found
+   */
+  public static BooleanType get(String literal) {
+    final String value = literal.trim();
+    final BooleanType booleanType = allLiterals.get(value);
+    if (booleanType == null) {
+      throw new IllegalArgumentException("Invalid value for boolean: " + literal);
+    }
+    return booleanType;
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/drill/blob/0672d49b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/SimpleCastFunctions.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/SimpleCastFunctions.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/SimpleCastFunctions.java
index 2c04afc..346190a 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/SimpleCastFunctions.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/SimpleCastFunctions.java
@@ -30,10 +30,8 @@ import org.apache.drill.exec.expr.holders.BigIntHolder;
 import org.apache.drill.exec.expr.holders.BitHolder;
 import org.apache.drill.exec.expr.holders.VarCharHolder;
 
-public class SimpleCastFunctions {
-  public static final byte[] TRUE = {'t','r','u','e'};
-  public static final byte[] FALSE = {'f','a','l','s','e'};
 
+public class SimpleCastFunctions {
 
   @FunctionTemplate(names = {"castBIT", "castBOOLEAN"}, scope = FunctionTemplate.FunctionScope.SIMPLE, nulls=NullHandling.NULL_IF_NULL)
   public static class CastVarCharBoolean implements DrillSimpleFunc {
@@ -48,14 +46,8 @@ public class SimpleCastFunctions {
     public void eval() {
       byte[] buf = new byte[in.end - in.start];
       in.buffer.getBytes(in.start, buf, 0, in.end - in.start);
-      String input = new String(buf, com.google.common.base.Charsets.UTF_8).toLowerCase();
-      if ("true".equals(input)) {
-        out.value = 1;
-      } else if ("false".equals(input)) {
-        out.value = 0;
-      } else {
-        throw new IllegalArgumentException("Invalid value for boolean: " + input);
-      }
+      String input = new String(buf, com.google.common.base.Charsets.UTF_8);
+      out.value = org.apache.drill.exec.expr.BooleanType.get(input).getNumericValue();
     }
   }
 
@@ -70,12 +62,11 @@ public class SimpleCastFunctions {
     public void setup() {}
 
     public void eval() {
-      byte[] outB = in.value == 1 ? org.apache.drill.exec.expr.fn.impl.SimpleCastFunctions.TRUE : org.apache.drill.exec.expr.fn.impl.SimpleCastFunctions.FALSE;
+      byte[] outB = org.apache.drill.exec.expr.BooleanType.get(String.valueOf(in.value)).name().toLowerCase().getBytes();
       buffer.setBytes(0, outB);
       out.buffer = buffer;
       out.start = 0;
       out.end = Math.min((int)len.value, outB.length); // truncate if target type has length smaller than that of input's string
     }
   }
-
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/0672d49b/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestSimpleCastFunctions.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestSimpleCastFunctions.java b/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestSimpleCastFunctions.java
index 4dae7fe..c5af557 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestSimpleCastFunctions.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestSimpleCastFunctions.java
@@ -17,19 +17,144 @@
  */
 package org.apache.drill.exec.sql;
 
+import com.google.common.base.Function;
+import com.google.common.collect.Lists;
 import org.apache.drill.BaseTestQuery;
+import org.apache.drill.common.exceptions.UserRemoteException;
 import org.junit.Test;
 
+import javax.annotation.Nullable;
+import java.util.Arrays;
+import java.util.List;
+
+import static org.hamcrest.CoreMatchers.containsString;
+import static org.junit.Assert.assertThat;
+
 public class TestSimpleCastFunctions extends BaseTestQuery {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestSimpleCastFunctions.class);
+
+  private static final List<Function<String, String>> inputFunctions = Lists.newArrayList();
+
+  static {
+    inputFunctions.add(new Function<String, String>() {
+      @Nullable
+      @Override
+      public String apply(String input) {
+        return input.toLowerCase();
+      }
+    });
+
+    inputFunctions.add(new Function<String, String>() {
+      @Nullable
+      @Override
+      public String apply(String input) {
+        return input.toUpperCase();
+      }
+    });
+
+    inputFunctions.add(new Function<String, String>() {
+      @Nullable
+      @Override
+      public String apply(String input) {
+        return "   " + input + "   ";
+      }
+    });
+  }
+
+  @Test
+  public void testCastFromBooleanToString() throws Exception {
+    testBuilder()
+        .sqlQuery("select" +
+            " cast(false as varchar(5)) c1," +
+            " cast(true as varchar(4)) c2," +
+            " cast((1 < 5) as varchar(4)) c3," +
+            " cast((1 > 5) as varchar(5)) c4" +
+            " from (values(1))")
+        .unOrdered()
+        .baselineColumns("c1", "c2", "c3", "c4")
+        .baselineValues("false", "true", "true", "false")
+        .go();
+  }
+
+  @Test
+  public void testCastStringToBooleanTrueValue() throws Exception {
+    List<String> literals =  Arrays.asList("t", "true", "y", "yes", "on", "1");
+    String query = "select cast('%s' as boolean) b_val from (values(1))";
+    for (String literal : literals) {
+      for (Function<String, String> function : inputFunctions) {
+        testBuilder()
+            .sqlQuery(query, function.apply(literal))
+            .unOrdered()
+            .baselineColumns("b_val")
+            .baselineValues(true)
+            .go();
+      }
+    }
+  }
+
+  @Test
+  public void testCastStringToBooleanFalseValue() throws Exception {
+    List<String> literals =  Arrays.asList("f", "false", "n", "no", "off", "0");
+    String query = "select cast('%s' as boolean) b_val from (values(1))";
+    for (String literal : literals) {
+      for (Function<String, String> function : inputFunctions) {
+        testBuilder()
+            .sqlQuery(query, function.apply(literal))
+            .unOrdered()
+            .baselineColumns("b_val")
+            .baselineValues(false)
+            .go();
+      }
+    }
+  }
+
+  @Test
+  public void testCastNumericToBooleanTrueValue() throws Exception {
+    testBuilder()
+        .sqlQuery("select cast(1 as boolean) b_val from (values(1))")
+        .unOrdered()
+        .baselineColumns("b_val")
+        .baselineValues(true)
+        .go();
+  }
 
   @Test
-  public void castFromBoolean() throws Exception {
-    test("select cast(false as varchar(5)), cast(true as varchar(4)), cast((1 < 5) as varchar(4)) from sys.options limit 1;");
+  public void testCastNumericToBooleanFalseValue() throws Exception {
+    testBuilder()
+        .sqlQuery("select cast(0 as boolean) b_val from (values(1))")
+        .unOrdered()
+        .baselineColumns("b_val")
+        .baselineValues(false)
+        .go();
   }
 
   @Test
-  public void castToBoolean() throws Exception {
-    test("select cast('false' as boolean), cast('true' as boolean) from sys.options limit 1;");
+  public void testCastNullToBoolean() throws Exception {
+    testBuilder()
+        .sqlQuery("select cast(null as boolean) b_val from (values(1))")
+        .unOrdered()
+        .baselineColumns("b_val")
+        .baselineValues((String) null)
+        .go();
+  }
+
+  @Test(expected = UserRemoteException.class)
+  public void testIncorrectStringBoolean() throws Exception {
+    try {
+      test("select cast('A' as boolean) b_val from (values(1))");
+    } catch (UserRemoteException e) {
+      assertThat(e.getMessage(), containsString("Invalid value for boolean: A"));
+      throw e;
+    }
   }
+
+  @Test(expected = UserRemoteException.class)
+  public void testIncorrectNumericBoolean() throws Exception {
+    try {
+      test("select cast(123 as boolean) b_val from (values(1))");
+    } catch (UserRemoteException e) {
+      assertThat(e.getMessage(), containsString("Invalid value for boolean: 123"));
+      throw e;
+    }
+  }
+
 }