You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@calcite.apache.org by jh...@apache.org on 2023/03/22 08:17:17 UTC

[calcite] branch main updated (ba975ba061 -> d2ca97e799)

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

jhyde pushed a change to branch main
in repository https://gitbox.apache.org/repos/asf/calcite.git


    from ba975ba061 [CALCITE-5476] Add DATETIME_TRUNC function (enabled in BigQuery library)
     new 8b1059ee8a Add .gitignore for Java VSCode plugin
     new 03fd95b97c [CALCITE-5548] Add MSSQL-style CONVERT function (enabled in MSSql library)
     new d2ca97e799 [CALCITE-5580] Add SPLIT function (enabled in BigQuery library)

The 3 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .gitignore                                         |  6 ++
 babel/src/test/resources/sql/big-query.iq          | 82 +++++++++++++++++++++
 core/src/main/codegen/templates/Parser.jj          | 38 ++++++++--
 .../calcite/adapter/enumerable/RexImpTable.java    |  2 +
 .../apache/calcite/runtime/CalciteResource.java    |  4 ++
 .../org/apache/calcite/runtime/SqlFunctions.java   | 46 ++++++++++++
 .../org/apache/calcite/sql/SqlBasicFunction.java   | 46 +++++++++---
 .../apache/calcite/sql/SqlWithinGroupOperator.java |  2 +-
 .../apache/calcite/sql/fun/SqlCastFunction.java    | 14 ++--
 .../apache/calcite/sql/fun/SqlConvertFunction.java |  6 +-
 .../calcite/sql/fun/SqlLibraryOperators.java       | 72 +++++++++++++++++++
 .../apache/calcite/sql/type/OperandHandlers.java   | 10 +++
 .../org/apache/calcite/sql/type/OperandTypes.java  |  9 +++
 .../calcite/runtime/CalciteResource.properties     |  1 +
 .../org/apache/calcite/test/SqlFunctionsTest.java  | 68 ++++++++++++++++++
 core/src/test/resources/sql/functions.iq           | 84 ++++++++++++++++++++++
 site/_docs/reference.md                            |  4 +-
 .../apache/calcite/sql/parser/SqlParserTest.java   | 19 +++++
 .../apache/calcite/sql/test/AbstractSqlTester.java |  3 +-
 .../java/org/apache/calcite/test/QuidemTest.java   |  5 ++
 .../org/apache/calcite/test/SqlOperatorTest.java   | 62 ++++++++++++++++
 21 files changed, 553 insertions(+), 30 deletions(-)


[calcite] 02/03: [CALCITE-5548] Add MSSQL-style CONVERT function (enabled in MSSql library)

Posted by jh...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 03fd95b97c4c63819d3af9525c06b0f40963def9
Author: askarbozcan <as...@metu.edu.tr>
AuthorDate: Thu Mar 2 18:03:01 2023 +0200

    [CALCITE-5548] Add MSSQL-style CONVERT function (enabled in MSSql library)
    
    Microsoft SQL Server's `CONVERT(type, exp [, style])`
    function is equivalent to `CAST(exp AS type)` and the JDBC
    standard function `{fn CONVERT(value, type)}`.
    
    This function is not to be confused with standard SQL's
    `CONVERT` function, which converts a character string from
    one character set to another.
    
    Add `mssqlfunc` connection for QuidemTest.
    
    Close apache/calcite#3100
---
 core/src/main/codegen/templates/Parser.jj          | 38 ++++++++--
 .../apache/calcite/sql/fun/SqlCastFunction.java    | 14 ++--
 .../apache/calcite/sql/fun/SqlConvertFunction.java |  6 +-
 .../calcite/sql/fun/SqlLibraryOperators.java       | 46 ++++++++++++
 .../apache/calcite/sql/type/OperandHandlers.java   | 10 +++
 core/src/test/resources/sql/functions.iq           | 84 ++++++++++++++++++++++
 site/_docs/reference.md                            |  3 +-
 .../apache/calcite/sql/parser/SqlParserTest.java   | 19 +++++
 .../apache/calcite/sql/test/AbstractSqlTester.java |  3 +-
 .../java/org/apache/calcite/test/QuidemTest.java   |  5 ++
 .../org/apache/calcite/test/SqlOperatorTest.java   | 28 ++++++++
 11 files changed, 238 insertions(+), 18 deletions(-)

diff --git a/core/src/main/codegen/templates/Parser.jj b/core/src/main/codegen/templates/Parser.jj
index a30408ff84..da7166b53d 100644
--- a/core/src/main/codegen/templates/Parser.jj
+++ b/core/src/main/codegen/templates/Parser.jj
@@ -5995,6 +5995,7 @@ SqlNode BuiltinFunctionCall() :
     SqlDataTypeSpec dt;
     final SqlIntervalQualifier unit;
     final SqlNode node;
