You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@calcite.apache.org by hy...@apache.org on 2020/08/23 18:38:02 UTC

[calcite] branch master updated: [CALCITE-3782] Bitwise functions BIT_AND, BIT_OR and BIT_XOR support binary and varbinary type (Hailong Wang)

This is an automated email from the ASF dual-hosted git repository.

hyuan pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/calcite.git


The following commit(s) were added to refs/heads/master by this push:
     new 9cf829b  [CALCITE-3782] Bitwise functions BIT_AND, BIT_OR and BIT_XOR support binary and varbinary type (Hailong Wang)
9cf829b is described below

commit 9cf829bc28113ba01f3ee0c826b502143478680e
Author: wangxlong <18...@163.com>
AuthorDate: Fri Mar 27 16:24:56 2020 +0800

    [CALCITE-3782] Bitwise functions BIT_AND, BIT_OR and BIT_XOR support binary and varbinary type (Hailong Wang)
    
    Close #1878
---
 .../calcite/adapter/enumerable/RexImpTable.java    | 10 +++-
 .../apache/calcite/runtime/CalciteResource.java    |  3 ++
 .../org/apache/calcite/runtime/SqlFunctions.java   | 55 +++++++++++++++++++---
 .../calcite/sql/fun/SqlBitOpAggFunction.java       | 19 ++++++--
 .../calcite/runtime/CalciteResource.properties     |  1 +
 .../apache/calcite/sql/test/AbstractSqlTester.java | 16 +++++++
 .../calcite/sql/test/SqlOperatorBaseTest.java      | 47 ++++++++++++++++--
 .../apache/calcite/sql/test/SqlRuntimeTester.java  | 10 ++++
 .../org/apache/calcite/sql/test/SqlTester.java     | 14 ++++++
 site/_docs/reference.md                            |  6 +--
 10 files changed, 162 insertions(+), 19 deletions(-)

diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/RexImpTable.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/RexImpTable.java
index 20a86a3..176ed1a 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/RexImpTable.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/RexImpTable.java
@@ -17,6 +17,7 @@
 package org.apache.calcite.adapter.enumerable;
 
 import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.avatica.util.ByteString;
 import org.apache.calcite.avatica.util.DateTimeUtils;
 import org.apache.calcite.avatica.util.TimeUnit;
 import org.apache.calcite.avatica.util.TimeUnitRange;