+    final SqlLiteral style; // mssql convert 'style' operand
 }
 {
     //~ FUNCTIONS WITH SPECIAL SYNTAX ---------------------------------------
@@ -6036,11 +6037,38 @@ SqlNode BuiltinFunctionCall() :
     |
         <CONVERT> { s = span(); }
         <LPAREN>
-        AddExpression(args, ExprContext.ACCEPT_SUB_QUERY)
-        <USING> name = SimpleIdentifier() { args.add(name); }
-        <RPAREN> {
-            return SqlStdOperatorTable.CONVERT.createCall(s.end(this), args);
-        }
+        (
+          // CONVERT in the form of CONVERT(x USING y)
+
+          // "AddExpression" matches INTERVAL,
+          // which can also be 1st token in args of MSSQL CONVERT
+          // So lookahead another token (to match <USING> vs. <COMMA>)
+          LOOKAHEAD(2)
+          AddExpression(args, ExprContext.ACCEPT_SUB_QUERY)
+          <USING> name = SimpleIdentifier() { args.add(name); }
+          <RPAREN> {
+              return SqlStdOperatorTable.CONVERT.createCall(s.end(this), args);
+          }
+        | // mssql CONVERT(type, val [,style])
+          (
+            dt = DataType() { args.add(dt); }
+            |
+            <INTERVAL> e = IntervalQualifier() { args.add(e); }
+          )
+          <COMMA>
+          AddExpression(args, ExprContext.ACCEPT_SUB_QUERY)
+          [
+            <COMMA>
+            (
+              style = UnsignedNumericLiteral() { args.add(style); }
+              |
+              <NULL> { args.add(SqlLiteral.createNull(getPos())); }
+            )
+          ]
+          <RPAREN> {
+            return SqlLibraryOperators.MSSQL_CONVERT.createCall(s.end(this), args);
+          }
+        )
     |
         <TRANSLATE> { s = span(); }
         <LPAREN>
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlCastFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlCastFunction.java
index ce45bf7e24..d3088823ce 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlCastFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlCastFunction.java
@@ -83,9 +83,7 @@ public class SqlCastFunction extends SqlFunction {
   //~ Constructors -----------------------------------------------------------
 
   public SqlCastFunction() {
-    super("CAST",
-        SqlKind.CAST,
-        null,
+    super("CAST", SqlKind.CAST, SqlCastFunction::inferReturnTypeImpl,
         InferTypes.FIRST_KNOWN,
         null,
         SqlFunctionCategory.SYSTEM);
@@ -93,14 +91,12 @@ public class SqlCastFunction extends SqlFunction {
 
   //~ Methods ----------------------------------------------------------------
 
-  @Override public RelDataType inferReturnType(
-      SqlOperatorBinding opBinding) {
+  static RelDataType inferReturnTypeImpl(SqlOperatorBinding opBinding) {
     assert opBinding.getOperandCount() == 2;
-    RelDataType ret = opBinding.getOperandType(1);
-    RelDataType firstType = opBinding.getOperandType(0);
-    ret =
+    final RelDataType firstType = opBinding.getOperandType(0);
+    final RelDataType ret =
         opBinding.getTypeFactory().createTypeWithNullability(
-            ret,
+            opBinding.getOperandType(1),
             firstType.isNullable());
     if (opBinding instanceof SqlCallBinding) {
       SqlCallBinding callBinding = (SqlCallBinding) opBinding;
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlConvertFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlConvertFunction.java
index 23fbb4b175..f36621858f 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlConvertFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlConvertFunction.java
@@ -21,6 +21,8 @@ import org.apache.calcite.sql.SqlFunction;
 import org.apache.calcite.sql.SqlFunctionCategory;
 import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.sql.SqlWriter;
+import org.apache.calcite.sql.type.OperandTypes;
+import org.apache.calcite.sql.type.ReturnTypes;
 
 /**
  * Common base for the <code>CONVERT</code> and <code>TRANSLATE</code>
@@ -33,9 +35,9 @@ public class SqlConvertFunction extends SqlFunction {
     super(
         name,
         SqlKind.OTHER_FUNCTION,
+        ReturnTypes.ARG0,
         null,
-        null,
-        null,
+        OperandTypes.ANY_ANY,
         SqlFunctionCategory.STRING);
   }
 
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlLibraryOperators.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlLibraryOperators.java
index fe03c2b2d7..cde127a424 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlLibraryOperators.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlLibraryOperators.java
@@ -22,6 +22,7 @@ import org.apache.calcite.sql.SqlAggFunction;
 import org.apache.calcite.sql.SqlBasicFunction;
 import org.apache.calcite.sql.SqlBinaryOperator;
 import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlCallBinding;
 import org.apache.calcite.sql.SqlFunction;
 import org.apache.calcite.sql.SqlFunctionCategory;
 import org.apache.calcite.sql.SqlKind;
@@ -40,6 +41,7 @@ import org.apache.calcite.sql.type.SqlReturnTypeInference;
 import org.apache.calcite.sql.type.SqlTypeFamily;
 import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.sql.type.SqlTypeTransforms;
+import org.apache.calcite.sql.validate.SqlValidator;
 import org.apache.calcite.util.Litmus;
 import org.apache.calcite.util.Optionality;
 
@@ -123,6 +125,50 @@ public abstract class SqlLibraryOperators {
           OperandTypes.family(SqlTypeFamily.ANY, SqlTypeFamily.DATE,
               SqlTypeFamily.DATE));
 
+  /** The "CONVERT(type, expr [,style])" function (Microsoft SQL Server).
+   *
+   * <p>Syntax:
+   * <blockquote>{@code
+   * CONVERT( data_type [ ( length ) ], expression [, style ] )
+   * }</blockquote>
+   *
+   * <p>The optional "style" argument specifies how the value is going to be
+   * converted; this implementation ignores the {@code style} parameter.
+   *
+   * <p>{@code CONVERT(type, expr, style)} is equivalent to CAST(expr AS type),
+   * and the implementation delegates most of its logic to actual CAST operator.
+   *
+   * <p>Not to be confused with standard {@link SqlStdOperatorTable#CONVERT},
+   * which converts a string from one character set to another. */
+  @LibraryOperator(libraries = {MSSQL})
+  public static final SqlFunction MSSQL_CONVERT =
+      SqlBasicFunction.create(SqlKind.CAST,
+              ReturnTypes.andThen(SqlLibraryOperators::transformConvert,
+                  SqlCastFunction::inferReturnTypeImpl),
+              OperandTypes.repeat(SqlOperandCountRanges.between(2, 3),
+                  OperandTypes.ANY))
+          .withName("CONVERT")
+          .withFunctionType(SqlFunctionCategory.SYSTEM)
+          .withOperandTypeInference(InferTypes.FIRST_KNOWN)
+          .withOperandHandler(
+              OperandHandlers.of(SqlLibraryOperators::transformConvert));
+
+  /** Transforms a call binding of {@code CONVERT} to an equivalent binding for
+   *  {@code CAST}. */
+  private static SqlCallBinding transformConvert(SqlOperatorBinding opBinding) {
+    // Guaranteed to be a SqlCallBinding, with 2 or 3 arguments
+    final SqlCallBinding binding = (SqlCallBinding) opBinding;
+    return new SqlCallBinding(binding.getValidator(), binding.getScope(),
+        transformConvert(binding.getValidator(), binding.getCall()));
+  }
+
+  /** Transforms a call to {@code CONVERT} to an equivalent call to
+   *  {@code CAST}. */
+  private static SqlCall transformConvert(SqlValidator validator, SqlCall call) {
+    return SqlStdOperatorTable.CAST.createCall(call.getParserPosition(),
+        call.operand(1), call.operand(0));
+  }
+
   /** The "DATE_PART(timeUnit, datetime)" function
    * (Databricks, Postgres, Redshift, Snowflake). */
   @LibraryOperator(libraries = {POSTGRESQL})
diff --git a/core/src/main/java/org/apache/calcite/sql/type/OperandHandlers.java b/core/src/main/java/org/apache/calcite/sql/type/OperandHandlers.java
index 596e26bad9..b45560390c 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/OperandHandlers.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/OperandHandlers.java
@@ -28,6 +28,7 @@ import org.checkerframework.checker.nullness.qual.Nullable;
 
 import java.util.ArrayList;
 import java.util.List;
+import java.util.function.BiFunction;
 import java.util.function.Function;
 
 /**
@@ -61,6 +62,15 @@ public abstract class OperandHandlers {
   public static final SqlOperandHandler OPERAND_1_MIGHT_BE_TIME_FRAME =
       new TimeFrameOperandHandler(1);
 
+  /** Creates an operand handler that applies a function to a call. */
+  public static SqlOperandHandler of(BiFunction<SqlValidator, SqlCall, SqlCall> fn) {
+    return new SqlOperandHandler() {
+      @Override public SqlNode rewriteCall(SqlValidator validator, SqlCall call) {
+        return fn.apply(validator, call);
+      }
+    };
+  }
+
   /** Operand handler for a function whose {@code timeFrameOperand} operand
    * (0-based) may be a time frame. If the operand is of type
    * {@link SqlIdentifier}, looks up the custom time frame and converts it to a
diff --git a/core/src/test/resources/sql/functions.iq b/core/src/test/resources/sql/functions.iq
index a232f838ef..9f5d6b02b1 100644
--- a/core/src/test/resources/sql/functions.iq
+++ b/core/src/test/resources/sql/functions.iq
@@ -219,4 +219,88 @@ EnumerableCalc(expr#0..4=[{inputs}], expr#5=[IS NOT NULL($t2)], expr#6=[CAST($t2
 
 !ok
 
+# [CALCITE-5548] - Add support for MSSQL CONVERT function
+!use mssqlfunc
+select CONVERT(INTEGER, 45.4);
++--------+
+| EXPR$0 |
++--------+
+|     45 |
++--------+
+(1 row)
+
+!ok
+
+select CONVERT(INTEGER, 45.4, 121);
++--------+
+| EXPR$0 |
++--------+
+|     45 |
++--------+
+(1 row)
+
+!ok
+
+select CONVERT(FLOAT, '34.5');
++--------+
+| EXPR$0 |
++--------+
+|   34.5 |
++--------+
+(1 row)
+
+!ok
+
+select CONVERT(DATE, '2011-01-05');
++------------+
+| EXPR$0     |
++------------+
+| 2011-01-05 |
++------------+
+(1 row)
+
+!ok
+
+# style (3rd operand) ignored (for now)
+select CONVERT(DATE, '2011-01-05', 999);
++------------+
+| EXPR$0     |
++------------+
+| 2011-01-05 |
++------------+
+(1 row)
+
+!ok
+
+select CONVERT(INTEGER, 45.4, 121, 99999);
+parse failed: Encountered ","
+!error
+
+# since 'style' operand is ignored, usual behaviour of returning NULL with NULL style is not present
+select CONVERT(INTEGER, 5, NULL);
++--------+
+| EXPR$0 |
++--------+
+|      5 |
++--------+
+(1 row)
+
+!ok
+
+select CONVERT(INTEGER, NULL, NULL);
++--------+
+| EXPR$0 |
++--------+
+|        |
++--------+
+(1 row)
+
+!ok
+
+# below not executed due to huge stack trace quidem insists on generating
+# unsupported format test for documentation (dd/mm/yyyy) (need MSSQL "style" operand support in enumerable)
+# however CONVERT right now is simply a wrapper around CAST
+# select CONVERT(DATE, '05/01/2000', 103);
+
+
 # End functions.iq
diff --git a/site/_docs/reference.md b/site/_docs/reference.md
index 587bac802e..bd37dba74d 100644
--- a/site/_docs/reference.md
+++ b/site/_docs/reference.md
@@ -2638,7 +2638,8 @@ BigQuery's type system uses confusingly different names for types and functions:
 | b o | COSH(numeric)                                | Returns the hyperbolic cosine of *numeric*
 | o | CONCAT(string, string)                         | Concatenates two strings
 | b m p | CONCAT(string [, string ]*)                | Concatenates two or more strings
-| m | COMPRESS(string)                               | Compresses a string using zlib compression and returns the result as a binary string.
+| m | COMPRESS(string)                               | Compresses a string using zlib compression and returns the result as a binary string
+| q | CONVERT(type, expression [ , style ])          | Equivalent to `CAST(expression AS type)`; ignores the *style* operand
 | p | CONVERT_TIMEZONE(tz1, tz2, datetime)           | Converts the timezone of *datetime* from *tz1* to *tz2*
 | b | CURRENT_DATETIME([ timeZone ])                 | Returns the current time as a TIMESTAMP from *timezone*
 | m | DAYNAME(datetime)                              | Returns the name, in the connection's locale, of the weekday in *datetime*; for example, it returns '星期日' for both DATE '2020-02-10' and TIMESTAMP '2020-02-10 10:10:10'
diff --git a/testkit/src/main/java/org/apache/calcite/sql/parser/SqlParserTest.java b/testkit/src/main/java/org/apache/calcite/sql/parser/SqlParserTest.java
index 8b22a93ad8..8468edf79d 100644
--- a/testkit/src/main/java/org/apache/calcite/sql/parser/SqlParserTest.java
+++ b/testkit/src/main/java/org/apache/calcite/sql/parser/SqlParserTest.java
@@ -1714,6 +1714,25 @@ public class SqlParserTest {
         .fails("(?s).*Encountered \"without\" at line 1, column 23.\n.*");
   }
 
+  /** Test for MSSQL CONVERT parsing, with focus on iffy DATE type and
+   * testing that the extra "style" operand is parsed
+   * Other tests are defined in functions.iq
+   */
+  @Test void testMssqlConvert() {
+    expr("CONVERT(VARCHAR(5), 'xx')")
+        .same();
+    expr("CONVERT(VARCHAR(5), 'xx')")
+        .same();
+    expr("CONVERT(VARCHAR(5), NULL)")
+        .same();
+    expr("CONVERT(VARCHAR(5), NULL, NULL)")
+        .same();
+    expr("CONVERT(DATE, 'xx', 121)")
+        .same();
+    expr("CONVERT(DATE, 'xx')")
+        .same();
+  }
+
   @Test void testLikeAndSimilar() {
     sql("select * from t where x like '%abc%'")
         .ok("SELECT *\n"
diff --git a/testkit/src/main/java/org/apache/calcite/sql/test/AbstractSqlTester.java b/testkit/src/main/java/org/apache/calcite/sql/test/AbstractSqlTester.java
index cc60222543..d68b381cba 100644
--- a/testkit/src/main/java/org/apache/calcite/sql/test/AbstractSqlTester.java
+++ b/testkit/src/main/java/org/apache/calcite/sql/test/AbstractSqlTester.java
@@ -325,9 +325,10 @@ public abstract class AbstractSqlTester implements SqlTester, AutoCloseable {
    * @return Query that evaluates a scalar expression
    */
   protected String buildQuery2(SqlTestFactory factory, String expression) {
-    if (expression.matches("(?i).*percentile_(cont|disc).*")) {
+    if (expression.matches("(?i).*(percentile_(cont|disc)|convert)\\(.*")) {
       // PERCENTILE_CONT requires its argument to be a literal,
       // so converting its argument to a column will cause false errors.
+      // Similarly, MSSQL-style CONVERT.
       return buildQuery(expression);
     }
     // "values (1 < 5)"
diff --git a/testkit/src/main/java/org/apache/calcite/test/QuidemTest.java b/testkit/src/main/java/org/apache/calcite/test/QuidemTest.java
index 555c01f95e..224f7a5e3f 100644
--- a/testkit/src/main/java/org/apache/calcite/test/QuidemTest.java
+++ b/testkit/src/main/java/org/apache/calcite/test/QuidemTest.java
@@ -291,6 +291,11 @@ public abstract class QuidemTest {
             .with(CalciteConnectionProperty.FUN, "oracle")
             .with(CalciteAssert.Config.REGULAR)
             .connect();
+      case "mssqlfunc":
+        return CalciteAssert.that()
+            .with(CalciteConnectionProperty.FUN, "mssql")
+            .with(CalciteAssert.Config.REGULAR)
+            .connect();
       case "catchall":
         return CalciteAssert.that()
             .with(CalciteConnectionProperty.TIME_ZONE, "UTC")
diff --git a/testkit/src/main/java/org/apache/calcite/test/SqlOperatorTest.java b/testkit/src/main/java/org/apache/calcite/test/SqlOperatorTest.java
index 98f7830048..82fb160d9b 100644
--- a/testkit/src/main/java/org/apache/calcite/test/SqlOperatorTest.java
+++ b/testkit/src/main/java/org/apache/calcite/test/SqlOperatorTest.java
@@ -1184,6 +1184,34 @@ public class SqlOperatorTest {
     f.checkNull("cast(cast(null as timestamp) as time)");
   }
 
+  @Test void testMssqlConvert() {
+    final SqlOperatorFixture f = fixture();
+    f.setFor(SqlLibraryOperators.MSSQL_CONVERT, VmName.EXPAND);
+    // happy-paths (no need to test all, proper functionality is tested by CAST already
+    // just need to make sure it works at all
+    f.checkScalar("convert(INTEGER, 45.4)", "45", "INTEGER NOT NULL");
+    f.checkScalar("convert(DATE, '2000-01-01')", "2000-01-01", "DATE NOT NULL");
+
+    // null-values
+    f.checkNull("convert(DATE, NULL)");
+  }
+
+  @Test void testMssqlConvertWithStyle() {
+    final SqlOperatorFixture f = fixture();
+    f.setFor(SqlLibraryOperators.MSSQL_CONVERT, VmName.EXPAND);
+    // ensure 'style' argument is ignored
+    // 3rd argument 'style' is a literal. However,
+    // AbstractSqlTester converts values to a single value in a column.
+    // see AbstractSqlTester.buildQuery2
+    // But CONVERT 'style' is supposed to be a literal.
+    // So for now, they are put in a @Disabled test
+    f.checkScalar("convert(INTEGER, 45.4, 999)", "45", "INTEGER NOT NULL");
+    f.checkScalar("convert(DATE, '2000-01-01', 999)", "2000-01-01", "DATE NOT NULL");
+    // including 'NULL' style argument
+    f.checkScalar("convert(DATE, '2000-01-01', NULL)", "2000-01-01", "DATE NOT NULL");
+
+  }
+
   private static Calendar getFixedCalendar() {
     Calendar calendar = Util.calendar();
     calendar.set(Calendar.YEAR, 2014);


[calcite] 03/03: [CALCITE-5580] Add SPLIT function (enabled in BigQuery library)

Posted by jh...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit d2ca97e79917a3ced509421509189e0358ba774e
Author: Tanner Clary <ta...@google.com>
AuthorDate: Mon Mar 13 18:38:03 2023 -0700

    [CALCITE-5580] Add SPLIT function (enabled in BigQuery library)
    
    Close apache/calcite#3109
---
 babel/src/test/resources/sql/big-query.iq          | 82 ++++++++++++++++++++++
 .../calcite/adapter/enumerable/RexImpTable.java    |  2 +
 .../apache/calcite/runtime/CalciteResource.java    |  4 ++
 .../org/apache/calcite/runtime/SqlFunctions.java   | 46 ++++++++++++
 .../org/apache/calcite/sql/SqlBasicFunction.java   | 46 +++++++++---
 .../apache/calcite/sql/SqlWithinGroupOperator.java |  2 +-
 .../calcite/sql/fun/SqlLibraryOperators.java       | 26 +++++++
 .../org/apache/calcite/sql/type/OperandTypes.java  |  9 +++
 .../calcite/runtime/CalciteResource.properties     |  1 +
 .../org/apache/calcite/test/SqlFunctionsTest.java  | 68 ++++++++++++++++++
 site/_docs/reference.md                            |  1 +
 .../org/apache/calcite/test/SqlOperatorTest.java   | 34 +++++++++
 12 files changed, 309 insertions(+), 12 deletions(-)

diff --git a/babel/src/test/resources/sql/big-query.iq b/babel/src/test/resources/sql/big-query.iq
index 9b6a0fc51b..e3723221a0 100755
--- a/babel/src/test/resources/sql/big-query.iq
+++ b/babel/src/test/resources/sql/big-query.iq
@@ -648,6 +648,88 @@ SELECT (19 % 19) as result;
 
 !ok
 
+#####################################################################
+# SPLIT
+#
+# SPLIT(string [, delimiter])
+#
+# Splits string using the delimiter argument. For STRING, the default
+# delimiter is the comma.
+#
+# Returns a STRING array as result.
+WITH letters AS
+  (SELECT '' as letter_group
+  UNION ALL
+  SELECT 'a' as letter_group
+  UNION ALL
+  SELECT 'b c d' as letter_group)
+SELECT SPLIT(letter_group, ' ') as example
+FROM letters;
++-----------+
+| example   |
++-----------+
+| []        |
+| [a]       |
+| [b, c, d] |
++-----------+
+(3 rows)
+
+!ok
+
+SELECT SPLIT("h,e,l,l,o") as result;
++-----------------+
+| result          |
++-----------------+
+| [h, e, l, l, o] |
++-----------------+
+(1 row)
+
+!ok
+
+SELECT SPLIT("") as result;
++--------+
+| result |
++--------+
+| []     |
++--------+
+(1 row)
+
+!ok
+
+# Careful to treat the delimiter as a string, not a regular expression
+SELECT SPLIT("abc.de.", ".") as result;
++-------------+
+| result      |
++-------------+
+| [abc, de, ] |
++-------------+
+(1 row)
+
+!ok
+
+WITH letters AS
+  (SELECT x'' as letter_group
+  UNION ALL
+  SELECT x'41' as letter_group
+  UNION ALL
+  SELECT x'42ff43ff44' as letter_group)
+SELECT SPLIT(letter_group, x'ff') as example
+FROM letters;
++-----------+
+| example   |
++-----------+
+| []        |
+| [A]       |
+| [B, C, D] |
++-----------+
+(3 rows)
+
+!ok
+
+SELECT SPLIT(x'abc2') as result;
+Call to function 'SPLIT' with argument of type 'BINARY(2)' requires extra delimiter argument
+!error
+
 #####################################################################
 # STRING
 #
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 503bee6e19..dabf91393d 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
@@ -169,6 +169,7 @@ import static org.apache.calcite.sql.fun.SqlLibraryOperators.SHA1;
 import static org.apache.calcite.sql.fun.SqlLibraryOperators.SINH;
 import static org.apache.calcite.sql.fun.SqlLibraryOperators.SOUNDEX;
 import static org.apache.calcite.sql.fun.SqlLibraryOperators.SPACE;
+import static org.apache.calcite.sql.fun.SqlLibraryOperators.SPLIT;
 import static org.apache.calcite.sql.fun.SqlLibraryOperators.STARTS_WITH;
 import static org.apache.calcite.sql.fun.SqlLibraryOperators.STRCMP;
 import static org.apache.calcite.sql.fun.SqlLibraryOperators.TANH;
@@ -443,6 +444,7 @@ public class RexImpTable {
       defineMethod(SOUNDEX, BuiltInMethod.SOUNDEX.method, NullPolicy.STRICT);
       defineMethod(DIFFERENCE, BuiltInMethod.DIFFERENCE.method, NullPolicy.STRICT);
       defineMethod(REVERSE, BuiltInMethod.REVERSE.method, NullPolicy.STRICT);
+      defineMethod(SPLIT, "split", NullPolicy.STRICT);
 
       map.put(TRIM, new TrimImplementor());
 
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 98111b314e..02a7b4aa49 100644
--- a/core/src/main/java/org/apache/calcite/runtime/CalciteResource.java
+++ b/core/src/main/java/org/apache/calcite/runtime/CalciteResource.java
@@ -598,6 +598,10 @@ public interface CalciteResource {
   ExInst<SqlValidatorException> argumentMustBeValidPrecision(String a0, int a1,
       int a2);
 
+  @BaseMessage("Call to function ''{0}'' with argument of type ''{1}'' requires extra delimiter argument")
+  ExInst<SqlValidatorException> delimiterIsRequired(String functionName,
+      String argumentTypeName);
+
   @BaseMessage("Wrong arguments for table function ''{0}'' call. Expected ''{1}'', actual ''{2}''")
   ExInst<CalciteException> illegalArgumentForTableFunctionCall(String a0,
       String a1, String a2);
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 abbec30a08..50e4cd043c 100644
--- a/core/src/main/java/org/apache/calcite/runtime/SqlFunctions.java
+++ b/core/src/main/java/org/apache/calcite/runtime/SqlFunctions.java
@@ -49,6 +49,7 @@ import org.apache.commons.codec.language.Soundex;
 
 import com.google.common.base.Splitter;
 import com.google.common.base.Strings;
+import com.google.common.collect.ImmutableList;
 
 import org.checkerframework.checker.nullness.qual.Nullable;
 import org.checkerframework.checker.nullness.qual.PolyNull;
@@ -463,6 +464,51 @@ public class SqlFunctions {
     return s0.startsWith(s1);
   }
 
+  /** SQL {@code SPLIT(string, string)} function. */
+  public static List<String> split(String s, String delimiter) {
+    if (s.isEmpty()) {
+      return ImmutableList.of();
+    }
+    if (delimiter.isEmpty()) {
+      return ImmutableList.of(s); // prevent mischief
+    }
+    final ImmutableList.Builder<String> list = ImmutableList.builder();
+    for (int i = 0;;) {
+      int j = s.indexOf(delimiter, i);
+      if (j < 0) {
+        list.add(s.substring(i));
+        return list.build();
+      }
+      list.add(s.substring(i, j));
+      i = j + delimiter.length();
+    }
+  }
+
+  /** SQL {@code SPLIT(string)} function. */
+  public static List<String> split(String s) {
+    return split(s, ",");
+  }
+
+  /** SQL {@code SPLIT(binary, binary)} function. */
+  public static List<ByteString> split(ByteString s, ByteString delimiter) {
+    if (s.length() == 0) {
+      return ImmutableList.of();
+    }
+    if (delimiter.length() == 0) {
+      return ImmutableList.of(s); // prevent mischief
+    }
+    final ImmutableList.Builder<ByteString> list = ImmutableList.builder();
+    for (int i = 0;;) {
+      int j = s.indexOf(delimiter, i);
+      if (j < 0) {
+        list.add(s.substring(i));
+        return list.build();
+      }
+      list.add(s.substring(i, j));
+      i = j + delimiter.length();
+    }
+  }
+
   /** SQL SUBSTRING(string FROM ...) function. */
   public static String substring(String c, int s) {
     final int s0 = s - 1;
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlBasicFunction.java b/core/src/main/java/org/apache/calcite/sql/SqlBasicFunction.java
index 5787938a44..a811228bea 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlBasicFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlBasicFunction.java
@@ -23,6 +23,7 @@ import org.apache.calcite.sql.type.SqlOperandTypeInference;
 import org.apache.calcite.sql.type.SqlReturnTypeInference;
 import org.apache.calcite.sql.validate.SqlMonotonicity;
 import org.apache.calcite.sql.validate.SqlValidator;
+import org.apache.calcite.sql.validate.SqlValidatorScope;
 
 import org.checkerframework.checker.nullness.qual.Nullable;
 
@@ -45,6 +46,7 @@ public class SqlBasicFunction extends SqlFunction {
   private final SqlSyntax syntax;
   private final boolean deterministic;
   private final SqlOperandHandler operandHandler;
+  private final int callValidator;
   private final Function<SqlOperatorBinding, SqlMonotonicity> monotonicityInference;
 
   //~ Constructors -----------------------------------------------------------
@@ -60,6 +62,7 @@ public class SqlBasicFunction extends SqlFunction {
    * @param operandTypeInference Strategy to use for parameter type inference
    * @param operandHandler Strategy to use for handling operands
    * @param operandTypeChecker Strategy to use for parameter type checking
+   * @param callValidator Strategy to validate calls
    * @param category Categorization for function
    * @param monotonicityInference Strategy to infer monotonicity of a call
    */
@@ -68,6 +71,7 @@ public class SqlBasicFunction extends SqlFunction {
       @Nullable SqlOperandTypeInference operandTypeInference,
       SqlOperandHandler operandHandler,
       SqlOperandTypeChecker operandTypeChecker,
+      Integer callValidator,
       SqlFunctionCategory category,
       Function<SqlOperatorBinding, SqlMonotonicity> monotonicityInference) {
     super(name, kind,
@@ -77,6 +81,7 @@ public class SqlBasicFunction extends SqlFunction {
     this.syntax = requireNonNull(syntax, "syntax");
     this.deterministic = deterministic;
     this.operandHandler = requireNonNull(operandHandler, "operandHandler");
+    this.callValidator = requireNonNull(callValidator, "callValidator");
     this.monotonicityInference =
         requireNonNull(monotonicityInference, "monotonicityInference");
   }
@@ -88,7 +93,7 @@ public class SqlBasicFunction extends SqlFunction {
       SqlOperandTypeChecker operandTypeChecker) {
     return new SqlBasicFunction(kind.name(), kind,
         SqlSyntax.FUNCTION, true, returnTypeInference, null,
-        OperandHandlers.DEFAULT, operandTypeChecker,
+        OperandHandlers.DEFAULT, operandTypeChecker, 0,
         SqlFunctionCategory.SYSTEM, call -> SqlMonotonicity.NOT_MONOTONIC);
   }
 
@@ -100,7 +105,7 @@ public class SqlBasicFunction extends SqlFunction {
       SqlOperandTypeChecker operandTypeChecker) {
     return new SqlBasicFunction(name, SqlKind.OTHER_FUNCTION,
         SqlSyntax.FUNCTION, true, returnTypeInference, null,
-        OperandHandlers.DEFAULT, operandTypeChecker,
+        OperandHandlers.DEFAULT, operandTypeChecker, 0,
         SqlFunctionCategory.NUMERIC, call -> SqlMonotonicity.NOT_MONOTONIC);
   }
 
@@ -111,7 +116,7 @@ public class SqlBasicFunction extends SqlFunction {
       SqlOperandTypeChecker operandTypeChecker, SqlFunctionCategory category) {
     return new SqlBasicFunction(name, SqlKind.OTHER_FUNCTION,
         SqlSyntax.FUNCTION, true, returnTypeInference, null,
-        OperandHandlers.DEFAULT, operandTypeChecker,
+        OperandHandlers.DEFAULT, operandTypeChecker, 0,
         category, call -> SqlMonotonicity.NOT_MONOTONIC);
   }
 
@@ -141,32 +146,40 @@ public class SqlBasicFunction extends SqlFunction {
     return operandHandler.rewriteCall(validator, call);
   }
 
+  @Override public void validateCall(SqlCall call, SqlValidator validator,
+      SqlValidatorScope scope, SqlValidatorScope operandScope) {
+    super.validateCall(call, validator, scope, operandScope);
+  }
+
   /** Returns a copy of this function with a given name. */
   public SqlBasicFunction withName(String name) {
     return new SqlBasicFunction(name, kind, syntax, deterministic,
         getReturnTypeInference(), getOperandTypeInference(), operandHandler,
-        getOperandTypeChecker(), getFunctionType(), monotonicityInference);
+        getOperandTypeChecker(), callValidator,
+        getFunctionType(), monotonicityInference);
   }
 
   /** Returns a copy of this function with a given kind. */
   public SqlBasicFunction withKind(SqlKind kind) {
     return new SqlBasicFunction(getName(), kind, syntax, deterministic,
         getReturnTypeInference(), getOperandTypeInference(), operandHandler,
-        getOperandTypeChecker(), getFunctionType(), monotonicityInference);
+        getOperandTypeChecker(), callValidator,
+        getFunctionType(), monotonicityInference);
   }
 
   /** Returns a copy of this function with a given category. */
   public SqlBasicFunction withFunctionType(SqlFunctionCategory category) {
     return new SqlBasicFunction(getName(), kind, syntax, deterministic,
         getReturnTypeInference(), getOperandTypeInference(), operandHandler,
-        getOperandTypeChecker(), category, monotonicityInference);
+        getOperandTypeChecker(), callValidator, category, monotonicityInference);
   }
 
   /** Returns a copy of this function with a given syntax. */
   public SqlBasicFunction withSyntax(SqlSyntax syntax) {
     return new SqlBasicFunction(getName(), kind, syntax, deterministic,
         getReturnTypeInference(), getOperandTypeInference(), operandHandler,
-        getOperandTypeChecker(), getFunctionType(), monotonicityInference);
+        getOperandTypeChecker(), callValidator,
+        getFunctionType(), monotonicityInference);
   }
 
   /** Returns a copy of this function with a given strategy for inferring
@@ -175,7 +188,8 @@ public class SqlBasicFunction extends SqlFunction {
       SqlOperandTypeInference operandTypeInference) {
     return new SqlBasicFunction(getName(), kind, syntax, deterministic,
         getReturnTypeInference(), operandTypeInference, operandHandler,
-        getOperandTypeChecker(), getFunctionType(), monotonicityInference);
+        getOperandTypeChecker(), callValidator,
+        getFunctionType(), monotonicityInference);
   }
 
   /** Returns a copy of this function with a given strategy for handling
@@ -183,13 +197,15 @@ public class SqlBasicFunction extends SqlFunction {
   public SqlBasicFunction withOperandHandler(SqlOperandHandler operandHandler) {
     return new SqlBasicFunction(getName(), kind, syntax, deterministic,
         getReturnTypeInference(), getOperandTypeInference(), operandHandler,
-        getOperandTypeChecker(), getFunctionType(), monotonicityInference);
+        getOperandTypeChecker(), callValidator,
+        getFunctionType(), monotonicityInference);
   }
   /** Returns a copy of this function with a given determinism. */
   public SqlBasicFunction withDeterministic(boolean deterministic) {
     return new SqlBasicFunction(getName(), kind, syntax, deterministic,
         getReturnTypeInference(), getOperandTypeInference(), operandHandler,
-        getOperandTypeChecker(), getFunctionType(), monotonicityInference);
+        getOperandTypeChecker(), callValidator,
+        getFunctionType(), monotonicityInference);
   }
 
   /** Returns a copy of this function with a given strategy for inferring
@@ -198,6 +214,14 @@ public class SqlBasicFunction extends SqlFunction {
       Function<SqlOperatorBinding, SqlMonotonicity> monotonicityInference) {
     return new SqlBasicFunction(getName(), kind, syntax, deterministic,
         getReturnTypeInference(), getOperandTypeInference(), operandHandler,
-        getOperandTypeChecker(), getFunctionType(), monotonicityInference);
+        getOperandTypeChecker(), callValidator,
+        getFunctionType(), monotonicityInference);
+  }
+
+  public SqlFunction withValidation(int callValidator) {
+    return new SqlBasicFunction(getName(), kind, syntax, deterministic,
+        getReturnTypeInference(), getOperandTypeInference(), operandHandler,
+        getOperandTypeChecker(), callValidator,
+        getFunctionType(), monotonicityInference);
   }
 }
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlWithinGroupOperator.java b/core/src/main/java/org/apache/calcite/sql/SqlWithinGroupOperator.java
index 87f6fb7b8f..d335948322 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlWithinGroupOperator.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlWithinGroupOperator.java
@@ -106,7 +106,7 @@ public class SqlWithinGroupOperator extends SqlBinaryOperator {
     }
   }
 
-  private SqlNode getCollationColumn(SqlCall call) {
+  private static SqlNode getCollationColumn(SqlCall call) {
     return ((SqlNodeList) call.operand(1)).get(0);
   }
 
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlLibraryOperators.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlLibraryOperators.java
index cde127a424..2cd4def708 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlLibraryOperators.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlLibraryOperators.java
@@ -41,9 +41,11 @@ import org.apache.calcite.sql.type.SqlReturnTypeInference;
 import org.apache.calcite.sql.type.SqlTypeFamily;
 import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.sql.type.SqlTypeTransforms;
+import org.apache.calcite.sql.type.SqlTypeUtil;
 import org.apache.calcite.sql.validate.SqlValidator;
 import org.apache.calcite.util.Litmus;
 import org.apache.calcite.util.Optionality;
+import org.apache.calcite.util.Static;
 
 import org.checkerframework.checker.nullness.qual.Nullable;
 
@@ -301,6 +303,30 @@ public abstract class SqlLibraryOperators {
           OperandTypes.STRING)
           .withFunctionType(SqlFunctionCategory.STRING);
 
+  /** The "SPLIT(string [, delimiter])" function. */
+  @LibraryOperator(libraries = {BIG_QUERY})
+  public static final SqlFunction SPLIT =
+      SqlBasicFunction.create("SPLIT",
+          ReturnTypes.ARG0
+              .andThen(SqlLibraryOperators::deriveTypeSplit)
+              .andThen(SqlTypeTransforms.TO_ARRAY),
+          OperandTypes.or(OperandTypes.CHARACTER_CHARACTER,
+              OperandTypes.CHARACTER,
+              OperandTypes.BINARY_BINARY,
+              OperandTypes.BINARY),
+          SqlFunctionCategory.STRING)
+          .withValidation(3);
+
+  static RelDataType deriveTypeSplit(SqlOperatorBinding operatorBinding,
+      RelDataType type) {
+    if (SqlTypeUtil.isBinary(type) && operatorBinding.getOperandCount() == 1) {
+      throw operatorBinding.newError(
+          Static.RESOURCE.delimiterIsRequired(
+              operatorBinding.getOperator().getName(), type.toString()));
+    }
+    return type;
+  }
+
   /** Generic "SUBSTR(string, position [, substringLength ])" function. */
   private static final SqlBasicFunction SUBSTR =
       SqlBasicFunction.create("SUBSTR", ReturnTypes.ARG0_NULLABLE_VARYING,
diff --git a/core/src/main/java/org/apache/calcite/sql/type/OperandTypes.java b/core/src/main/java/org/apache/calcite/sql/type/OperandTypes.java
index 7c191977aa..c336f87b84 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/OperandTypes.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/OperandTypes.java
@@ -374,12 +374,21 @@ public abstract class OperandTypes {
   public static final SqlSingleOperandTypeChecker BINARY =
       family(SqlTypeFamily.BINARY);
 
+  public static final SqlSingleOperandTypeChecker BINARY_BINARY =
+      family(SqlTypeFamily.BINARY, SqlTypeFamily.BINARY);
+
   public static final SqlSingleOperandTypeChecker STRING =
       family(SqlTypeFamily.STRING);
 
   public static final FamilyOperandTypeChecker STRING_STRING =
       family(SqlTypeFamily.STRING, SqlTypeFamily.STRING);
 
+  public static final FamilyOperandTypeChecker STRING_OPTIONAL_STRING =
+      family(
+          ImmutableList.of(SqlTypeFamily.STRING, SqlTypeFamily.STRING),
+          // Second operand optional (operand index 0, 1)
+          number -> number == 1);
+
   public static final FamilyOperandTypeChecker STRING_STRING_STRING =
       family(SqlTypeFamily.STRING, SqlTypeFamily.STRING, SqlTypeFamily.STRING);
 
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 874e96043a..ad0acc822b 100644
--- a/core/src/main/resources/org/apache/calcite/runtime/CalciteResource.properties
+++ b/core/src/main/resources/org/apache/calcite/runtime/CalciteResource.properties
@@ -199,6 +199,7 @@ NullIllegal=Illegal use of ''NULL''
 DynamicParamIllegal=Illegal use of dynamic parameter
 InvalidBoolean=''{0}'' is not a valid boolean value
 ArgumentMustBeValidPrecision=Argument to function ''{0}'' must be a valid precision between ''{1,number,#}'' and ''{2,number,#}''
+DelimiterIsRequired=Call to function ''{0}'' with argument of type ''{1}'' requires extra delimiter argument
 IllegalArgumentForTableFunctionCall=Wrong arguments for table function ''{0}'' call. Expected ''{1}'', actual ''{2}''
 CannotCallTableFunctionHere=Cannot call table function here: ''{0}''
 InvalidTimeFrame=''{0}'' is not a valid time frame
diff --git a/core/src/test/java/org/apache/calcite/test/SqlFunctionsTest.java b/core/src/test/java/org/apache/calcite/test/SqlFunctionsTest.java
index 35d0375b4e..704b020871 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlFunctionsTest.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlFunctionsTest.java
@@ -22,6 +22,8 @@ import org.apache.calcite.runtime.CalciteException;
 import org.apache.calcite.runtime.SqlFunctions;
 import org.apache.calcite.runtime.Utilities;
 
+import com.google.common.collect.ImmutableList;
+
 import org.junit.jupiter.api.Test;
 
 import java.math.BigDecimal;
@@ -81,6 +83,14 @@ import static java.nio.charset.StandardCharsets.UTF_8;
  * rather than {@code assertEquals}.
  */
 class SqlFunctionsTest {
+  static <E> List<E> list(E... es) {
+    return Arrays.asList(es);
+  }
+
+  static <E> List<E> list() {
+    return ImmutableList.of();
+  }
+
   @Test void testCharLength() {
     assertThat(charLength("xyz"), is(3));
   }
@@ -507,6 +517,64 @@ class SqlFunctionsTest {
     assertThat(SqlFunctions.sround(-12000, -5), within(0d, 0.001));
   }
 
+  @Test void testSplit() {
+    assertThat("no occurrence of delimiter",
+        SqlFunctions.split("abc", ","), is(list("abc")));
+    assertThat("delimiter in middle",
+        SqlFunctions.split("abc", "b"), is(list("a", "c")));
+    assertThat("delimiter at end",
+        SqlFunctions.split("abc", "c"), is(list("ab", "")));
+    assertThat("delimiter at start",
+        SqlFunctions.split("abc", "a"), is(list("", "bc")));
+    assertThat("empty delimiter",
+        SqlFunctions.split("abc", ""), is(list("abc")));
+    assertThat("empty delimiter and string",
+        SqlFunctions.split("", ""), is(list()));
+    assertThat("empty string",
+        SqlFunctions.split("", ","), is(list()));
+    assertThat("long delimiter (occurs at start)",
+        SqlFunctions.split("abracadabra", "ab"), is(list("", "racad", "ra")));
+    assertThat("long delimiter (occurs at end)",
+        SqlFunctions.split("sabracadabrab", "ab"),
+        is(list("s", "racad", "r", "")));
+
+    // Same as above but for ByteString
+    final ByteString a = ByteString.of("aa", 16);
+    final ByteString ab = ByteString.of("aabb", 16);
+    final ByteString abc = ByteString.of("aabbcc", 16);
+    final ByteString abracadabra = ByteString.of("aabb44aaccaaddaabb44aa", 16);
+    final ByteString b = ByteString.of("bb", 16);
+    final ByteString bc = ByteString.of("bbcc", 16);
+    final ByteString c = ByteString.of("cc", 16);
+    final ByteString f = ByteString.of("ff", 16);
+    final ByteString r = ByteString.of("44", 16);
+    final ByteString ra = ByteString.of("44aa", 16);
+    final ByteString racad = ByteString.of("44aaccaadd", 16);
+    final ByteString empty = ByteString.of("", 16);
+    final ByteString s = ByteString.of("55", 16);
+    final ByteString sabracadabrab =
+        ByteString.of("55", 16).concat(abracadabra).concat(b);
+    assertThat("no occurrence of delimiter",
+        SqlFunctions.split(abc, f), is(list(abc)));
+    assertThat("delimiter in middle",
+        SqlFunctions.split(abc, b), is(list(a, c)));
+    assertThat("delimiter at end",
+        SqlFunctions.split(abc, c), is(list(ab, empty)));
+    assertThat("delimiter at start",
+        SqlFunctions.split(abc, a), is(list(empty, bc)));
+    assertThat("empty delimiter",
+        SqlFunctions.split(abc, empty), is(list(abc)));
+    assertThat("empty delimiter and string",
+        SqlFunctions.split(empty, empty), is(list()));
+    assertThat("empty string",
+        SqlFunctions.split(empty, f), is(list()));
+    assertThat("long delimiter (occurs at start)",
+        SqlFunctions.split(abracadabra, ab), is(list(empty, racad, ra)));
+    assertThat("long delimiter (occurs at end)",
+        SqlFunctions.split(sabracadabrab, ab),
+        is(list(s, racad, r, empty)));
+  }
+
   @Test void testByteString() {
     final byte[] bytes = {(byte) 0xAB, (byte) 0xFF};
     final ByteString byteString = new ByteString(bytes);
diff --git a/site/_docs/reference.md b/site/_docs/reference.md
index bd37dba74d..ddd8dd7e3a 100644
--- a/site/_docs/reference.md
+++ b/site/_docs/reference.md
@@ -2713,6 +2713,7 @@ BigQuery's type system uses confusingly different names for types and functions:
 | b o | SINH(numeric)                                | Returns the hyperbolic sine of *numeric*
 | b m o p | SOUNDEX(string)                          | Returns the phonetic representation of *string*; throws if *string* is encoded with multi-byte encoding such as UTF-8
 | m | SPACE(integer)                                 | Returns a string of *integer* spaces; returns an empty string if *integer* is less than 1
+| b | SPLIT(string [, delimiter ])                   | Returns the string array of *string* split at *delimiter* (if omitted, default is comma)
 | b | STARTS_WITH(string1, string2)                  | Returns whether *string2* is a prefix of *string1*
 | m | STRCMP(string, string)                         | Returns 0 if both of the strings are same and returns -1 when the first argument is smaller than the second and 1 when the second one is smaller than the first one
 | b m o p | SUBSTR(string, position [, substringLength ]) | Returns a portion of *string*, beginning at character *position*, *substringLength* characters long. SUBSTR calculates lengths using characters as defined by the input character set
diff --git a/testkit/src/main/java/org/apache/calcite/test/SqlOperatorTest.java b/testkit/src/main/java/org/apache/calcite/test/SqlOperatorTest.java
index 82fb160d9b..523cbe0609 100644
--- a/testkit/src/main/java/org/apache/calcite/test/SqlOperatorTest.java
+++ b/testkit/src/main/java/org/apache/calcite/test/SqlOperatorTest.java
@@ -6069,6 +6069,40 @@ public class SqlOperatorTest {
     f.checkBoolean("ends_with(x'', x'')", true);
   }
 
+  /** Tests the {@code SPLIT} operator. */
+  @Test void testSplitFunction() {
+    final SqlOperatorFixture f0 = fixture().setFor(SqlLibraryOperators.SPLIT);
+    f0.checkFails("^split('hello')^",
+        "No match found for function signature SPLIT\\(<CHARACTER>\\)",
+        false);
+    final SqlOperatorFixture f = f0.withLibrary(SqlLibrary.BIG_QUERY);
+    f.checkScalar("SPLIT('h,e,l,l,o')", "[h, e, l, l, o]",
+        "CHAR(9) NOT NULL ARRAY NOT NULL");
+    f.checkScalar("SPLIT('h-e-l-l-o', '-')", "[h, e, l, l, o]",
+        "CHAR(9) NOT NULL ARRAY NOT NULL");
+    f.checkScalar("SPLIT('hello', '-')", "[hello]",
+        "CHAR(5) NOT NULL ARRAY NOT NULL");
+    f.checkScalar("SPLIT('')", "[]",
+        "CHAR(0) NOT NULL ARRAY NOT NULL");
+    f.checkScalar("SPLIT('', '-')", "[]",
+        "CHAR(0) NOT NULL ARRAY NOT NULL");
+    f.checkNull("SPLIT(null)");
+    f.checkNull("SPLIT('hello', null)");
+
+    // In ASCII, x'41' = 'A', x'42' = 'B', x'43' = 'C'
+    f.checkScalar("SPLIT(x'414243', x'ff')", "[ABC]",
+        "BINARY(3) NOT NULL ARRAY NOT NULL");
+    f.checkScalar("SPLIT(x'414243', x'41')", "[, BC]",
+        "BINARY(3) NOT NULL ARRAY NOT NULL");
+    f.checkScalar("SPLIT(x'414243', x'42')", "[A, C]",
+        "BINARY(3) NOT NULL ARRAY NOT NULL");
+    f.checkScalar("SPLIT(x'414243', x'43')", "[AB, ]",
+        "BINARY(3) NOT NULL ARRAY NOT NULL");
+    f.checkFails("^SPLIT(x'aabbcc')^",
+        "Call to function 'SPLIT' with argument of type 'BINARY\\(3\\)' "
+            + "requires extra delimiter argument", false);
+  }
+
   /** Tests the {@code SUBSTRING} operator. Many test cases that used to be
    * have been moved to {@link SubFunChecker#assertSubFunReturns}, and are
    * called for both {@code SUBSTRING} and {@code SUBSTR}. */


[calcite] 01/03: Add .gitignore for Java VSCode plugin

Posted by jh...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 8b1059ee8a621e0b3986679daea9d128ca7aa36e
Author: askarbozcan <as...@metu.edu.tr>
AuthorDate: Thu Mar 2 18:02:29 2023 +0200

    Add .gitignore for Java VSCode plugin
---
 .gitignore | 6 ++++++
 1 file changed, 6 insertions(+)

diff --git a/.gitignore b/.gitignore
index 0602bdcdd1..9425771f74 100644
--- a/.gitignore
+++ b/.gitignore
@@ -26,6 +26,12 @@
 /buildSrc/subprojects/*/build
 /site/.jekyll-cache
 
+# VSCode Java plugin
+/example/*/bin
+/*/bin
+/bin
+/.vscode/*
+
 # IDEA
 /out
 /*/out/