@@ -1231,8 +1232,13 @@ public class RexImpTable {
   static class BitOpImplementor extends StrictAggImplementor {
     @Override protected void implementNotNullReset(AggContext info,
         AggResetContext reset) {
-      Object initValue = info.aggregation() == BIT_AND ? -1 : 0;
-      Expression start = Expressions.constant(initValue, info.returnType());
+      Expression start;
+      if (SqlTypeUtil.isBinary(info.returnRelType())) {
+        start = Expressions.field(null, ByteString.class, "EMPTY");
+      } else {
+        Object initValue = info.aggregation() == BIT_AND ? -1L : 0;
+        start = Expressions.constant(initValue, info.returnType());
+      }
 
       reset.currentBlock().add(
           Expressions.statement(
diff --git a/core/src/main/java/org/apache/calcite/runtime/CalciteResource.java b/core/src/main/java/org/apache/calcite/runtime/CalciteResource.java
index 3aaec91..fa13471 100644
--- a/core/src/main/java/org/apache/calcite/runtime/CalciteResource.java
+++ b/core/src/main/java/org/apache/calcite/runtime/CalciteResource.java
@@ -928,4 +928,7 @@ public interface CalciteResource {
 
   @BaseMessage("Invalid input for EXTRACTVALUE: xml: ''{0}'', xpath expression: ''{1}''")
   ExInst<CalciteException> invalidInputForExtractValue(String xml, String xpath);
+
+  @BaseMessage("Different length for bitwise operands: the first: {0,number,#}, the second: {1,number,#}")
+  ExInst<CalciteException> differentLengthForBitwiseOperands(int l0, int l1);
 }
diff --git a/core/src/main/java/org/apache/calcite/runtime/SqlFunctions.java b/core/src/main/java/org/apache/calcite/runtime/SqlFunctions.java
index 608af6f..2c55290 100644
--- a/core/src/main/java/org/apache/calcite/runtime/SqlFunctions.java
+++ b/core/src/main/java/org/apache/calcite/runtime/SqlFunctions.java
@@ -73,6 +73,7 @@ import java.util.Objects;
 import java.util.Set;
 import java.util.TimeZone;
 import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.BinaryOperator;
 import java.util.regex.Pattern;
 import javax.annotation.Nonnull;
 
@@ -1109,24 +1110,66 @@ public class SqlFunctions {
         op, b1.getClass().toString()).ex();
   }
 
-  // &
-  /** Helper function for implementing <code>BIT_AND</code>. */
+  /** Bitwise function <code>BIT_AND</code> applied to integer values. */
   public static long bitAnd(long b0, long b1) {
     return b0 & b1;
   }
 
-  // |
-  /** Helper function for implementing <code>BIT_OR</code>. */
+  /** Bitwise function <code>BIT_AND</code> applied to binary values. */
+  public static ByteString bitAnd(ByteString b0, ByteString b1) {
+    return binaryOperator(b0, b1, (x, y) -> (byte) (x & y));
+  }
+
+  /** Bitwise function <code>BIT_OR</code> applied to integer values. */
   public static long bitOr(long b0, long b1) {
     return b0 | b1;
   }
 
-  // ^
-  /** Helper function for implementing <code>BIT_XOR</code>. */
+  /** Bitwise function <code>BIT_OR</code> applied to binary values. */
+  public static ByteString bitOr(ByteString b0, ByteString b1) {
+    return binaryOperator(b0, b1, (x, y) -> (byte) (x | y));
+  }
+
+  /** Bitwise function <code>BIT_XOR</code> applied to integer values. */
   public static long bitXor(long b0, long b1) {
     return b0 ^ b1;
   }
 
+  /** Bitwise function <code>BIT_XOR</code> applied to binary values. */
+  public static ByteString bitXor(ByteString b0, ByteString b1) {
+    return binaryOperator(b0, b1, (x, y) -> (byte) (x ^ y));
+  }
+
+  /**
+   * Utility for bitwise function applied to two byteString values.
+   *
+   * @param b0 The first byteString value operand of bitwise function.
+   * @param b1 The second byteString value operand of bitwise function.
+   * @param bitOp BitWise binary operator.
+   * @return ByteString after bitwise operation.
+   */
+  private static ByteString binaryOperator(
+      ByteString b0, ByteString b1, BinaryOperator<Byte> bitOp) {
+    if (b0.length() == 0) {
+      return b1;
+    }
+    if (b1.length() == 0) {
+      return b0;
+    }
+
+    if (b0.length() != b1.length()) {
+      throw RESOURCE.differentLengthForBitwiseOperands(
+          b0.length(), b1.length()).ex();
+    }
+
+    final byte[] result = new byte[b0.length()];
+    for (int i = 0; i < b0.length(); i++) {
+      result[i] = bitOp.apply(b0.byteAt(i), b1.byteAt(i));
+    }
+
+    return new ByteString(result);
+  }
+
   // EXP
 
   /** SQL <code>EXP</code> operator applied to double values. */
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlBitOpAggFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlBitOpAggFunction.java
index 38116fd..cc3db36 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlBitOpAggFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlBitOpAggFunction.java
@@ -30,8 +30,8 @@ import com.google.common.base.Preconditions;
  * Definition of the <code>BIT_AND</code> and <code>BIT_OR</code> aggregate functions,
  * returning the bitwise AND/OR of all non-null input values, or null if none.
  *
- * <p>Only INTEGER types are supported:
- * tinyint, smallint, int, bigint
+ * <p>INTEGER and BINARY types are supported:
+ * tinyint, smallint, int, bigint, binary, varbinary
  */
 public class SqlBitOpAggFunction extends SqlAggFunction {
 
@@ -44,7 +44,7 @@ public class SqlBitOpAggFunction extends SqlAggFunction {
         kind,
         ReturnTypes.ARG0_NULLABLE_IF_EMPTY,
         null,
-        OperandTypes.INTEGER,
+        OperandTypes.or(OperandTypes.INTEGER, OperandTypes.BINARY),
         SqlFunctionCategory.NUMERIC,
         false,
         false,
@@ -62,6 +62,17 @@ public class SqlBitOpAggFunction extends SqlAggFunction {
   }
 
   @Override public Optionality getDistinctOptionality() {
-    return Optionality.IGNORED;
+    final Optionality optionality;
+
+    switch (kind) {
+    case BIT_AND:
+    case BIT_OR:
+      optionality = Optionality.IGNORED;
+      break;
+    default:
+      optionality = Optionality.OPTIONAL;
+      break;
+    }
+    return optionality;
   }
 }
diff --git a/core/src/main/resources/org/apache/calcite/runtime/CalciteResource.properties b/core/src/main/resources/org/apache/calcite/runtime/CalciteResource.properties
index 5a91208..40ac80c 100644
--- a/core/src/main/resources/org/apache/calcite/runtime/CalciteResource.properties
+++ b/core/src/main/resources/org/apache/calcite/runtime/CalciteResource.properties
@@ -304,4 +304,5 @@ InvalidInputForXmlTransform=Invalid input for XMLTRANSFORM xml: ''{0}''
 InvalidInputForExtractValue=Invalid input for EXTRACTVALUE: xml: ''{0}'', xpath expression: ''{1}''
 InvalidInputForExtractXml=Invalid input for EXTRACT xpath: ''{0}'', namespace: ''{1}''
 InvalidInputForExistsNode=Invalid input for EXISTSNODE xpath: ''{0}'', namespace: ''{1}''
+DifferentLengthForBitwiseOperands=Different length for bitwise operands: the first: {0,number,#}, the second: {1,number,#}
 # End CalciteResource.properties
diff --git a/core/src/test/java/org/apache/calcite/sql/test/AbstractSqlTester.java b/core/src/test/java/org/apache/calcite/sql/test/AbstractSqlTester.java
index 4a6bbf1..beef0ba 100644
--- a/core/src/test/java/org/apache/calcite/sql/test/AbstractSqlTester.java
+++ b/core/src/test/java/org/apache/calcite/sql/test/AbstractSqlTester.java
@@ -526,6 +526,22 @@ public abstract class AbstractSqlTester implements SqlTester, AutoCloseable {
     assertExceptionIsThrown(sql, expectedError);
   }
 
+  @Override public void checkAggFails(
+      String expr,
+      String[] inputValues,
+      String expectedError,
+      boolean runtime) {
+    final String sql =
+        SqlTests.generateAggQuery(expr, inputValues);
+    if (runtime) {
+      SqlValidator validator = getValidator();
+      SqlNode n = parseAndValidate(validator, sql);
+      assertNotNull(n);
+    } else {
+      checkQueryFails(sql, expectedError);
+    }
+  }
+
   public void checkQuery(String sql) {
     assertExceptionIsThrown(sql, null);
   }
diff --git a/core/src/test/java/org/apache/calcite/sql/test/SqlOperatorBaseTest.java b/core/src/test/java/org/apache/calcite/sql/test/SqlOperatorBaseTest.java
index f01f83b..1d3f834 100644
--- a/core/src/test/java/org/apache/calcite/sql/test/SqlOperatorBaseTest.java
+++ b/core/src/test/java/org/apache/calcite/sql/test/SqlOperatorBaseTest.java
@@ -9142,8 +9142,10 @@ public abstract class SqlOperatorBaseTest {
     tester.checkType("bit_and(CAST(2 AS TINYINT))", "TINYINT");
     tester.checkType("bit_and(CAST(2 AS SMALLINT))", "SMALLINT");
     tester.checkType("bit_and(distinct CAST(2 AS BIGINT))", "BIGINT");
+    tester.checkType("bit_and(CAST(x'02' AS BINARY(1)))", "BINARY(1)");
     tester.checkFails("^bit_and(1.2)^",
-        "Cannot apply 'BIT_AND' to arguments of type 'BIT_AND\\(<DECIMAL\\(2, 1\\)>\\)'\\. Supported form\\(s\\): 'BIT_AND\\(<INTEGER>\\)'",
+        "Cannot apply 'BIT_AND' to arguments of type 'BIT_AND\\(<DECIMAL\\(2, 1\\)>\\)'\\. Supported form\\(s\\): 'BIT_AND\\(<INTEGER>\\)'\n"
+            + "'BIT_AND\\(<BINARY>\\)'",
         false);
     tester.checkFails(
         "^bit_and()^",
@@ -9154,7 +9156,24 @@ public abstract class SqlOperatorBaseTest {
         "Invalid number of arguments to function 'BIT_AND'. Was expecting 1 arguments",
         false);
     final String[] values = {"3", "2", "2"};
-    tester.checkAgg("bit_and(x)", values, 2, 0);
+    tester.checkAgg("bit_and(x)", values, "2", 0);
+    final String[] binaryValues = {
+        "CAST(x'03' AS BINARY)",
+        "cast(x'02' as BINARY)",
+        "cast(x'02' AS BINARY)",
+        "cast(null AS BINARY)"};
+    tester.checkAgg("bit_and(x)", binaryValues, "02", 0);
+    tester.checkAgg("bit_and(x)", new String[]{"CAST(x'02' AS BINARY)"}, "02", 0);
+
+    tester.checkAggFails(
+        "bit_and(x)",
+        new String[]{"CAST(x'0201' AS VARBINARY)", "CAST(x'02' AS VARBINARY)"},
+        "Error while executing SQL"
+          +  " \"SELECT bit_and\\(x\\)"
+          +  " FROM \\(SELECT CAST\\(x'0201' AS VARBINARY\\) AS x FROM \\(VALUES \\(1\\)\\)"
+          + " UNION ALL SELECT CAST\\(x'02' AS VARBINARY\\) AS x FROM \\(VALUES \\(1\\)\\)\\)\":"
+          + " Different length for bitwise operands: the first: 2, the second: 1",
+        true);
   }
 
   @Test void testBitOrFunc() {
@@ -9164,8 +9183,10 @@ public abstract class SqlOperatorBaseTest {
     tester.checkType("bit_or(CAST(2 AS TINYINT))", "TINYINT");
     tester.checkType("bit_or(CAST(2 AS SMALLINT))", "SMALLINT");
     tester.checkType("bit_or(distinct CAST(2 AS BIGINT))", "BIGINT");
+    tester.checkType("bit_or(CAST(x'02' AS BINARY(1)))", "BINARY(1)");
     tester.checkFails("^bit_or(1.2)^",
-        "Cannot apply 'BIT_OR' to arguments of type 'BIT_OR\\(<DECIMAL\\(2, 1\\)>\\)'\\. Supported form\\(s\\): 'BIT_OR\\(<INTEGER>\\)'",
+        "Cannot apply 'BIT_OR' to arguments of type 'BIT_OR\\(<DECIMAL\\(2, 1\\)>\\)'\\. Supported form\\(s\\): 'BIT_OR\\(<INTEGER>\\)'\n"
+            + "'BIT_OR\\(<BINARY>\\)'",
         false);
     tester.checkFails(
         "^bit_or()^",
@@ -9177,6 +9198,13 @@ public abstract class SqlOperatorBaseTest {
         false);
     final String[] values = {"1", "2", "2"};
     tester.checkAgg("bit_or(x)", values, 3, 0);
+    final String[] binaryValues = {
+        "CAST(x'01' AS BINARY)",
+        "cast(x'02' as BINARY)",
+        "cast(x'02' AS BINARY)",
+        "cast(null AS BINARY)"};
+    tester.checkAgg("bit_or(x)", binaryValues, "03", 0);
+    tester.checkAgg("bit_or(x)", new String[]{"CAST(x'02' AS BINARY)"}, "02", 0);
   }
 
   @Test void testBitXorFunc() {
@@ -9186,8 +9214,10 @@ public abstract class SqlOperatorBaseTest {
     tester.checkType("bit_xor(CAST(2 AS TINYINT))", "TINYINT");
     tester.checkType("bit_xor(CAST(2 AS SMALLINT))", "SMALLINT");
     tester.checkType("bit_xor(distinct CAST(2 AS BIGINT))", "BIGINT");
+    tester.checkType("bit_xor(CAST(x'02' AS BINARY(1)))", "BINARY(1)");
     tester.checkFails("^bit_xor(1.2)^",
-        "Cannot apply 'BIT_XOR' to arguments of type 'BIT_XOR\\(<DECIMAL\\(2, 1\\)>\\)'\\. Supported form\\(s\\): 'BIT_XOR\\(<INTEGER>\\)'",
+        "Cannot apply 'BIT_XOR' to arguments of type 'BIT_XOR\\(<DECIMAL\\(2, 1\\)>\\)'\\. Supported form\\(s\\): 'BIT_XOR\\(<INTEGER>\\)'\n"
+            + "'BIT_XOR\\(<BINARY>\\)'",
         false);
     tester.checkFails(
         "^bit_xor()^",
@@ -9199,6 +9229,15 @@ public abstract class SqlOperatorBaseTest {
         false);
     final String[] values = {"1", "2", "1"};
     tester.checkAgg("bit_xor(x)", values, 2, 0);
+    final String[] binaryValues = {
+        "CAST(x'01' AS BINARY)",
+        "cast(x'02' as BINARY)",
+        "cast(x'01' AS BINARY)",
+        "cast(null AS BINARY)"};
+    tester.checkAgg("bit_xor(x)", binaryValues, "02", 0);
+    tester.checkAgg("bit_xor(x)", new String[]{"CAST(x'02' AS BINARY)"}, "02", 0);
+    tester.checkAgg("bit_xor(distinct(x))",
+        new String[]{"CAST(x'02' AS BINARY)", "CAST(x'02' AS BINARY)"}, "02", 0);
   }
 
   /**
diff --git a/core/src/test/java/org/apache/calcite/sql/test/SqlRuntimeTester.java b/core/src/test/java/org/apache/calcite/sql/test/SqlRuntimeTester.java
index cd87c79..1cbe882 100644
--- a/core/src/test/java/org/apache/calcite/sql/test/SqlRuntimeTester.java
+++ b/core/src/test/java/org/apache/calcite/sql/test/SqlRuntimeTester.java
@@ -50,6 +50,16 @@ class SqlRuntimeTester extends AbstractSqlTester {
     assertExceptionIsThrown(sql, expectedError, runtime);
   }
 
+  @Override public void checkAggFails(
+      String expr,
+      String[] inputValues,
+      String expectedError,
+      boolean runtime) {
+    String query =
+        SqlTests.generateAggQuery(expr, inputValues);
+    assertExceptionIsThrown(query, expectedError, runtime);
+  }
+
   public void assertExceptionIsThrown(
       String sql,
       String expectedMsgPattern) {
diff --git a/core/src/test/java/org/apache/calcite/sql/test/SqlTester.java b/core/src/test/java/org/apache/calcite/sql/test/SqlTester.java
index a3479b7..a2f9944 100644
--- a/core/src/test/java/org/apache/calcite/sql/test/SqlTester.java
+++ b/core/src/test/java/org/apache/calcite/sql/test/SqlTester.java
@@ -371,6 +371,20 @@ public interface SqlTester extends AutoCloseable, SqlValidatorTestCase.Tester {
       double delta);
 
   /**
+   * Tests that an aggregate expression fails at run time.
+   * @param expr An aggregate expression
+   * @param inputValues Array of input values
+   * @param expectedError Pattern for expected error
+   * @param runtime       If true, must fail at runtime; if false, must fail at
+   *                      validate time
+   */
+  void checkAggFails(
+      String expr,
+      String[] inputValues,
+      String expectedError,
+      boolean runtime);
+
+  /**
    * Tests that a scalar SQL expression fails at run time.
    *
    * @param expression    SQL scalar expression
diff --git a/site/_docs/reference.md b/site/_docs/reference.md
index 982c797..2428525 100644
--- a/site/_docs/reference.md
+++ b/site/_docs/reference.md
@@ -1776,9 +1776,9 @@ and `LISTAGG`).
 | ANY_VALUE( [ ALL &#124; DISTINCT ] value)     | Returns one of the values of *value* across all input values; this is NOT specified in the SQL standard
 | SOME(condition)                               | Returns true if any condition is true.
 | EVERY(condition)                              | Returns true if all conditions are true.
-| BIT_AND( [ ALL &#124; DISTINCT ] value)       | Returns the bitwise AND of all non-null input values, or null if none
-| BIT_OR( [ ALL &#124; DISTINCT ] value)        | Returns the bitwise OR of all non-null input values, or null if none
-| BIT_XOR( [ ALL &#124; DISTINCT ] value)       | Returns the bitwise XOR of all non-null input values, or null if none
+| BIT_AND( [ ALL &#124; DISTINCT ] value)       | Returns the bitwise AND of all non-null input values, or null if none; integer and binary types are supported
+| BIT_OR( [ ALL &#124; DISTINCT ] value)        | Returns the bitwise OR of all non-null input values, or null if none; integer and binary types are supported
+| BIT_XOR( [ ALL &#124; DISTINCT ] value)       | Returns the bitwise XOR of all non-null input values, or null if none; integer and binary types are supported
 | STDDEV_POP( [ ALL &#124; DISTINCT ] numeric)  | Returns the population standard deviation of *numeric* across all input values
 | STDDEV_SAMP( [ ALL &#124; DISTINCT ] numeric) | Returns the sample standard deviation of *numeric* across all input values
 | STDDEV( [ ALL &#124; DISTINCT ] numeric)      | Synonym for `STDDEV_SAMP`