{
+ checkNonQueryExpression(exprContext);
+ list.add(new SqlParserUtil.ToTreeListItem(SqlLibraryOperators.NULL_SAFE_EQUAL, getPos()));
+ }
+ Expression2b(ExprContext.ACCEPT_SUB_QUERY, list)
+}
diff --git a/babel/src/main/java/org/apache/calcite/sql/babel/SqlBabelCreateTable.java b/babel/src/main/java/org/apache/calcite/sql/babel/SqlBabelCreateTable.java
new file mode 100644
index 000000000000..511bef36872b
--- /dev/null
+++ b/babel/src/main/java/org/apache/calcite/sql/babel/SqlBabelCreateTable.java
@@ -0,0 +1,80 @@
+/*
+ * 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.calcite.sql.babel;
+
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlNodeList;
+import org.apache.calcite.sql.SqlWriter;
+import org.apache.calcite.sql.ddl.SqlCreateTable;
+import org.apache.calcite.sql.parser.SqlParserPos;
+
+/**
+ * Parse tree for {@code CREATE TABLE} statement, with extensions for particular
+ * SQL dialects supported by Babel.
+ */
+public class SqlBabelCreateTable extends SqlCreateTable {
+ private final TableCollectionType tableCollectionType;
+ // CHECKSTYLE: IGNORE 2; can't use 'volatile' because it is a Java keyword
+ // but checkstyle does not like trailing '_'.
+ private final boolean volatile_;
+
+ /** Creates a SqlBabelCreateTable. */
+ public SqlBabelCreateTable(SqlParserPos pos, boolean replace,
+ TableCollectionType tableCollectionType, boolean volatile_,
+ boolean ifNotExists, SqlIdentifier name, SqlNodeList columnList,
+ SqlNode query) {
+ super(pos, replace, ifNotExists, name, columnList, query);
+ this.tableCollectionType = tableCollectionType;
+ this.volatile_ = volatile_;
+ }
+
+ @Override public void unparse(SqlWriter writer, int leftPrec, int rightPrec) {
+ writer.keyword("CREATE");
+ switch (tableCollectionType) {
+ case SET:
+ writer.keyword("SET");
+ break;
+ case MULTISET:
+ writer.keyword("MULTISET");
+ break;
+ default:
+ break;
+ }
+ if (volatile_) {
+ writer.keyword("VOLATILE");
+ }
+ writer.keyword("TABLE");
+ if (ifNotExists) {
+ writer.keyword("IF NOT EXISTS");
+ }
+ name.unparse(writer, leftPrec, rightPrec);
+ if (columnList != null) {
+ SqlWriter.Frame frame = writer.startList("(", ")");
+ for (SqlNode c : columnList) {
+ writer.sep(",");
+ c.unparse(writer, 0, 0);
+ }
+ writer.endList(frame);
+ }
+ if (query != null) {
+ writer.keyword("AS");
+ writer.newlineAndIndent();
+ query.unparse(writer, 0, 0);
+ }
+ }
+}
diff --git a/babel/src/main/java/org/apache/calcite/sql/babel/TableCollectionType.java b/babel/src/main/java/org/apache/calcite/sql/babel/TableCollectionType.java
new file mode 100644
index 000000000000..df8b76118054
--- /dev/null
+++ b/babel/src/main/java/org/apache/calcite/sql/babel/TableCollectionType.java
@@ -0,0 +1,48 @@
+/*
+ * 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.calcite.sql.babel;
+
+/**
+ * Enumerates the collection type of a table: {@code MULTISET} allows duplicates
+ * and {@code SET} does not.
+ *
+ * This feature is supported in Teradata, which originally required rows in a
+ * table to be unique, and later added the {@code MULTISET} keyword to
+ * its {@code CREATE TABLE} command to allow the duplicate rows.
+ *
+ *
In other databases and in the SQL standard, {@code MULTISET} is the only
+ * supported option, so there is no explicit syntax.
+ */
+public enum TableCollectionType {
+ /**
+ * Table collection type is not specified.
+ *
+ *
Defaults to {@code MULTISET} in ANSI mode,
+ * and {@code SET} in Teradata mode.
+ */
+ UNSPECIFIED,
+
+ /**
+ * Duplicate rows are not permitted.
+ */
+ SET,
+
+ /**
+ * Duplicate rows are permitted, in compliance with the ANSI SQL:2011 standard.
+ */
+ MULTISET,
+}
diff --git a/linq4j/src/test/java/org/apache/calcite/linq4j/tree/package-info.java b/babel/src/main/java/org/apache/calcite/sql/babel/package-info.java
similarity index 83%
rename from linq4j/src/test/java/org/apache/calcite/linq4j/tree/package-info.java
rename to babel/src/main/java/org/apache/calcite/sql/babel/package-info.java
index 86d8d9d8e3d5..4e83feda71be 100644
--- a/linq4j/src/test/java/org/apache/calcite/linq4j/tree/package-info.java
+++ b/babel/src/main/java/org/apache/calcite/sql/babel/package-info.java
@@ -16,11 +16,6 @@
*/
/**
- * Tests for expressions.
+ * Parse tree for SQL extensions used by the Babel parser.
*/
-@PackageMarker
-package org.apache.calcite.linq4j.tree;
-
-import org.apache.calcite.linq4j.PackageMarker;
-
-// End package-info.java
+package org.apache.calcite.sql.babel;
diff --git a/babel/src/test/java/org/apache/calcite/test/BabelParserTest.java b/babel/src/test/java/org/apache/calcite/test/BabelParserTest.java
new file mode 100644
index 000000000000..eb3ba0b116f3
--- /dev/null
+++ b/babel/src/test/java/org/apache/calcite/test/BabelParserTest.java
@@ -0,0 +1,355 @@
+/*
+ * 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.calcite.test;
+
+import org.apache.calcite.sql.SqlDialect;
+import org.apache.calcite.sql.dialect.MysqlSqlDialect;
+import org.apache.calcite.sql.parser.SqlAbstractParserImpl;
+import org.apache.calcite.sql.parser.SqlParser;
+import org.apache.calcite.sql.parser.SqlParserFixture;
+import org.apache.calcite.sql.parser.SqlParserTest;
+import org.apache.calcite.sql.parser.StringAndPos;
+import org.apache.calcite.sql.parser.babel.SqlBabelParserImpl;
+import org.apache.calcite.tools.Hoist;
+
+import org.apache.kylin.guava30.shaded.common.base.Throwables;
+
+import org.checkerframework.checker.nullness.qual.Nullable;
+import org.junit.jupiter.api.Disabled;
+import org.junit.jupiter.api.Test;
+
+import java.util.Arrays;
+import java.util.Locale;
+import java.util.Objects;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.MatcherAssert.assertThat;
+
+/**
+ * Tests the "Babel" SQL parser, that understands all dialects of SQL.
+ */
+class BabelParserTest extends SqlParserTest {
+
+ @Override public SqlParserFixture fixture() {
+ return super.fixture()
+ .withTester(new BabelTesterImpl())
+ .withConfig(c -> c.withParserFactory(SqlBabelParserImpl.FACTORY));
+ }
+
+ @Test void testReservedWords() {
+ assertThat(isReserved("escape"), is(false));
+ }
+
+ /** {@inheritDoc}
+ *
+ *
Copy-pasted from base method, but with some key differences.
+ */
+ @Override @Test protected void testMetadata() {
+ SqlAbstractParserImpl.Metadata metadata = fixture().parser().getMetadata();
+ assertThat(metadata.isReservedFunctionName("ABS"), is(true));
+ assertThat(metadata.isReservedFunctionName("FOO"), is(false));
+
+ assertThat(metadata.isContextVariableName("CURRENT_USER"), is(true));
+ assertThat(metadata.isContextVariableName("CURRENT_CATALOG"), is(true));
+ assertThat(metadata.isContextVariableName("CURRENT_SCHEMA"), is(true));
+ assertThat(metadata.isContextVariableName("ABS"), is(false));
+ assertThat(metadata.isContextVariableName("FOO"), is(false));
+
+ assertThat(metadata.isNonReservedKeyword("A"), is(true));
+ assertThat(metadata.isNonReservedKeyword("KEY"), is(true));
+ assertThat(metadata.isNonReservedKeyword("SELECT"), is(false));
+ assertThat(metadata.isNonReservedKeyword("FOO"), is(false));
+ assertThat(metadata.isNonReservedKeyword("ABS"), is(true)); // was false
+
+ assertThat(metadata.isKeyword("ABS"), is(true));
+ assertThat(metadata.isKeyword("CURRENT_USER"), is(true));
+ assertThat(metadata.isKeyword("CURRENT_CATALOG"), is(true));
+ assertThat(metadata.isKeyword("CURRENT_SCHEMA"), is(true));
+ assertThat(metadata.isKeyword("KEY"), is(true));
+ assertThat(metadata.isKeyword("SELECT"), is(true));
+ assertThat(metadata.isKeyword("HAVING"), is(true));
+ assertThat(metadata.isKeyword("A"), is(true));
+ assertThat(metadata.isKeyword("BAR"), is(false));
+
+ assertThat(metadata.isReservedWord("SELECT"), is(true));
+ assertThat(metadata.isReservedWord("CURRENT_CATALOG"), is(false)); // was true
+ assertThat(metadata.isReservedWord("CURRENT_SCHEMA"), is(false)); // was true
+ assertThat(metadata.isReservedWord("KEY"), is(false));
+
+ String jdbcKeywords = metadata.getJdbcKeywords();
+ assertThat(jdbcKeywords.contains(",COLLECT,"), is(false)); // was true
+ assertThat(!jdbcKeywords.contains(",SELECT,"), is(true));
+ }
+
+ @Test void testSelect() {
+ final String sql = "select 1 from t";
+ final String expected = "SELECT 1\n"
+ + "FROM `T`";
+ sql(sql).ok(expected);
+ }
+
+ @Test void testYearIsNotReserved() {
+ final String sql = "select 1 as year from t";
+ final String expected = "SELECT 1 AS `YEAR`\n"
+ + "FROM `T`";
+ sql(sql).ok(expected);
+ }
+
+ /** Tests that there are no reserved keywords. */
+ @Disabled
+ @Test void testKeywords() {
+ final String[] reserved = {"AND", "ANY", "END-EXEC"};
+ final StringBuilder sql = new StringBuilder("select ");
+ final StringBuilder expected = new StringBuilder("SELECT ");
+ for (String keyword : keywords(null)) {
+ // Skip "END-EXEC"; I don't know how a keyword can contain '-'
+ if (!Arrays.asList(reserved).contains(keyword)) {
+ sql.append("1 as ").append(keyword).append(", ");
+ expected.append("1 as `").append(keyword.toUpperCase(Locale.ROOT))
+ .append("`,\n");
+ }
+ }
+ sql.setLength(sql.length() - 2); // remove ', '
+ expected.setLength(expected.length() - 2); // remove ',\n'
+ sql.append(" from t");
+ expected.append("\nFROM t");
+ sql(sql.toString()).ok(expected.toString());
+ }
+
+ /** In Babel, AS is not reserved. */
+ @Test void testAs() {
+ final String expected = "SELECT `AS`\n"
+ + "FROM `T`";
+ sql("select as from t").ok(expected);
+ }
+
+ /** In Babel, DESC is not reserved. */
+ @Test void testDesc() {
+ final String sql = "select desc\n"
+ + "from t\n"
+ + "order by desc asc, desc desc";
+ final String expected = "SELECT `DESC`\n"
+ + "FROM `T`\n"
+ + "ORDER BY `DESC`, `DESC` DESC";
+ sql(sql).ok(expected);
+ }
+
+ /**
+ * This is a failure test making sure the LOOKAHEAD for WHEN clause is 2 in Babel, where
+ * in core parser this number is 1.
+ *
+ * @see SqlParserTest#testCaseExpression()
+ * @see [CALCITE-2847]
+ * Optimize global LOOKAHEAD for SQL parsers
+ */
+ @Test void testCaseExpressionBabel() {
+ sql("case x when 2, 4 then 3 ^when^ then 5 else 4 end")
+ .fails("(?s)Encountered \"when then\" at .*");
+ }
+
+ /** In Redshift, DATE is a function. It requires special treatment in the
+ * parser because it is a reserved keyword.
+ * (Curiously, TIMESTAMP and TIME are not functions.) */
+ @Test void testDateFunction() {
+ final String expected = "SELECT `DATE`(`X`)\n"
+ + "FROM `T`";
+ sql("select date(x) from t").ok(expected);
+ }
+
+ /** In Redshift, PostgreSQL the DATEADD, DATEDIFF and DATE_PART functions have
+ * ordinary function syntax except that its first argument is a time unit
+ * (e.g. DAY). We must not parse that first argument as an identifier. */
+ @Test void testRedshiftFunctionsWithDateParts() {
+ final String sql = "SELECT DATEADD(day, 1, t),\n"
+ + " DATEDIFF(week, 2, t),\n"
+ + " DATE_PART(year, t) FROM mytable";
+ final String expected = "SELECT `DATEADD`(DAY, 1, `T`),"
+ + " `DATEDIFF`(WEEK, 2, `T`), `DATE_PART`(YEAR, `T`)\n"
+ + "FROM `MYTABLE`";
+
+ sql(sql).ok(expected);
+ }
+
+ /** PostgreSQL and Redshift allow TIMESTAMP literals that contain only a
+ * date part. */
+ @Test void testShortTimestampLiteral() {
+ sql("select timestamp '1969-07-20'")
+ .ok("SELECT TIMESTAMP '1969-07-20 00:00:00'");
+ // PostgreSQL allows the following. We should too.
+ sql("select ^timestamp '1969-07-20 1:2'^")
+ .fails("Illegal TIMESTAMP literal '1969-07-20 1:2': not in format "
+ + "'yyyy-MM-dd HH:mm:ss'"); // PostgreSQL gives 1969-07-20 01:02:00
+ sql("select ^timestamp '1969-07-20:23:'^")
+ .fails("Illegal TIMESTAMP literal '1969-07-20:23:': not in format "
+ + "'yyyy-MM-dd HH:mm:ss'"); // PostgreSQL gives 1969-07-20 23:00:00
+ }
+
+ /** Tests parsing PostgreSQL-style "::" cast operator. */
+ @Test void testParseInfixCast() {
+ checkParseInfixCast("integer");
+ checkParseInfixCast("varchar");
+ checkParseInfixCast("boolean");
+ checkParseInfixCast("double");
+ checkParseInfixCast("bigint");
+
+ final String sql = "select -('12' || '.34')::VARCHAR(30)::INTEGER as x\n"
+ + "from t";
+ final String expected = ""
+ + "SELECT (- ('12' || '.34') :: VARCHAR(30) :: INTEGER) AS `X`\n"
+ + "FROM `T`";
+ sql(sql).ok(expected);
+ }
+
+ private void checkParseInfixCast(String sqlType) {
+ String sql = "SELECT x::" + sqlType + " FROM (VALUES (1, 2)) as tbl(x,y)";
+ String expected = "SELECT `X` :: " + sqlType.toUpperCase(Locale.ROOT) + "\n"
+ + "FROM (VALUES (ROW(1, 2))) AS `TBL` (`X`, `Y`)";
+ sql(sql).ok(expected);
+ }
+
+ /** Tests parsing MySQL-style "<=>" equal operator. */
+ @Test void testParseNullSafeEqual() {
+ // x <=> y
+ final String projectSql = "SELECT x <=> 3 FROM (VALUES (1, 2)) as tbl(x,y)";
+ sql(projectSql).ok("SELECT (`X` <=> 3)\n"
+ + "FROM (VALUES (ROW(1, 2))) AS `TBL` (`X`, `Y`)");
+ final String filterSql = "SELECT y FROM (VALUES (1, 2)) as tbl(x,y) WHERE x <=> null";
+ sql(filterSql).ok("SELECT `Y`\n"
+ + "FROM (VALUES (ROW(1, 2))) AS `TBL` (`X`, `Y`)\n"
+ + "WHERE (`X` <=> NULL)");
+ final String joinConditionSql = "SELECT tbl1.y FROM (VALUES (1, 2)) as tbl1(x,y)\n"
+ + "LEFT JOIN (VALUES (null, 3)) as tbl2(x,y) ON tbl1.x <=> tbl2.x";
+ sql(joinConditionSql).ok("SELECT `TBL1`.`Y`\n"
+ + "FROM (VALUES (ROW(1, 2))) AS `TBL1` (`X`, `Y`)\n"
+ + "LEFT JOIN (VALUES (ROW(NULL, 3))) AS `TBL2` (`X`, `Y`) ON (`TBL1`.`X` <=> `TBL2`.`X`)");
+ // (a, b) <=> (x, y)
+ final String rowComparisonSql = "SELECT y\n"
+ + "FROM (VALUES (1, 2)) as tbl(x,y) WHERE (x,y) <=> (null,2)";
+ sql(rowComparisonSql).ok("SELECT `Y`\n"
+ + "FROM (VALUES (ROW(1, 2))) AS `TBL` (`X`, `Y`)\n"
+ + "WHERE ((ROW(`X`, `Y`)) <=> (ROW(NULL, 2)))");
+ // the higher precedence
+ final String highPrecedenceSql = "SELECT x <=> 3 + 3 FROM (VALUES (1, 2)) as tbl(x,y)";
+ sql(highPrecedenceSql).ok("SELECT (`X` <=> (3 + 3))\n"
+ + "FROM (VALUES (ROW(1, 2))) AS `TBL` (`X`, `Y`)");
+ // the lower precedence
+ final String lowPrecedenceSql = "SELECT NOT x <=> 3 FROM (VALUES (1, 2)) as tbl(x,y)";
+ sql(lowPrecedenceSql).ok("SELECT (NOT (`X` <=> 3))\n"
+ + "FROM (VALUES (ROW(1, 2))) AS `TBL` (`X`, `Y`)");
+ }
+
+ @Test void testCreateTableWithNoCollectionTypeSpecified() {
+ final String sql = "create table foo (bar integer not null, baz varchar(30))";
+ final String expected = "CREATE TABLE `FOO` (`BAR` INTEGER NOT NULL, `BAZ` VARCHAR(30))";
+ sql(sql).ok(expected);
+ }
+
+ @Test void testCreateSetTable() {
+ final String sql = "create set table foo (bar int not null, baz varchar(30))";
+ final String expected = "CREATE SET TABLE `FOO` (`BAR` INTEGER NOT NULL, `BAZ` VARCHAR(30))";
+ sql(sql).ok(expected);
+ }
+
+ @Test void testCreateMultisetTable() {
+ final String sql = "create multiset table foo (bar int not null, baz varchar(30))";
+ final String expected = "CREATE MULTISET TABLE `FOO` "
+ + "(`BAR` INTEGER NOT NULL, `BAZ` VARCHAR(30))";
+ sql(sql).ok(expected);
+ }
+
+ @Test void testCreateVolatileTable() {
+ final String sql = "create volatile table foo (bar int not null, baz varchar(30))";
+ final String expected = "CREATE VOLATILE TABLE `FOO` "
+ + "(`BAR` INTEGER NOT NULL, `BAZ` VARCHAR(30))";
+ sql(sql).ok(expected);
+ }
+
+ /** Similar to {@link #testHoist()} but using custom parser. */
+ @Test void testHoistMySql() {
+ // SQL contains back-ticks, which require MySQL's quoting,
+ // and DATEADD, which requires Babel.
+ final String sql = "select 1 as x,\n"
+ + " 'ab' || 'c' as y\n"
+ + "from `my emp` /* comment with 'quoted string'? */ as e\n"
+ + "where deptno < 40\n"
+ + "and DATEADD(day, 1, hiredate) > date '2010-05-06'";
+ final SqlDialect dialect = MysqlSqlDialect.DEFAULT;
+ final Hoist.Hoisted hoisted =
+ Hoist.create(Hoist.config()
+ .withParserConfig(
+ dialect.configureParser(SqlParser.config())
+ .withParserFactory(SqlBabelParserImpl::new)))
+ .hoist(sql);
+
+ // Simple toString converts each variable to '?N'
+ final String expected = "select ?0 as x,\n"
+ + " ?1 || ?2 as y\n"
+ + "from `my emp` /* comment with 'quoted string'? */ as e\n"
+ + "where deptno < ?3\n"
+ + "and DATEADD(day, ?4, hiredate) > ?5";
+ assertThat(hoisted.toString(), is(expected));
+
+ // Custom string converts variables to '[N:TYPE:VALUE]'
+ final String expected2 = "select [0:DECIMAL:1] as x,\n"
+ + " [1:CHAR:ab] || [2:CHAR:c] as y\n"
+ + "from `my emp` /* comment with 'quoted string'? */ as e\n"
+ + "where deptno < [3:DECIMAL:40]\n"
+ + "and DATEADD(day, [4:DECIMAL:1], hiredate) > [5:DATE:2010-05-06]";
+ assertThat(hoisted.substitute(SqlParserTest::varToStr), is(expected2));
+ }
+
+ /**
+ * Babel parser's global {@code LOOKAHEAD} is larger than the core
+ * parser's. This causes different parse error message between these two
+ * parsers. Here we define a looser error checker for Babel, so that we can
+ * reuse failure testing codes from {@link SqlParserTest}.
+ *
+ *
If a test case is written in this file -- that is, not inherited -- it
+ * is still checked by {@link SqlParserTest}'s checker.
+ */
+ public static class BabelTesterImpl extends TesterImpl {
+ @Override protected void checkEx(String expectedMsgPattern,
+ StringAndPos sap, @Nullable Throwable thrown) {
+ if (thrown != null && thrownByBabelTest(thrown)) {
+ super.checkEx(expectedMsgPattern, sap, thrown);
+ } else {
+ checkExNotNull(sap, thrown);
+ }
+ }
+
+ private boolean thrownByBabelTest(Throwable ex) {
+ Throwable rootCause = Throwables.getRootCause(ex);
+ StackTraceElement[] stackTrace = rootCause.getStackTrace();
+ for (StackTraceElement stackTraceElement : stackTrace) {
+ String className = stackTraceElement.getClassName();
+ if (Objects.equals(className, BabelParserTest.class.getName())) {
+ return true;
+ }
+ }
+ return false;
+ }
+
+ private void checkExNotNull(StringAndPos sap,
+ @Nullable Throwable thrown) {
+ if (thrown == null) {
+ throw new AssertionError("Expected query to throw exception, "
+ + "but it did not; query [" + sap.sql
+ + "]");
+ }
+ }
+ }
+}
diff --git a/babel/src/test/java/org/apache/calcite/test/BabelQuidemTest.java b/babel/src/test/java/org/apache/calcite/test/BabelQuidemTest.java
new file mode 100644
index 000000000000..9492c0b28701
--- /dev/null
+++ b/babel/src/test/java/org/apache/calcite/test/BabelQuidemTest.java
@@ -0,0 +1,194 @@
+/*
+ * 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.calcite.test;
+
+import org.apache.calcite.config.CalciteConnectionProperty;
+import org.apache.calcite.jdbc.CalciteConnection;
+import org.apache.calcite.materialize.MaterializationService;
+import org.apache.calcite.plan.Contexts;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlWriter;
+import org.apache.calcite.sql.parser.SqlParser;
+import org.apache.calcite.sql.parser.babel.SqlBabelParserImpl;
+import org.apache.calcite.sql.pretty.SqlPrettyWriter;
+import org.apache.calcite.sql.validate.SqlConformanceEnum;
+import org.apache.calcite.tools.Frameworks;
+import org.apache.calcite.tools.Planner;
+
+import org.apache.kylin.guava30.shaded.common.collect.ImmutableList;
+import org.apache.kylin.guava30.shaded.common.collect.ImmutableSet;
+
+import net.hydromatic.quidem.AbstractCommand;
+import net.hydromatic.quidem.Command;
+import net.hydromatic.quidem.CommandHandler;
+import net.hydromatic.quidem.Quidem;
+
+import org.junit.jupiter.api.BeforeEach;
+
+import java.sql.Connection;
+import java.util.Collection;
+import java.util.List;
+import java.util.Set;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+/**
+ * Unit tests for the Babel SQL parser.
+ */
+class BabelQuidemTest extends QuidemTest {
+ /** Runs a test from the command line.
+ *
+ *
For example:
+ *
+ *
+ * java BabelQuidemTest sql/table.iq
+ * */
+ public static void main(String[] args) throws Exception {
+ for (String arg : args) {
+ new BabelQuidemTest().test(arg);
+ }
+ }
+
+ @BeforeEach public void setup() {
+ MaterializationService.setThreadLocal();
+ }
+
+ /** For {@link QuidemTest#test(String)} parameters. */
+ public static Collection data() {
+ // Start with a test file we know exists, then find the directory and list
+ // its files.
+ final String first = "sql/select.iq";
+ return data(first);
+ }
+
+ @Override protected Quidem.ConnectionFactory createConnectionFactory() {
+ return new QuidemConnectionFactory() {
+ @Override public Connection connect(String name, boolean reference)
+ throws Exception {
+ switch (name) {
+ case "babel":
+ return BabelTest.connect();
+ case "scott-babel":
+ return CalciteAssert.that()
+ .with(CalciteAssert.Config.SCOTT)
+ .with(CalciteConnectionProperty.PARSER_FACTORY,
+ SqlBabelParserImpl.class.getName() + "#FACTORY")
+ .with(CalciteConnectionProperty.CONFORMANCE,
+ SqlConformanceEnum.BABEL)
+ .connect();
+ case "scott-redshift":
+ return CalciteAssert.that()
+ .with(CalciteAssert.Config.SCOTT)
+ .with(CalciteConnectionProperty.FUN, "standard,postgresql,oracle")
+ .with(CalciteConnectionProperty.PARSER_FACTORY,
+ SqlBabelParserImpl.class.getName() + "#FACTORY")
+ .with(CalciteConnectionProperty.CONFORMANCE,
+ SqlConformanceEnum.BABEL)
+ .with(CalciteConnectionProperty.LENIENT_OPERATOR_LOOKUP, true)
+ .connect();
+ case "scott-big-query":
+ return CalciteAssert.that()
+ .with(CalciteAssert.Config.SCOTT)
+ .with(CalciteConnectionProperty.FUN, "standard,bigquery")
+ .with(CalciteConnectionProperty.PARSER_FACTORY,
+ SqlBabelParserImpl.class.getName() + "#FACTORY")
+ .with(CalciteConnectionProperty.CONFORMANCE,
+ SqlConformanceEnum.BABEL)
+ .with(CalciteConnectionProperty.LENIENT_OPERATOR_LOOKUP, true)
+ .connect();
+ default:
+ return super.connect(name, reference);
+ }
+ }
+ };
+ }
+
+ @Override protected CommandHandler createCommandHandler() {
+ return new BabelCommandHandler();
+ }
+
+ /** Command that prints the validated parse tree of a SQL statement. */
+ static class ExplainValidatedCommand extends AbstractCommand {
+ private final ImmutableList lines;
+ private final ImmutableList content;
+ private final Set productSet;
+
+ ExplainValidatedCommand(List lines, List content,
+ Set productSet) {
+ this.lines = ImmutableList.copyOf(lines);
+ this.content = ImmutableList.copyOf(content);
+ this.productSet = ImmutableSet.copyOf(productSet);
+ }
+
+ @Override public void execute(Context x, boolean execute) throws Exception {
+ if (execute) {
+ // use Babel parser
+ final SqlParser.Config parserConfig =
+ SqlParser.config().withParserFactory(SqlBabelParserImpl.FACTORY);
+
+ // extract named schema from connection and use it in planner
+ final CalciteConnection calciteConnection =
+ x.connection().unwrap(CalciteConnection.class);
+ final String schemaName = calciteConnection.getSchema();
+ final SchemaPlus schema =
+ schemaName != null
+ ? calciteConnection.getRootSchema().getSubSchema(schemaName)
+ : calciteConnection.getRootSchema();
+ final Frameworks.ConfigBuilder config =
+ Frameworks.newConfigBuilder()
+ .defaultSchema(schema)
+ .parserConfig(parserConfig)
+ .context(Contexts.of(calciteConnection.config()));
+
+ // parse, validate and un-parse
+ final Quidem.SqlCommand sqlCommand = x.previousSqlCommand();
+ final Planner planner = Frameworks.getPlanner(config.build());
+ final SqlNode node = planner.parse(sqlCommand.sql);
+ final SqlNode validateNode = planner.validate(node);
+ final SqlWriter sqlWriter = new SqlPrettyWriter();
+ validateNode.unparse(sqlWriter, 0, 0);
+ x.echo(ImmutableList.of(sqlWriter.toSqlString().getSql()));
+ } else {
+ x.echo(content);
+ }
+ x.echo(lines);
+ }
+ }
+
+ /** Command handler that adds a "!explain-validated-on dialect..." command
+ * (see {@link ExplainValidatedCommand}). */
+ private static class BabelCommandHandler implements CommandHandler {
+ @Override public Command parseCommand(List lines,
+ List content, String line) {
+ final String prefix = "explain-validated-on";
+ if (line.startsWith(prefix)) {
+ final Pattern pattern =
+ Pattern.compile("explain-validated-on( [-_+a-zA-Z0-9]+)*?");
+ final Matcher matcher = pattern.matcher(line);
+ if (matcher.matches()) {
+ final ImmutableSet.Builder set = ImmutableSet.builder();
+ for (int i = 0; i < matcher.groupCount(); i++) {
+ set.add(matcher.group(i + 1));
+ }
+ return new ExplainValidatedCommand(lines, content, set.build());
+ }
+ }
+ return null;
+ }
+ }
+}
diff --git a/babel/src/test/java/org/apache/calcite/test/BabelTest.java b/babel/src/test/java/org/apache/calcite/test/BabelTest.java
new file mode 100644
index 000000000000..64301cf6e5c1
--- /dev/null
+++ b/babel/src/test/java/org/apache/calcite/test/BabelTest.java
@@ -0,0 +1,163 @@
+/*
+ * 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.calcite.test;
+
+import org.apache.calcite.config.CalciteConnectionProperty;
+import org.apache.calcite.sql.parser.SqlParserFixture;
+import org.apache.calcite.sql.parser.babel.SqlBabelParserImpl;
+
+import org.junit.jupiter.api.Test;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+import java.sql.ResultSetMetaData;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.sql.Types;
+import java.util.Properties;
+import java.util.function.UnaryOperator;
+
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.core.Is.is;
+
+/**
+ * Unit tests for Babel framework.
+ */
+class BabelTest {
+
+ static final String URL = "jdbc:calcite:";
+
+ private static UnaryOperator useParserFactory() {
+ return propBuilder ->
+ propBuilder.set(CalciteConnectionProperty.PARSER_FACTORY,
+ SqlBabelParserImpl.class.getName() + "#FACTORY");
+ }
+
+ private static UnaryOperator useLibraryList(
+ String libraryList) {
+ return propBuilder ->
+ propBuilder.set(CalciteConnectionProperty.FUN, libraryList);
+ }
+
+ private static UnaryOperator useLenientOperatorLookup(
+ boolean lenient) {
+ return propBuilder ->
+ propBuilder.set(CalciteConnectionProperty.LENIENT_OPERATOR_LOOKUP,
+ Boolean.toString(lenient));
+ }
+
+ static Connection connect() throws SQLException {
+ return connect(UnaryOperator.identity());
+ }
+
+ static Connection connect(UnaryOperator propBuild)
+ throws SQLException {
+ final CalciteAssert.PropBuilder propBuilder = CalciteAssert.propBuilder();
+ final Properties info =
+ propBuild.andThen(useParserFactory())
+ .andThen(useLenientOperatorLookup(true))
+ .apply(propBuilder)
+ .build();
+ return DriverManager.getConnection(URL, info);
+ }
+
+ @Test void testInfixCast() throws SQLException {
+ try (Connection connection = connect(useLibraryList("standard,postgresql"));
+ Statement statement = connection.createStatement()) {
+ checkInfixCast(statement, "integer", Types.INTEGER);
+ checkInfixCast(statement, "varchar", Types.VARCHAR);
+ checkInfixCast(statement, "boolean", Types.BOOLEAN);
+ checkInfixCast(statement, "double", Types.DOUBLE);
+ checkInfixCast(statement, "bigint", Types.BIGINT);
+ }
+ }
+
+ private void checkInfixCast(Statement statement, String typeName, int sqlType)
+ throws SQLException {
+ final String sql = "SELECT x::" + typeName + "\n"
+ + "FROM (VALUES ('1', '2')) as tbl(x, y)";
+ try (ResultSet resultSet = statement.executeQuery(sql)) {
+ final ResultSetMetaData metaData = resultSet.getMetaData();
+ assertThat("Invalid column count", metaData.getColumnCount(), is(1));
+ assertThat("Invalid column type", metaData.getColumnType(1),
+ is(sqlType));
+ }
+ }
+
+ /** Tests that you can run tests via {@link Fixtures}. */
+ @Test void testFixtures() {
+ final SqlValidatorFixture v = Fixtures.forValidator();
+ v.withSql("select ^1 + date '2002-03-04'^")
+ .fails("(?s).*Cannot apply '\\+' to arguments of"
+ + " type ' \\+ '.*");
+
+ v.withSql("select 1 + 2 as three")
+ .type("RecordType(INTEGER NOT NULL THREE) NOT NULL");
+
+ // 'as' as identifier is invalid with Core parser
+ final SqlParserFixture p = Fixtures.forParser();
+ p.sql("select ^as^ from t")
+ .fails("(?s)Encountered \"as\".*");
+
+ // 'as' as identifier is invalid if you use Babel's tester and Core parser
+ p.sql("select ^as^ from t")
+ .withTester(new BabelParserTest.BabelTesterImpl())
+ .fails("(?s)Encountered \"as\".*");
+
+ // 'as' as identifier is valid with Babel parser
+ p.withConfig(c -> c.withParserFactory(SqlBabelParserImpl.FACTORY))
+ .sql("select as from t")
+ .ok("SELECT `AS`\n"
+ + "FROM `T`");
+
+ // Postgres cast is invalid with core parser
+ p.sql("select 1 ^:^: integer as x")
+ .fails("(?s).*Encountered \":\" at .*");
+ }
+
+ @Test void testNullSafeEqual() {
+ // x <=> y
+ checkSqlResult("mysql", "SELECT 1 <=> NULL", "EXPR$0=false\n");
+ checkSqlResult("mysql", "SELECT NULL <=> NULL", "EXPR$0=true\n");
+ // (a, b) <=> (x, y)
+ checkSqlResult("mysql",
+ "SELECT (CAST(NULL AS Integer), 1) <=> (1, CAST(NULL AS Integer))",
+ "EXPR$0=false\n");
+ checkSqlResult("mysql",
+ "SELECT (CAST(NULL AS Integer), CAST(NULL AS Integer))\n"
+ + "<=> (CAST(NULL AS Integer), CAST(NULL AS Integer))",
+ "EXPR$0=true\n");
+ // the higher precedence
+ checkSqlResult("mysql",
+ "SELECT x <=> 1 + 3 FROM (VALUES (1, 2)) as tbl(x,y)",
+ "EXPR$0=false\n");
+ // the lower precedence
+ checkSqlResult("mysql",
+ "SELECT NOT x <=> 1 FROM (VALUES (1, 2)) as tbl(x,y)",
+ "EXPR$0=false\n");
+ }
+
+ private void checkSqlResult(String funLibrary, String query, String result) {
+ CalciteAssert.that()
+ .with(CalciteConnectionProperty.PARSER_FACTORY,
+ SqlBabelParserImpl.class.getName() + "#FACTORY")
+ .with(CalciteConnectionProperty.FUN, funLibrary)
+ .query(query)
+ .returns(result);
+ }
+}
diff --git a/babel/src/test/java/org/apache/calcite/test/package-info.java b/babel/src/test/java/org/apache/calcite/test/package-info.java
new file mode 100644
index 000000000000..c06f789d71ca
--- /dev/null
+++ b/babel/src/test/java/org/apache/calcite/test/package-info.java
@@ -0,0 +1,28 @@
+/*
+ * 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.
+ */
+
+/**
+ * Tests for Calcite.
+ */
+@DefaultQualifier(value = NonNull.class, locations = TypeUseLocation.FIELD)
+@DefaultQualifier(value = NonNull.class, locations = TypeUseLocation.PARAMETER)
+@DefaultQualifier(value = NonNull.class, locations = TypeUseLocation.RETURN)
+package org.apache.calcite.test;
+
+import org.checkerframework.checker.nullness.qual.NonNull;
+import org.checkerframework.framework.qual.DefaultQualifier;
+import org.checkerframework.framework.qual.TypeUseLocation;
diff --git a/babel/src/test/resources/sql/big-query.iq b/babel/src/test/resources/sql/big-query.iq
new file mode 100755
index 000000000000..6792fb6ed2f1
--- /dev/null
+++ b/babel/src/test/resources/sql/big-query.iq
@@ -0,0 +1,137 @@
+# big-query.iq - Babel test for BigQuery dialect of SQL
+#
+# 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.
+#
+!use scott-big-query
+!set outputformat csv
+
+# TIMESTAMP_SECONDS, TIMESTAMP_MILLIS, TIMESTAMP_MICROS
+select v,
+ timestamp_seconds(v) as t0,
+ timestamp_millis(v * 1000) as t1,
+ timestamp_micros(v * 1000 * 1000) as t2
+from (values cast(0 as bigint),
+ cast(null as bigint),
+ cast(1230219000 as bigint),
+ cast(-1230219000 as bigint)) as t (v)
+order by v;
+V, T0, T1, T2
+-1230219000, 1931-01-07 08:30:00, 1931-01-07 08:30:00, 1931-01-07 08:30:00
+0, 1970-01-01 00:00:00, 1970-01-01 00:00:00, 1970-01-01 00:00:00
+1230219000, 2008-12-25 15:30:00, 2008-12-25 15:30:00, 2008-12-25 15:30:00
+null, null, null, null
+!ok
+
+select timestamp_seconds(1234567890) as t;
+T
+2009-02-13 23:31:30
+!ok
+
+select timestamp_millis(1234567890) as t;
+T
+1970-01-15 06:56:07
+!ok
+
+select timestamp_micros(1234567890) as t;
+T
+1970-01-01 00:20:34
+!ok
+
+# UNIX_SECONDS, UNIX_MILLIS, UNIX_MICROS
+select v,
+ unix_seconds(v) as t0,
+ unix_millis(v) as t1,
+ unix_micros(v) as t2
+from (values TIMESTAMP '1970-01-01 00:00:00',
+ cast(null as timestamp),
+ TIMESTAMP '2008-12-25 15:30:00',
+ TIMESTAMP '1931-01-07 08:30:00') as t (v)
+order by v;
+V, T0, T1, T2
+1931-01-07 08:30:00, -1230219000, -1230219000000, -1230219000000000
+1970-01-01 00:00:00, 0, 0, 0
+2008-12-25 15:30:00, 1230219000, 1230219000000, 1230219000000000
+null, null, null, null
+!ok
+
+select unix_seconds(timestamp '2008-12-25 15:30:00') as t;
+T
+1230219000
+!ok
+
+select unix_millis(timestamp '2008-12-25 15:30:00') as t;
+T
+1230219000000
+!ok
+
+select unix_micros(timestamp '2008-12-25 15:30:00') as t;
+T
+1230219000000000
+!ok
+
+# DATE_FROM_UNIX_DATE
+select v,
+ date_from_unix_date(v) as d
+from (values 0,
+ cast(null as integer),
+ 1230219000 / 86400,
+ -1230219000 / 86400) as t (v)
+order by v;
+V, D
+-14238, 1931-01-08
+0, 1970-01-01
+14238, 2008-12-25
+null, null
+!ok
+
+select date_from_unix_date(14238);
+EXPR$0
+2008-12-25
+!ok
+
+# UNIX_DATE
+select v,
+ unix_date(v) as d
+from (values date '1970-01-01',
+ cast(null as date),
+ DATE '2008-12-25',
+ DATE '1931-01-07') as t (v)
+order by v;
+V, D
+1931-01-07, -14239
+1970-01-01, 0
+2008-12-25, 14238
+null, null
+!ok
+
+select unix_date(timestamp '2008-12-25');
+EXPR$0
+14238
+!ok
+
+# DATE
+# 'date(x) is shorthand for 'cast(x as date)'
+select date('1970-01-01') as d;
+D
+1970-01-01
+!ok
+
+select date(cast(null as varchar(10))) as d;
+D
+null
+!ok
+
+# End big-query.iq
diff --git a/core/src/test/resources/log4j.properties b/babel/src/test/resources/sql/dummy.iq
old mode 100644
new mode 100755
similarity index 71%
rename from core/src/test/resources/log4j.properties
rename to babel/src/test/resources/sql/dummy.iq
index 834e2db6842e..e5aa26938bb2
--- a/core/src/test/resources/log4j.properties
+++ b/babel/src/test/resources/sql/dummy.iq
@@ -1,3 +1,5 @@
+# dummy.iq
+#
# 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.
@@ -12,13 +14,14 @@
# 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.
+#
+!use scott
+!set outputformat mysql
-# Root logger is configured at INFO and is sent to A1
-log4j.rootLogger=INFO, A1
+# VALUES as top-level (not Oracle)
+VALUES 1 + 2;
-# A1 goes to the console
-log4j.appender.A1=org.apache.log4j.ConsoleAppender
+VALUES ROW(1 + 2)
+!explain-validated-on calcite postgres
-# Set the pattern for each log message
-log4j.appender.A1.layout=org.apache.log4j.PatternLayout
-log4j.appender.A1.layout.ConversionPattern=%d [%t] %-5p - %m%n
+# End dummy.iq
diff --git a/babel/src/test/resources/sql/redshift.iq b/babel/src/test/resources/sql/redshift.iq
new file mode 100755
index 000000000000..5cfc27978abd
--- /dev/null
+++ b/babel/src/test/resources/sql/redshift.iq
@@ -0,0 +1,2206 @@
+# redshift.iq - Babel test for Redshift dialect of SQL
+#
+# 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.
+#
+# The following functions have ordinary syntax and are not defined in Calcite.
+# We call them using '!explain-validated-on calcite', which generates a plan;
+# if we tried to execute, using '!ok', the command would fail. Ideally we would
+# define each of these in Calcite's Redshift (or PostgreSQL) operator table,
+# and we could execute the functions.
+# get_bit
+# get_byte
+# set_bit
+# set_byte
+# to_ascii
+# isfinite
+# now()
+# date_cmp
+# date_cmp_timestamp
+# date_cmp_timestamptz
+# date_part_year
+# date_trunc
+# interval_cmp
+# getdate()
+# nvl2
+# age
+# add_months
+# trunc
+# to_hex
+# random
+# pow
+# log
+# dlog10
+# dlog1
+# checksum
+# cbrt
+# to_timestamp
+# timezone
+# bpcharcmp
+# btrim
+# charindex
+# bttext_pattern_cmp
+# crc32
+# func_sha1
+# convert_timezone
+#
+# The following functions work correctly, and can be executed, but we use
+# '!explain-validated-on' because the results of execution differ each time:
+# current_date
+# current_time
+# current_timestamp
+# localtime
+# localtimestamp
+#
+# lenientAgg - aggregate functions not in operator table:
+# median
+# bool_and
+# bool_or
+# percentile_cont
+# percentile_disc
+# cume_dist
+# percent_rank
+# ratio_to_report
+#
+# approximate - keyword before aggregate functions, e.g. 'approximate avg(x)'
+#
+# atTimeZone - AT TIME ZONE operator
+#
+# firstValueRespectNulls - RESPECT NULLS in FIRST_VALUE and LAST_VALUE
+#
+# leadIgnoreNulls - allow IGNORE NULLS in LEAD, LAG, NTH_VALUE
+#
+# nvlVarArgs - NVL with more than 2 arguments
+#
+# coerce - apply CONVERT_TIMEZONE to CHAR rather than TIMESTAMP last arg,
+# or with only 2 args,
+# apply DATEADD to CHAR rather than DATE
+# apply CONVERT_TIMEZONE to CHAR rather than DATE
+#
+# dateAdd - "No match found for function signature DATEADD(,
+# , )" due to "month" argument;
+# similarly DATEDIFF and DATE_PART;
+# similarly 'm' and 'qtr' arguments
+#
+# sysdate - operator with no parens:
+# SYSDATE
+# CURRENT_USER_ID
+#
+# emptyParens - Allow nilary built-in functions to be called with empty parens:
+# PI
+# CURRENT_SCHEMA
+#
+# position - Non-ANSI position function; similarly CONVERT
+#
+# pluralInterval - allow INTERVAL '2' DAYS as well as INTERVAL '2' DAY; [CALCITE-3383]
+#
+# TODO:
+# * Why does CONCAT not work?
+#
+!use scott-redshift
+!set outputformat csv
+
+# 1 Leader Node–Only Functions
+# The following SQL functions are leader-node only functions and are not
+# supported on the compute nodes:
+#
+# 1.1 System information functions
+# CURRENT_SCHEMA
+# CURRENT_SCHEMAS
+# HAS_DATABASE_PRIVILEGE
+# HAS_SCHEMA_PRIVILEGE
+# HAS_TABLE_PRIVILEGE
+
+# 1.2.1 Date functions
+# AGE
+# CURRENT_TIME
+# CURRENT_TIMESTAMP
+# LOCALTIME
+# ISFINITE
+# NOW
+
+# 1.2.2 String functions
+
+# ASCII
+select ascii('xyz');
+EXPR$0
+120
+!ok
+
+# GET_BIT
+select get_bit(CAST('FFFF' as BINARY), 1);
+SELECT "GET_BIT"(CAST('FFFF' AS BINARY), 1)
+!explain-validated-on calcite
+
+# GET_BYTE
+select get_byte(CAST('FFFF' as BINARY), 1);
+SELECT "GET_BYTE"(CAST('FFFF' AS BINARY), 1)
+!explain-validated-on calcite
+
+# SET_BIT
+select set_bit(CAST('FFFF' as BINARY), 1, 61);
+SELECT "SET_BIT"(CAST('FFFF' AS BINARY), 1, 61)
+!explain-validated-on calcite
+
+# SET_BYTE
+select set_byte(CAST('FFFF' as BINARY), 1, 61);
+SELECT "SET_BYTE"(CAST('FFFF' AS BINARY), 1, 61)
+!explain-validated-on calcite
+
+# TO_ASCII
+select to_ascii(120);
+SELECT "TO_ASCII"(120)
+!explain-validated-on calcite
+
+# 2 Compute Node–Only Functions
+#
+# The following SQL functions must execute only on the compute nodes.
+#
+# LISTAGG
+# MEDIAN
+# PERCENTILE_CONT
+# PERCENTILE_DISC and APPROXIMATE PERCENTILE_DISC
+
+# 3 Aggregate Functions
+
+# APPROXIMATE PERCENTILE_DISC
+!if (approximate) {
+select approximate percentile_disc(0.5) within group (order by sal) from emp group by deptno;
+!ok
+!}
+
+# AVG
+select avg(sal) from emp;
+EXPR$0
+2073.214285714286
+!ok
+
+# COUNT
+!if (approximate) {
+select approximate count(distinct sal) from emp;
+!ok
+!}
+
+select count(*) from emp;
+EXPR$0
+14
+!ok
+
+select count(sal) from emp;
+EXPR$0
+14
+!ok
+
+select count(all sal) from emp;
+EXPR$0
+14
+!ok
+
+select count(distinct sal) from emp;
+EXPR$0
+12
+!ok
+
+# LISTAGG
+select listagg(empno) from emp group by deptno;
+EXPR$0
+7369,7566,7788,7876,7902
+7499,7521,7654,7698,7844,7900
+7782,7839,7934
+!ok
+
+select listagg(empno) within group (order by sal) from emp group by deptno;
+EXPR$0
+7369,7876,7566,7788,7902
+7900,7521,7654,7844,7499,7698
+7934,7782,7839
+!ok
+
+select listagg(empno, ',') from emp group by deptno;
+EXPR$0
+7369,7566,7788,7876,7902
+7499,7521,7654,7698,7844,7900
+7782,7839,7934
+!ok
+
+# MAX
+select max(distinct sal) from emp;
+EXPR$0
+5000.00
+!ok
+
+select max(all sal) from emp;
+EXPR$0
+5000.00
+!ok
+
+select max(sal) from emp;
+EXPR$0
+5000.00
+!ok
+
+# MEDIAN
+!if (lenientAgg) {
+select median(sal) from emp;
+!ok
+!}
+
+# MIN
+select min(distinct sal) from emp;
+EXPR$0
+800.00
+!ok
+
+select min(all sal) from emp;
+EXPR$0
+800.00
+!ok
+
+select min(sal) from emp;
+EXPR$0
+800.00
+!ok
+
+# PERCENTILE_CONT
+!if (lenientAgg) {
+select percentile_cont(0.6) within group (order by sal) from emp group by deptno;
+!ok
+!}
+
+# PERCENTILE_DISC
+!if (lenientAgg) {
+select percentile_disc(0.6) within group (order by sal) from emp group by deptno;
+!ok
+!}
+
+# STDDEV_SAMP and STDDEV_POP
+select stddev_samp(sal) from emp;
+EXPR$0
+1182.503223516271873450023122131824493408203125
+!ok
+
+select stddev_pop(sal) from emp;
+EXPR$0
+1139.488618295281639802851714193820953369140625
+!ok
+
+# SUM
+select sum(sal) from emp;
+EXPR$0
+29025.00
+!ok
+
+select sum(distinct sal) from emp;
+EXPR$0
+24775.00
+!ok
+
+# VAR_SAMP and VAR_POP
+select var_samp(sal) from emp;
+EXPR$0
+1398313.873626374
+!ok
+
+select var_samp(distinct sal) from emp;
+EXPR$0
+1512779.356060606
+!ok
+
+select var_samp(all sal) from emp;
+EXPR$0
+1398313.873626374
+!ok
+
+select var_pop(sal) from emp;
+EXPR$0
+1298434.31122449
+!ok
+
+# 4 Bit-Wise Aggregate Functions
+
+# BIT_AND
+select bit_and(deptno) from emp;
+EXPR$0
+0
+!ok
+
+# BIT_OR
+select bit_or(deptno) from emp;
+EXPR$0
+30
+!ok
+
+# BOOL_AND operates on a single Boolean or integer column or expression
+!if (lenientAgg) {
+select bool_and(deptno < 20) from emp;
+!ok
+!}
+
+!if (lenientAgg) {
+select bool_and(deptno) from emp;
+!ok
+!}
+
+!if (lenientAgg) {
+select bool_and(distinct deptno) from emp;
+!ok
+!}
+
+# BOOL_OR operates on a single Boolean or integer column or expression
+!if (lenientAgg) {
+select bool_or(deptno < 20) from emp;
+!ok
+!}
+
+!if (lenientAgg) {
+select bool_or(deptno) from emp;
+!ok
+!}
+
+# 5 Window and ranking functions
+
+# 5.1 Window functions:
+
+# AVG
+select empno, avg(sal) over (order by empno rows unbounded preceding) from emp where deptno = 30 order by 1;
+EMPNO, EXPR$1
+7499, 1600.00
+7521, 1425.00
+7654, 1366.666666666667
+7698, 1737.50
+7844, 1690.00
+7900, 1566.666666666667
+!ok
+
+# COUNT
+select empno, count(comm) over (order by empno rows unbounded preceding) from emp where deptno = 30 order by 1;
+EMPNO, EXPR$1
+7499, 1
+7521, 2
+7654, 3
+7698, 3
+7844, 4
+7900, 4
+!ok
+
+# CUME_DIST
+!if (lenientAgg) {
+select empno, cume_dist() over (order by sal rows unbounded preceding) from emp where deptno = 30 order by 1;
+!ok
+!}
+
+# FIRST_VALUE
+select empno, first_value(sal) over (order by empno rows unbounded preceding) from emp where deptno = 30 order by 1;
+EMPNO, EXPR$1
+7499, 1600.00
+7521, 1600.00
+7654, 1600.00
+7698, 1600.00
+7844, 1600.00
+7900, 1600.00
+!ok
+
+!if (firstValueRespectNulls) {
+select empno, first_value(sal respect nulls) over (order by empno rows unbounded preceding) from emp where deptno = 30 order by 1;
+!ok
+!}
+
+# LAG
+select empno, lag(sal) respect nulls over (order by empno) from emp where deptno = 30 order by 1;
+EMPNO, EXPR$1
+7499, null
+7521, 1600.00
+7654, 1250.00
+7698, 1250.00
+7844, 2850.00
+7900, 1500.00
+!ok
+
+select empno, lag(sal, 2) respect nulls over (order by empno) from emp where deptno = 30 order by 1;
+EMPNO, EXPR$1
+7499, null
+7521, null
+7654, 1600.00
+7698, 1250.00
+7844, 1250.00
+7900, 2850.00
+!ok
+
+# LAST_VALUE
+!if (firstValueRespectNulls) {
+select empno, last_value(sal) over (order by empno rows unbounded preceding) from emp order by 1;
+!ok
+!}
+
+!if (firstValueRespectNulls) {
+select empno, last_value(sal respect nulls) over (order by empno rows unbounded preceding) from emp order by 1;
+!ok
+!}
+
+# LEAD
+!if (leadIgnoreNulls) {
+select empno, lead(sal, 2) ignore nulls over (order by empno) from emp order by 1;
+!ok
+!}
+
+# MAX
+select empno, max(comm) over (order by empno rows unbounded preceding) from emp where deptno = 30 order by 1;
+EMPNO, EXPR$1
+7499, 300.00
+7521, 500.00
+7654, 1400.00
+7698, 1400.00
+7844, 1400.00
+7900, 1400.00
+!ok
+
+# MEDIAN
+!if (lenientAgg) {
+select empno, median(comm) over (order by empno rows unbounded preceding) from emp where deptno = 30 order by 1;
+!ok
+!}
+
+# MIN
+select empno, min(comm) over (order by empno rows unbounded preceding) from emp where deptno = 30 order by 1;
+EMPNO, EXPR$1
+7499, 300.00
+7521, 300.00
+7654, 300.00
+7698, 300.00
+7844, 0.00
+7900, 0.00
+!ok
+
+# NTH_VALUE
+!if (leadIgnoreNulls) {
+select empno, nth_value(sal, 2) ignore nulls over (order by empno rows unbounded preceding) from emp order by 1;
+!ok
+!}
+
+# PERCENTILE_CONT
+!if (lenientAgg) {
+select percentile_cont(0.6) within group (order by sal) over () from emp;
+!ok
+!}
+
+!if (lenientAgg) {
+select percentile_cont(0.6) within group (order by sal) over (partition by deptno) from emp;
+!ok
+!}
+
+# PERCENTILE_DISC
+!if (lenientAgg) {
+select percentile_disc(0.6) within group (order by sal) over () from emp;
+!ok
+!}
+
+!if (lenientAgg) {
+select percentile_disc(0.6) within group (order by sal) over (partition by deptno) from emp;
+!ok
+!}
+
+# RATIO_TO_REPORT
+!if (lenientAgg) {
+select ratio_to_report(sal) over () from emp where deptno = 30;
+!ok
+!}
+
+!if (lenientAgg) {
+select deptno, ratio_to_report(sal) over (partition by deptno) from emp;
+!ok
+!}
+
+# STDDEV_POP
+select empno, stddev_pop(comm) over (order by empno rows unbounded preceding) from emp where deptno = 30 order by 1;
+EMPNO, EXPR$1
+7499, 0
+7521, 100
+7654, 478.42333648024424519462627358734607696533203125
+7698, 478.42333648024424519462627358734607696533203125
+7844, 522.0153254455275373402400873601436614990234375
+7900, 522.0153254455275373402400873601436614990234375
+!ok
+
+# STDDEV_SAMP (synonym for STDDEV)
+select empno, stddev_samp(comm) over (order by empno rows unbounded preceding) from emp where deptno = 30 order by 1;
+EMPNO, EXPR$1
+7499, null
+7521, 141.421356237309510106570087373256683349609375
+7654, 585.9465277082316561063635163009166717529296875
+7698, 585.9465277082316561063635163009166717529296875
+7844, 602.7713773341707792496890760958194732666015625
+7900, 602.7713773341707792496890760958194732666015625
+!ok
+
+select empno, stddev(comm) over (order by empno rows unbounded preceding) from emp where deptno = 30 order by 1;
+EMPNO, EXPR$1
+7499, null
+7521, 141.421356237309510106570087373256683349609375
+7654, 585.9465277082316561063635163009166717529296875
+7698, 585.9465277082316561063635163009166717529296875
+7844, 602.7713773341707792496890760958194732666015625
+7900, 602.7713773341707792496890760958194732666015625
+!ok
+
+# SUM
+select empno, sum(comm) over (order by empno rows unbounded preceding) from emp where deptno = 30 order by 1;
+EMPNO, EXPR$1
+7499, 300.00
+7521, 800.00
+7654, 2200.00
+7698, 2200.00
+7844, 2200.00
+7900, 2200.00
+!ok
+
+# VAR_POP
+select empno, var_pop(comm) over (order by empno rows unbounded preceding) from emp where deptno = 30 order by 1;
+EMPNO, EXPR$1
+7499, 0.0000
+7521, 10000.0000
+7654, 228888.888888889
+7698, 228888.888888889
+7844, 272500.0000
+7900, 272500.0000
+!ok
+
+# VAR_SAMP (synonym for VARIANCE)
+select empno, var_samp(comm) over (order by empno rows unbounded preceding) from emp where deptno = 30 order by 1;
+EMPNO, EXPR$1
+7499, null
+7521, 20000.0000
+7654, 343333.3333333335
+7698, 343333.3333333335
+7844, 363333.3333333333
+7900, 363333.3333333333
+!ok
+
+select empno, variance(comm) over (order by empno rows unbounded preceding) from emp where deptno = 30 order by 1;
+EMPNO, EXPR$1
+7499, null
+7521, 20000.0000
+7654, 343333.3333333335
+7698, 343333.3333333335
+7844, 363333.3333333333
+7900, 363333.3333333333
+!ok
+
+# 5.2 Ranking functions
+
+# DENSE_RANK
+select dense_rank() over () from emp where deptno = 30;
+EXPR$0
+6
+1
+2
+3
+4
+5
+!ok
+
+select dense_rank() over (partition by deptno) from emp;
+EXPR$0
+3
+3
+3
+5
+5
+6
+1
+1
+1
+2
+2
+2
+4
+4
+!ok
+
+select dense_rank() over (partition by deptno order by sal) from emp;
+EXPR$0
+1
+1
+1
+2
+2
+2
+2
+3
+3
+3
+4
+4
+4
+5
+!ok
+
+# NTILE
+select ntile(4) over (order by sal desc) from emp;
+EXPR$0
+1
+1
+1
+1
+2
+2
+2
+3
+3
+3
+3
+4
+4
+4
+!ok
+
+# PERCENT_RANK
+!if (lenientAgg) {
+select percent_rank() over () from emp;
+!ok
+!}
+
+!if (lenientAgg) {
+select percent_rank() over (partition by deptno) from emp;
+!ok
+!}
+
+!if (lenientAgg) {
+select percent_rank() over (partition by deptno order by sal) from emp;
+!ok
+!}
+
+# RANK
+select rank() over () from emp;
+EXPR$0
+14
+1
+10
+11
+12
+13
+2
+3
+4
+5
+6
+7
+8
+9
+!ok
+
+select rank() over (partition by deptno) from emp;
+EXPR$0
+3
+3
+3
+5
+5
+6
+1
+1
+1
+2
+2
+2
+4
+4
+!ok
+
+select rank() over (partition by deptno order by sal) from emp;
+EXPR$0
+1
+1
+1
+2
+2
+2
+2
+3
+3
+4
+4
+4
+5
+6
+!ok
+
+# ROW_NUMBER
+select row_number() over () from emp;
+EXPR$0
+1
+10
+11
+12
+13
+14
+2
+3
+4
+5
+6
+7
+8
+9
+!ok
+
+# 6 Conditional Expressions
+
+# CASE
+select case when deptno < 20 then 'x' else 'y' end from emp;
+EXPR$0
+y
+y
+y
+y
+y
+y
+y
+y
+y
+y
+y
+x
+x
+x
+!ok
+
+select case when deptno < 20 then 'x' end from emp;
+EXPR$0
+null
+null
+null
+null
+null
+null
+null
+null
+null
+null
+null
+x
+x
+x
+!ok
+
+select case deptno when 10 then 'x' when 20 then 'y' end from emp;
+EXPR$0
+null
+null
+null
+null
+null
+null
+x
+x
+x
+y
+y
+y
+y
+y
+!ok
+
+select case deptno when 10 then 'x' when 20 then 'y' else 'z' end from emp;
+EXPR$0
+x
+x
+x
+y
+y
+y
+y
+y
+z
+z
+z
+z
+z
+z
+!ok
+
+# COALESCE is a synonym for NVL
+select coalesce(1, 2, 3);
+EXPR$0
+1
+!ok
+
+# DECODE
+select decode(deptno, 10, 'x', 20, 'y', 'z') from emp;
+EXPR$0
+x
+x
+x
+y
+y
+y
+y
+y
+z
+z
+z
+z
+z
+z
+!ok
+
+# GREATEST and LEAST
+select greatest(deptno) from emp where deptno = 30;
+EXPR$0
+30
+30
+30
+30
+30
+30
+!ok
+
+select greatest(deptno, empno) from emp where deptno = 30;
+EXPR$0
+7499
+7521
+7654
+7698
+7844
+7900
+!ok
+
+select greatest(deptno, empno, sal) from emp where deptno = 30;
+EXPR$0
+7499
+7521
+7654
+7698
+7844
+7900
+!ok
+
+select least(deptno, empno) from emp where deptno = 30;
+EXPR$0
+30
+30
+30
+30
+30
+30
+!ok
+
+# NVL
+select nvl(1, 2);
+EXPR$0
+1
+!ok
+
+!if (nvlVarArgs) {
+select nvl(1, 2, 3);
+!ok
+!}
+
+select nvl(comm, sal) from emp where deptno = 30;
+EXPR$0
+0.00
+1400.00
+2850.00
+300.00
+500.00
+950.00
+!ok
+
+# NVL2
+select nvl2(comm, sal, sal + 10) from emp where deptno = 30;
+SELECT "NVL2"("EMP"."COMM", "EMP"."SAL", "EMP"."SAL" + 10)
+FROM "scott"."EMP" AS "EMP"
+WHERE "EMP"."DEPTNO" = 30
+!explain-validated-on calcite
+
+# NULLIF
+select nullif(comm, sal) from emp;
+EXPR$0
+0.00
+1400.00
+300.00
+500.00
+null
+null
+null
+null
+null
+null
+null
+null
+null
+null
+!ok
+
+# 7 Date and Time Functions
+
+# 7.1 Deprecated leader-node only functions
+
+# The following date functions are deprecated because they execute only on
+# the leader node.
+
+# AGE. Use DATEDIFF Function instead.
+select age('2017-01-01','2011-06-24');
+SELECT "AGE"('2017-01-01', '2011-06-24')
+!explain-validated-on calcite
+
+# CURRENT_TIME. Use GETDATE Function or SYSDATE instead.
+select current_time;
+SELECT CURRENT_TIME AS CURRENT_TIME
+!explain-validated-on calcite
+
+select current_time(2);
+SELECT CURRENT_TIME(2)
+!explain-validated-on calcite
+
+# CURRENT_TIMESTAMP. Use GETDATE Function or SYSDATE instead.
+select current_timestamp;
+SELECT CURRENT_TIMESTAMP AS CURRENT_TIMESTAMP
+!explain-validated-on calcite
+
+select current_timestamp(2);
+SELECT CURRENT_TIMESTAMP(2)
+!explain-validated-on calcite
+
+# LOCALTIME. Use GETDATE Function or SYSDATE instead.
+select localtime;
+SELECT LOCALTIME AS LOCALTIME
+!explain-validated-on calcite
+
+select localtime(2);
+SELECT LOCALTIME(2)
+!explain-validated-on calcite
+
+# LOCALTIMESTAMP. Use GETDATE Function or SYSDATE instead.
+select localtimestamp;
+SELECT LOCALTIMESTAMP AS LOCALTIMESTAMP
+!explain-validated-on calcite
+
+# ISFINITE
+select isfinite(date '2002-09-17');
+SELECT "ISFINITE"(DATE '2002-09-17')
+!explain-validated-on calcite
+
+# NOW. Use GETDATE Function or SYSDATE instead.
+select now();
+SELECT "NOW"()
+!explain-validated-on calcite
+
+# 7.2 Date and Time functions
+
+# ADD_MONTHS ({date|timestamp}, integer) returns TIMESTAMP
+# Adds the specified number of months to a date or time stamp.
+# If the date you are adding to is the last day of the month, the result is
+# always the last day of the result month, regardless of the length of the month.
+
+select add_months('2008-03-31',1);
+SELECT "ADD_MONTHS"('2008-03-31', 1)
+!explain-validated-on calcite
+
+-- returns '2008-04-30'
+select add_months(date '2008-03-31',1);
+SELECT "ADD_MONTHS"(DATE '2008-03-31', 1)
+!explain-validated-on calcite
+
+-- returns '2008-05-31'
+select add_months(date '2008-04-30',1);
+SELECT "ADD_MONTHS"(DATE '2008-04-30', 1)
+!explain-validated-on calcite
+
+select add_months(date '2008-03-31',-1);
+SELECT "ADD_MONTHS"(DATE '2008-03-31', -1)
+!explain-validated-on calcite
+
+select add_months(timestamp '2008-03-31 12:23:34',1);
+SELECT "ADD_MONTHS"(TIMESTAMP '2008-03-31 12:23:34', 1)
+!explain-validated-on calcite
+
+# AT TIME ZONE 'timezone' returns TIMESTAMP
+# Specifies which time zone to use with a TIMESTAMP or TIMESTAMPTZ expression.
+!if (atTimeZone) {
+-- returns '2001-02-16 19:38:40-08'
+SELECT TIMESTAMP '2001-02-16 20:38:40' AT TIME ZONE 'MST';
+!ok
+!}
+
+!if (atTimeZone) {
+-- returns '2001-02-16 18:38:40'
+SELECT TIMESTAMP WITH TIME ZONE '2001-02-16 20:38:40-05' AT TIME ZONE 'MST';
+!ok
+!}
+
+# CONVERT_TIMEZONE (['timezone',] 'timezone', timestamp) returns TIMESTAMP
+# Converts a time stamp from one time zone to another.
+-- returns '2008-03-05 09:25:29'
+select convert_timezone('EST', 'PST', timestamp '2008-03-05 12:25:29');
+SELECT "CONVERT_TIMEZONE"('EST', 'PST', TIMESTAMP '2008-03-05 12:25:29')
+!explain-validated-on calcite
+
+!if (coerce) {
+-- returns '2008-03-05 09:25:29'
+select convert_timezone('EST', 'PST', '20080305 12:25:29');
+!ok
+!}
+
+!if (coerce) {
+-- returns '2013-02-01 03:00:00'
+select convert_timezone('America/New_York', '2013-02-01 08:00:00');
+!ok
+!}
+
+!if (coerce) {
+-- returns '2014-05-17 10:00:00'
+select CONVERT_TIMEZONE('GMT','GMT+2','2014-05-17 12:00:00');
+!ok
+!}
+
+# CURRENT_DATE returns DATE
+# Returns a date in the current session time zone (UTC by default) for the start
+# of the current transaction.
+select current_date;
+SELECT CURRENT_DATE AS CURRENT_DATE
+!explain-validated-on calcite
+
+# DATE_CMP (date1, date2) returns INTEGER
+# Compares two dates and returns 0 if the dates are identical, 1 if date1 is
+# greater, and -1 if date2 is greater.
+-- returns -1
+select date_cmp('2008-01-01', '2008-01-04');
+SELECT "DATE_CMP"('2008-01-01', '2008-01-04')
+!explain-validated-on calcite
+
+-- returns 0
+select date_cmp(date '2008-01-04', '2008-01-04');
+SELECT "DATE_CMP"(DATE '2008-01-04', '2008-01-04')
+!explain-validated-on calcite
+
+-- returns 1
+select date_cmp(date '2008-01-05', date '2008-01-04');
+SELECT "DATE_CMP"(DATE '2008-01-05', DATE '2008-01-04')
+!explain-validated-on calcite
+
+# DATE_CMP_TIMESTAMP (date, timestamp) returns INTEGER
+# Compares a date to a time and returns 0 if the values are identical, 1 if date
+# is greater and -1 if timestamp is greater.
+-- returns -1
+select date_cmp_timestamp('2008-01-01', '2008-01-04 00:00:00');
+SELECT "DATE_CMP_TIMESTAMP"('2008-01-01', '2008-01-04 00:00:00')
+!explain-validated-on calcite
+
+-- returns 0
+select date_cmp_timestamp(date '2008-01-04', '2008-01-04 00:00:00');
+SELECT "DATE_CMP_TIMESTAMP"(DATE '2008-01-04', '2008-01-04 00:00:00')
+!explain-validated-on calcite
+
+-- I presume that this returns -1, but doc does not specify
+select date_cmp_timestamp(date '2008-01-04', '2008-01-04 01:23:45');
+SELECT "DATE_CMP_TIMESTAMP"(DATE '2008-01-04', '2008-01-04 01:23:45')
+!explain-validated-on calcite
+
+-- returns 1
+select date_cmp_timestamp(date '2008-01-05', timestamp '2008-01-04 00:00:00');
+SELECT "DATE_CMP_TIMESTAMP"(DATE '2008-01-05', TIMESTAMP '2008-01-04 00:00:00')
+!explain-validated-on calcite
+
+# DATE_CMP_TIMESTAMPTZ (date, timestamptz) returns INTEGER
+# Compares a date and a time stamp with time zone and returns 0 if the values
+# are identical, 1 if date is greater and -1 if timestamptz is greater.
+!if (atTimeZone) {
+-- returns -1
+select date_cmp_timestamptz('2008-01-01', '2008-01-04 00:00:00' at time zone 'gmt');
+!ok
+!}
+
+# DATE_PART_YEAR (date) returns INTEGER
+# Extracts the year from a date.
+-- returns 2008
+select date_part_year('2008-01-05');
+SELECT "DATE_PART_YEAR"('2008-01-05')
+!explain-validated-on calcite
+
+select date_part_year(date '2008-01-05');
+SELECT "DATE_PART_YEAR"(DATE '2008-01-05')
+!explain-validated-on calcite
+
+# DATEADD (datepart, interval, {date|timestamp}) returns TIMESTAMP
+# Increments a date or time by a specified interval.
+# The DATEADD(month, ...) and ADD_MONTHS functions handle dates that fall at the
+# ends of months differently.
+!if (dateAdd) {
+-- returns '2009-08-28 00:00:00'
+select dateadd(month,18,date '2008-02-28');
+!ok
+!}
+
+!if (dateAdd) {
+-- as above, for uses char rather than date
+select dateadd(month,18,'2008-02-28');
+!ok
+!}
+
+!if (dateAdd) {
+-- returns '2004-05-30 00:00:00' (ADD_MONTHS would return '2004-05-31')
+select dateadd(month,1,date '2004-04-30');
+!ok
+!}
+
+!if (dateAdd) {
+-- returns '2008-02-28 00:30:00' ('m' stands for minutes, not months)
+select dateadd(m,18,date '2008-02-28');
+!ok
+!}
+
+!if (dateAdd) {
+-- returns '2017-02-28 00:00:00'
+select dateadd(month,12,date '2016-02-29');
+!ok
+!}
+
+!if (dateAdd) {
+-- returns '2017-03-01 00:00:00'
+select dateadd(year, 1, date '2016-02-29');
+!ok
+!}
+
+# DATEDIFF (datepart, {date|time}, {date|timestamp}) returns BIGINT
+# Returns the difference between two dates or times for a given date part, such
+# as a day or month.
+!if (dateAdd) {
+-- returns 52
+select datediff(week,date '2009-01-01',date '2009-12-31') as numweeks;
+!ok
+!}
+
+!if (dateAdd) {
+-- as above but CHAR rather than DATE
+select datediff(week,'2009-01-01','2009-12-31') as numweeks;
+!ok
+
+select datediff(week,date '2009-01-01','2009-12-31') as numweeks;
+!ok
+
+select datediff(week,'2009-01-01',date '2009-12-31') as numweeks;
+!ok
+!}
+
+!if (dateAdd) {
+-- returns 40 or more
+select datediff(qtr, date '1998-07-01', current_date);
+!ok
+!}
+
+!if (dateAdd) {
+select datediff(hours, date '1998-07-01', current_date);
+!ok
+!}
+
+!if (dateAdd) {
+select datediff(day, date '1998-07-01', current_date);
+!ok
+!}
+
+# DATE_PART (datepart, {date|time}) returns DOUBLE
+# Extracts a date part value from date or time.
+!if (dateAdd) {
+-- returns 25
+select date_part(w, timestamp '2008-06-17 09:44:54');
+!ok
+!}
+
+!if (dateAdd) {
+select date_part(w, timestamp '2008-06-17 09:44:54');
+!ok
+!}
+
+!if (dateAdd) {
+-- returns 8
+select date_part(minute, timestamp '2009-01-01 02:08:01');
+!ok
+!}
+
+!if (dateAdd) {
+select date_part(minute, time '02:08:01');
+!ok
+!}
+
+!if (dateAdd) {
+select date_part(minute, date '2009-01-01');
+!ok
+!}
+
+# DATE_TRUNC ('datepart', timestamp) returns TIMESTAMP
+# Truncates a time stamp based on a date part.
+-- returns '2008-09-01'
+select date_trunc('week', date '2008-09-07');
+SELECT "DATE_TRUNC"('week', DATE '2008-09-07')
+!explain-validated-on calcite
+
+-- as above, but CHAR rather than DATE
+select date_trunc('week', '2008-09-07');
+SELECT "DATE_TRUNC"('week', '2008-09-07')
+!explain-validated-on calcite
+
+-- returns '2008-09-08'
+select date_trunc('week', date '2008-09-08');
+SELECT "DATE_TRUNC"('week', DATE '2008-09-08')
+!explain-validated-on calcite
+
+-- returns '2008-09-08'
+select date_trunc('week', date '2008-09-09');
+SELECT "DATE_TRUNC"('week', DATE '2008-09-09')
+!explain-validated-on calcite
+
+# EXTRACT (datepart FROM {TIMESTAMP 'literal' | timestamp}) returns DOUBLE
+# Extracts a date part from a timestamp or literal.
+-- returns 8
+select extract(minute from timestamp '2009-09-09 12:08:43');
+EXPR$0
+8
+!ok
+
+!if (coerce) {
+-- as above, but CHAR rather than TIMESTAMP
+select extract(minute from '2009-09-09 12:08:43');
+!ok
+!}
+
+# GETDATE() returns TIMESTAMP
+# Returns the current date and time in the current session time zone (UTC by
+# default). The parentheses are required.
+select getdate();
+SELECT "GETDATE"()
+!explain-validated-on calcite
+
+# INTERVAL_CMP (interval1, interval2) returns INTEGER
+# Compares two intervals and returns 0 if the intervals are equal, 1 if
+# interval1 is greater, and -1 if interval2 is greater.
+!if (pluralInterval) {
+-- returns -1
+select interval_cmp(interval '3' days,interval '1');
+!explain-validated-on calcite
+!}
+
+-- as above but CHAR rather than INTERVAL
+select interval_cmp('3 days','1 year');
+SELECT "INTERVAL_CMP"('3 days', '1 year')
+!explain-validated-on calcite
+
+-- returns 0
+select interval_cmp('7 days','1 week');
+SELECT "INTERVAL_CMP"('7 days', '1 week')
+!explain-validated-on calcite
+
+-- should this return 0 or 1?
+select interval_cmp('366 days','1 year');
+SELECT "INTERVAL_CMP"('366 days', '1 year')
+!explain-validated-on calcite
+
+-- should this return -1, 0 or 1?
+select interval_cmp('30 days','1 month');
+SELECT "INTERVAL_CMP"('30 days', '1 month')
+!explain-validated-on calcite
+
+# LAST_DAY(date) returns DATE
+# Returns the date of the last day of the month that contains date.
+# Always returns DATE, even if argument is TIMESTAMP.
+-- returns '2004-01-31'
+select last_day(date '2004-01-25');
+EXPR$0
+2004-01-31
+!ok
+
+-- returns '2004-01-31'
+select last_day(timestamp '2004-01-25 12:23:45');
+EXPR$0
+2004-01-31
+!ok
+
+# MONTHS_BETWEEN (date, date) returns FLOAT8
+# Returns the number of months between two dates.
+-- returns -2
+select months_between('1969-01-18', '1969-03-18') as months;
+SELECT "MONTHS_BETWEEN"('1969-01-18', '1969-03-18') AS "MONTHS"
+!explain-validated-on calcite
+
+# NEXT_DAY (date, day) returns DATE
+# Returns the date of the first instance of day that is later than date.
+-- returns '2014-08-26'
+select next_day('2014-08-20','Tuesday');
+SELECT "NEXT_DAY"('2014-08-20', 'Tuesday')
+!explain-validated-on calcite
+
+-- returns '2014-08-26'
+select next_day('2014-08-20','Tu');
+SELECT "NEXT_DAY"('2014-08-20', 'Tu')
+!explain-validated-on calcite
+
+-- not valid ('T' could be 'Tue' or 'Thu')
+select next_day('2014-08-20','T');
+SELECT "NEXT_DAY"('2014-08-20', 'T')
+!explain-validated-on calcite
+
+-- returns '2014-08-22'
+select next_day('2014-08-20','F');
+SELECT "NEXT_DAY"('2014-08-20', 'F')
+!explain-validated-on calcite
+
+# SYSDATE returns TIMESTAMP
+# Returns the date and time in the current session time zone (UTC by default)
+# for the start of the current transaction.
+!if (sysdate) {
+select sysdate;
+!ok
+!}
+
+# TIMEOFDAY() returns VARCHAR
+# Returns the current weekday, date, and time in the current session time zone
+# (UTC by default) as a string value.
+-- returns something like 'Thu Sep 19 22:53:50.333525 2013 UTC'
+select timeofday();
+SELECT "TIMEOFDAY"()
+!explain-validated-on calcite
+
+# TIMESTAMP_CMP (timestamp1, timestamp2) returns INTEGER
+# Compares two timestamps and returns 0 if the timestamps are equal, 1 if
+# timestamp1 is greater, and -1 if timestamp2 is greater.
+-- returns -1
+select timestamp_cmp('2008-01-01 00:00:00', '2008-01-04 12:34:56');
+SELECT "TIMESTAMP_CMP"('2008-01-01 00:00:00', '2008-01-04 12:34:56')
+!explain-validated-on calcite
+
+# TIMESTAMP_CMP_DATE (timestamp, date) returns INTEGER
+# Compares a timestamp to a date and returns 0 if the values are equal, 1 if
+# timestamp is greater, and -1 if date is greater.
+-- returns -1
+select timestamp_cmp_date('2008-01-01 00:00:00', '2008-01-04');
+SELECT "TIMESTAMP_CMP_DATE"('2008-01-01 00:00:00', '2008-01-04')
+!explain-validated-on calcite
+
+# TIMESTAMP_CMP_TIMESTAMPTZ (timestamp, timestamptz) returns INTEGER
+# Compares a timestamp with a time stamp with time zone and returns 0 if the
+# values are equal, 1 if timestamp is greater, and -1 if timestamptz is greater.
+-- returns -1
+select timestamp_cmp_timestamptz('2008-01-01 00:00:00', '2008-01-04 00:00:00');
+SELECT "TIMESTAMP_CMP_TIMESTAMPTZ"('2008-01-01 00:00:00', '2008-01-04 00:00:00')
+!explain-validated-on calcite
+
+# TIMESTAMPTZ_CMP (timestamptz1, timestamptz2) returns INTEGER
+# Compares two timestamp with time zone values and returns 0 if the values are
+# equal, 1 if timestamptz1 is greater, and -1 if timestamptz2 is greater.
+-- returns -1
+select timestamptz_cmp('2008-01-01 00:00:00', '2008-01-04 00:00:00');
+SELECT "TIMESTAMPTZ_CMP"('2008-01-01 00:00:00', '2008-01-04 00:00:00')
+!explain-validated-on calcite
+
+# TIMESTAMPTZ_CMP_DATE (timestamptz, date) returns INTEGER
+# Compares the value of a time stamp with time zone and a date and returns 0 if
+# the values are equal, 1 if timestamptz is greater, and -1 if date is greater.
+-- returns -1
+select timestamptz_cmp_date('2008-01-01 00:00:00', '2008-01-04');
+SELECT "TIMESTAMPTZ_CMP_DATE"('2008-01-01 00:00:00', '2008-01-04')
+!explain-validated-on calcite
+
+# TIMESTAMPTZ_CMP_TIMESTAMP (timestamptz, timestamp) returns INTEGER
+# Compares a timestamp with time zone with a time stamp and returns 0 if the
+# values are equal, 1 if timestamptz is greater, and -1 if timestamp is greater.
+-- returns -1
+select timestamptz_cmp_timestamp('2008-01-01 00:00:00', '2008-01-04');
+SELECT "TIMESTAMPTZ_CMP_TIMESTAMP"('2008-01-01 00:00:00', '2008-01-04')
+!explain-validated-on calcite
+
+# TIMEZONE ('timezone', timestamp | timestamptz ) returns TIMESTAMP or TIMESTAMPTZ
+# Returns a time stamp or time stamp with time zone for the specified time zone
+# and time stamp value.
+select timezone('Africa/Kampala', '2008-01-01 00:00:00');
+SELECT "TIMEZONE"('Africa/Kampala', '2008-01-01 00:00:00')
+!explain-validated-on calcite
+
+# TO_TIMESTAMP ('timestamp', 'format') returns TIMESTAMPTZ
+# Returns a time stamp with time zone for the specified time stamp and time zone
+# format.
+select to_timestamp('05 Dec 2000', 'DD Mon YYYY');
+SELECT "TO_TIMESTAMP"('05 Dec 2000', 'DD Mon YYYY')
+!explain-validated-on calcite
+
+# TRUNC(timestamp) returns DATE
+# Truncates a time stamp and returns a date.
+-- returns '2011-07-21'
+select trunc(timestamp '2011-07-21 10:32:38.248109');
+SELECT "TRUNC"(TIMESTAMP '2011-07-21 10:32:38.248109')
+!explain-validated-on calcite
+
+# 8 Math functions
+
+# ABS
+select abs(2);
+EXPR$0
+2
+!ok
+
+select -abs(-pi);
+EXPR$0
+-3.141592653589793
+!ok
+
+# ACOS
+select acos(0);
+EXPR$0
+1.5707963267948966
+!ok
+
+# ASIN
+select asin(0);
+EXPR$0
+0.0
+!ok
+
+# ATAN
+select atan(0);
+EXPR$0
+0.0
+!ok
+
+# ATAN2
+select atan2(2,2) * 4 as pi;
+PI
+3.141592653589793
+!ok
+
+# CBRT
+select cbrt(-8);
+EXPR$0
+-2.0
+!ok
+
+# CEILING (or CEIL)
+select ceiling(10.5);
+EXPR$0
+11
+!ok
+
+select ceiling(-10.5);
+EXPR$0
+-10
+!ok
+
+select ceil(pi);
+EXPR$0
+4.0
+!ok
+
+# CHECKSUM
+select checksum(comm) from emp;
+SELECT "CHECKSUM"("EMP"."COMM")
+FROM "scott"."EMP" AS "EMP"
+!explain-validated-on calcite
+
+# COS
+select cos(180);
+EXPR$0
+-0.5984600690578581
+!ok
+
+# COT
+select cot(45);
+EXPR$0
+0.6173696237835551
+!ok
+
+# DEGREES
+select degrees(pi);
+EXPR$0
+180.0
+!ok
+
+# DEXP
+select dexp(0);
+SELECT "DEXP"(0)
+!explain-validated-on calcite
+
+# DLOG1 is a synonym for LN
+select dlog1(10);
+SELECT "DLOG1"(10)
+!explain-validated-on calcite
+
+# DLOG10
+select dlog10(100);
+SELECT "DLOG10"(100)
+!explain-validated-on calcite
+
+# EXP
+select exp(0);
+EXPR$0
+1.0
+!ok
+
+# FLOOR
+select floor(10.5);
+EXPR$0
+10
+!ok
+
+select floor(-10.5);
+EXPR$0
+-11
+!ok
+
+# LN
+select ln(1);
+EXPR$0
+0.0
+!ok
+
+# LOG
+select log(1000);
+SELECT "LOG"(1000)
+!explain-validated-on calcite
+
+# MOD
+select mod(-50, 17);
+EXPR$0
+-16
+!ok
+
+# PI
+# In standard SQL you write 'pi', but for redshift you write 'pi()'
+!if (emptyParens) {
+select pi();
+!ok
+!}
+
+# POWER
+select power(2, 3);
+EXPR$0
+8.0
+!ok
+
+select pow(-2, 3);
+SELECT "POW"(-2, 3)
+!explain-validated-on calcite
+
+# RADIANS
+select radians(180);
+EXPR$0
+3.141592653589793
+!ok
+
+# RANDOM
+select random();
+SELECT "RANDOM"()
+!explain-validated-on calcite
+
+# ROUND
+select round(pi);
+EXPR$0
+3.0
+!ok
+
+select round(pi, 2);
+EXPR$0
+3.14
+!ok
+
+select round(-pi, 2);
+EXPR$0
+-3.14
+!ok
+
+# SIN
+select sin(-90);
+EXPR$0
+-0.8939966636005579
+!ok
+
+# SINH
+select sinh(1);
+EXPR$0
+1.1752011936438014
+!ok
+
+# SIGN
+select sign(23);
+EXPR$0
+1
+!ok
+
+# SQRT
+select sqrt(225);
+EXPR$0
+15.0
+!ok
+
+# TAN
+select tan(45);
+EXPR$0
+1.6197751905438615
+!ok
+
+# TO_HEX
+select to_hex(2147676847);
+SELECT "TO_HEX"(2147676847)
+!explain-validated-on calcite
+
+# TRUNC
+select trunc(111.86);
+SELECT "TRUNC"(111.86)
+!explain-validated-on calcite
+
+select trunc(-111.86, 1);
+SELECT "TRUNC"(-111.86, 1)
+!explain-validated-on calcite
+
+# 9 String functions
+
+# || (Concatenation) Operator
+select 'a' || 'b';
+EXPR$0
+ab
+!ok
+
+# BPCHARCMP
+select ename, dname, bpcharcmp(ename, dname) from emp join dept using (deptno);
+SELECT "EMP"."ENAME", "DEPT"."DNAME", "BPCHARCMP"("EMP"."ENAME", "DEPT"."DNAME")
+FROM "scott"."EMP" AS "EMP"
+ INNER JOIN "scott"."DEPT" AS "DEPT" USING ("DEPTNO")
+!explain-validated-on calcite
+
+# BTRIM
+select btrim(' abc ');
+SELECT "BTRIM"(' abc ')
+!explain-validated-on calcite
+
+select btrim('xyzaxyzbxyzcxyz', 'xyz');
+SELECT "BTRIM"('xyzaxyzbxyzcxyz', 'xyz')
+!explain-validated-on calcite
+
+# BTTEXT_PATTERN_CMP is a synonym for BPCHARCMP
+select ename, dname, bttext_pattern_cmp(ename, dname) from emp join dept using (deptno);
+SELECT "EMP"."ENAME", "DEPT"."DNAME", "BTTEXT_PATTERN_CMP"("EMP"."ENAME", "DEPT"."DNAME")
+FROM "scott"."EMP" AS "EMP"
+ INNER JOIN "scott"."DEPT" AS "DEPT" USING ("DEPTNO")
+!explain-validated-on calcite
+
+# CHAR_LENGTH is a synonym for LEN
+select char_length('abc');
+EXPR$0
+3
+!ok
+
+# CHARACTER_LENGTH is a synonym for LEN
+select character_length('abc');
+EXPR$0
+3
+!ok
+
+# CHARINDEX
+select charindex('dog', 'fish');
+SELECT "CHARINDEX"('dog', 'fish')
+!explain-validated-on calcite
+
+select charindex('fish', 'dogfish');
+SELECT "CHARINDEX"('fish', 'dogfish')
+!explain-validated-on calcite
+
+# CHR
+select chr(65);
+EXPR$0
+A
+!ok
+
+# CONCAT (Oracle Compatibility Function)
+select concat('a', 'b');
+SELECT "CONCAT"('a', 'b')
+!explain-validated-on calcite
+
+# CRC32
+-- returns 'f2726906'
+select crc32('Amazon Redshift');
+SELECT "CRC32"('Amazon Redshift')
+!explain-validated-on calcite
+
+# FUNC_SHA1
+select func_sha1('Amazon Redshift');
+SELECT "FUNC_SHA1"('Amazon Redshift')
+!explain-validated-on calcite
+
+# INITCAP
+-- returns 'Nibh.Enim@Egestas.Ca'
+select initcap('nibh.enim@egestas.ca');
+EXPR$0
+Nibh.Enim@Egestas.Ca
+!ok
+
+# LEFT and RIGHT
+-- returns 'Chica'
+select left('Chicago', 5);
+EXPR$0
+Chica
+!ok
+
+-- returns 'icago'
+select right('Chicago', 5);
+EXPR$0
+icago
+!ok
+
+# LEN is a synonym for LENGTH
+select len('gth');
+SELECT "LEN"('gth')
+!explain-validated-on calcite
+
+# LENGTH
+select length('ily');
+SELECT "LENGTH"('ily')
+!explain-validated-on calcite
+
+-- returns 8 (cf OCTET_LENGTH)
+select length('français');
+SELECT "LENGTH"(u&'fran\00e7ais')
+!explain-validated-on calcite
+
+# LOWER
+select lower('Manhattan');
+EXPR$0
+manhattan
+!ok
+
+# LPAD and RPAD
+select lpad('cat', 7);
+SELECT "LPAD"('cat', 7)
+!explain-validated-on calcite
+
+-- returns 'eeriness'
+select rpad(lpad('rine', 6, 'e'), 8, 's');
+SELECT "RPAD"("LPAD"('rine', 6, 'e'), 8, 's')
+!explain-validated-on calcite
+
+select rpad('cat', 7);
+SELECT "RPAD"('cat', 7)
+!explain-validated-on calcite
+
+# LTRIM
+-- returns 'kery'
+select ltrim('bakery', 'abc');
+SELECT "LTRIM"('bakery', 'abc')
+!explain-validated-on calcite
+
+# MD5
+-- returns 'f7415e33f972c03abd4f3fed36748f7a'
+select md5('Amazon Redshift');
+EXPR$0
+f7415e33f972c03abd4f3fed36748f7a
+!ok
+
+# OCTET_LENGTH
+-- returns 9 (cf LENGTH)
+select octet_length('français');
+SELECT OCTET_LENGTH(CAST(u&'fran\00e7ais' AS VARBINARY))
+!explain-validated-on calcite
+
+# POSITION is a synonym for STRPOS
+!if (position) {
+select position('fish', 'dogfish');
+!ok
+!}
+
+# QUOTE_IDENT
+-- returns '"ab cd"'
+select quote_ident('ab cd');
+SELECT "QUOTE_IDENT"('ab cd')
+!explain-validated-on calcite
+
+-- returns 'EMP'
+select quote_ident('EMP');
+SELECT "QUOTE_IDENT"('EMP')
+!explain-validated-on calcite
+
+# QUOTE_LITERAL
+-- returns "'it''s a literal'"
+select quote_literal('it''s a literal');
+SELECT "QUOTE_LITERAL"('it''s a literal')
+!explain-validated-on calcite
+
+# REGEXP_COUNT
+-- returns 8
+select regexp_count('abcdefghijklmnopqrstuvwxyz', '[a-z]{3}');
+SELECT "REGEXP_COUNT"('abcdefghijklmnopqrstuvwxyz', '[a-z]{3}')
+!explain-validated-on calcite
+
+# REGEXP_INSTR ( source_string, pattern [, position [, occurrence] [, option
+# [, parameters ] ] ] ] )
+select regexp_instr('The Home Depot Center', '[cC]ent(er|re)$');
+SELECT "REGEXP_INSTR"('The Home Depot Center', '[cC]ent(er|re)$')
+!explain-validated-on calcite
+
+# REGEXP_REPLACE ( source_string, pattern [, replace_string [ , position ] ] )
+select regexp_replace('DonecFri@semperpretiumneque.com', '@.*\\.(org|gov|com)$');
+SELECT "REGEXP_REPLACE"('DonecFri@semperpretiumneque.com', '@.*\\.(org|gov|com)$')
+!explain-validated-on calcite
+
+# REGEXP_SUBSTR ( source_string, pattern [, position [, occurrence
+# [, parameters ] ] ] )
+select regexp_substr('Suspendisse.tristique@nonnisiAenean.edu','@[^.]*');
+SELECT "REGEXP_SUBSTR"('Suspendisse.tristique@nonnisiAenean.edu', '@[^.]*')
+!explain-validated-on calcite
+
+# REPEAT
+select repeat('ba', 3);
+EXPR$0
+bababa
+!ok
+
+# REPLACE
+select replace('catching catfish', 'cat', 'dog');
+EXPR$0
+dogching dogfish
+!ok
+
+# REPLICATE is a synonym for REPEAT
+select replicate('ba', 3);
+SELECT "REPLICATE"('ba', 3)
+!explain-validated-on calcite
+
+# REVERSE
+select reverse('ab c');
+SELECT "REVERSE"('ab c')
+!explain-validated-on calcite
+
+# RTRIM
+-- returns 'baker'
+select rtrim('bakery', 'xyz');
+SELECT "RTRIM"('bakery', 'xyz')
+!explain-validated-on calcite
+
+# SPLIT_PART
+-- returns '03'
+select split_part('2008-03-05', '-', 2);
+SELECT "SPLIT_PART"('2008-03-05', '-', 2)
+!explain-validated-on calcite
+
+# STRPOS is a synonym for CHARINDEX and POSITION
+select strpos('fish', 'dogfish');
+SELECT "STRPOS"('fish', 'dogfish')
+!explain-validated-on calcite
+
+# STRTOL
+-- returns 2882343476
+select strtol('abcd1234',16);
+SELECT "STRTOL"('abcd1234', 16)
+!explain-validated-on calcite
+
+-- returns 53
+select strtol('110101', 2);
+SELECT "STRTOL"('110101', 2)
+!explain-validated-on calcite
+
+# SUBSTRING
+-- returns 'pill'
+select substring('caterpillar',6,4);
+EXPR$0
+pill
+!ok
+
+-- returns 'pillar'
+select substring('caterpillar',6,8);
+EXPR$0
+pillar
+!ok
+
+-- returns 'pill'
+select substring('caterpillar' from 6 for 4);
+EXPR$0
+pill
+!ok
+
+# TEXTLEN is a synonym for LEN
+select textlen('abc');
+SELECT "TEXTLEN"('abc')
+!explain-validated-on calcite
+
+# TRANSLATE ( expression, characters_to_replace, characters_to_substitute )
+-- returns 'most tin'
+select translate('mint tea', 'inea', 'osin');
+EXPR$0
+most tin
+!ok
+
+# TRIM( [ BOTH ] ['characters' FROM ] string ] )
+-- returns 'dog'
+select trim('"' FROM '"dog"');
+EXPR$0
+dog
+!ok
+
+# UPPER
+select upper('Pop');
+EXPR$0
+POP
+!ok
+
+# 10 JSON Functions
+
+# IS_VALID_JSON
+select is_valid_json('{"a":2}');
+SELECT "IS_VALID_JSON"('{"a":2}')
+!explain-validated-on calcite
+
+# IS_VALID_JSON_ARRAY
+-- returns true
+select is_valid_json_array('[]');
+SELECT "IS_VALID_JSON_ARRAY"('[]')
+!explain-validated-on calcite
+
+-- returns false
+select is_valid_json_array('{}');
+SELECT "IS_VALID_JSON_ARRAY"('{}')
+!explain-validated-on calcite
+
+# JSON_ARRAY_LENGTH('json_array' [, null_if_invalid ] )
+-- returns 3
+select json_array_length('[2,3,[4,5]]');
+SELECT "JSON_ARRAY_LENGTH"('[2,3,[4,5]]')
+!explain-validated-on calcite
+
+-- returns null
+select json_array_length('[2,3', true);
+SELECT "JSON_ARRAY_LENGTH"('[2,3', TRUE)
+!explain-validated-on calcite
+
+# JSON_EXTRACT_ARRAY_ELEMENT_TEXT('json string', pos [, null_if_invalid ] )
+-- returns '113'
+select json_extract_array_element_text('[111,112,113]', 2);
+SELECT "JSON_EXTRACT_ARRAY_ELEMENT_TEXT"('[111,112,113]', 2)
+!explain-validated-on calcite
+
+# JSON_EXTRACT_PATH_TEXT('json_string', 'path_elem' [,'path_elem'[, ...] ] [, null_if_invalid ] )
+-- returns 'star'
+select json_extract_path_text('{"f2":{"f3":1},"f4":{"f5":99,"f6":"star"}}','f4', 'f6');
+SELECT "JSON_EXTRACT_PATH_TEXT"('{"f2":{"f3":1},"f4":{"f5":99,"f6":"star"}}', 'f4', 'f6')
+!explain-validated-on calcite
+
+# 11 Data Type Formatting Functions
+
+# CAST and CONVERT
+select cast(stddev_samp(sal) as dec(14, 2)) from emp;
+EXPR$0
+1182.503223516271873450023122131824493408203125
+!ok
+
+select 123.456::decimal(8,4);
+EXPR$0
+123.456
+!ok
+
+!if (position) {
+select convert(integer, comm) from emp;
+!ok
+!}
+
+select cast(comm as integer) from emp where deptno = 30 order by empno;
+EXPR$0
+300
+500
+1400
+null
+0
+null
+!ok
+
+select comm::integer from emp where deptno = 30 order by empno;
+EXPR$0
+300
+500
+1400
+null
+0
+null
+!ok
+
+# TO_CHAR
+-- returns '23:15:59'
+select to_char(timestamp '2009-12-31 23:15:59','HH24:MI:SS');
+SELECT "TO_CHAR"(TIMESTAMP '2009-12-31 23:15:59', 'HH24:MI:SS')
+!explain-validated-on calcite
+
+-- returns '125.80-'
+select to_char(-125.8, '999D99S');
+SELECT "TO_CHAR"(-125.8, '999D99S')
+!explain-validated-on calcite
+
+# TO_DATE
+-- returns '2001-10-02'
+select to_date ('02 Oct 2001', 'DD Mon YYYY');
+SELECT "TO_DATE"('02 Oct 2001', 'DD Mon YYYY')
+!explain-validated-on calcite
+
+# TO_NUMBER
+-- returns -12454.8
+select to_number('12,454.8-', '99G999D9S');
+SELECT "TO_NUMBER"('12,454.8-', '99G999D9S')
+!explain-validated-on calcite
+
+# 12 System Administration Functions
+
+# CHANGE_QUERY_PRIORITY(query_id, priority)
+select change_query_priority(1076, 'Critical');
+SELECT "CHANGE_QUERY_PRIORITY"(1076, 'Critical')
+!explain-validated-on calcite
+
+# CHANGE_SESSION_PRIORITY(pid, priority)
+select change_session_priority(30311, 'Lowest');
+SELECT "CHANGE_SESSION_PRIORITY"(30311, 'Lowest')
+!explain-validated-on calcite
+
+# CHANGE_USER_PRIORITY(user_name, priority)
+-- returns 'Succeeded to change user priority. Changed user (analysis_user) priority to lowest.'
+select change_user_priority('analysis_user', 'lowest');
+SELECT "CHANGE_USER_PRIORITY"('analysis_user', 'lowest')
+!explain-validated-on calcite
+
+# CURRENT_SETTING('parameter')
+-- returns 'unset'
+select current_setting('query_group');
+SELECT "CURRENT_SETTING"('query_group')
+!explain-validated-on calcite
+
+# PG_CANCEL_BACKEND(pid)
+select pg_cancel_backend(802);
+SELECT "PG_CANCEL_BACKEND"(802)
+!explain-validated-on calcite
+
+# PG_TERMINATE_BACKEND(pid)
+select pg_terminate_backend(8585);
+SELECT "PG_TERMINATE_BACKEND"(8585)
+!explain-validated-on calcite
+
+# SET_CONFIG('parameter', 'new_value' , is_local)
+-- returns 'test'
+select set_config('query_group', 'test', true);
+SELECT "SET_CONFIG"('query_group', 'test', TRUE)
+!explain-validated-on calcite
+
+# 13 System Information Functions
+
+# CURRENT_DATABASE
+!if (emptyParens) {
+select current_database();
+!ok
+!}
+
+# CURRENT_SCHEMA
+!if (emptyParens) {
+select current_schema();
+!ok
+!}
+
+# CURRENT_SCHEMAS(include_implicit)
+select current_schemas(false);
+SELECT "CURRENT_SCHEMAS"(FALSE)
+!explain-validated-on calcite
+
+# CURRENT_USER
+select current_user;
+CURRENT_USER
+sa
+!ok
+
+# CURRENT_USER_ID
+!if (sysdate) {
+select current_user_id;
+!ok
+!}
+
+# HAS_DATABASE_PRIVILEGE( [ user, ] database, privilege)
+select has_database_privilege('guest', 'tickit', 'temp');
+SELECT "HAS_DATABASE_PRIVILEGE"('guest', 'tickit', 'temp')
+!explain-validated-on calcite
+
+# HAS_SCHEMA_PRIVILEGE( [ user, ] schema, privilege)
+select has_schema_privilege('guest', 'public', 'create');
+SELECT "HAS_SCHEMA_PRIVILEGE"('guest', 'public', 'create')
+!explain-validated-on calcite
+
+# HAS_TABLE_PRIVILEGE( [ user, ] table, privilege)
+select has_table_privilege('guest', 'listing', 'select');
+SELECT "HAS_TABLE_PRIVILEGE"('guest', 'listing', 'select')
+!explain-validated-on calcite
+
+# PG_BACKEND_PID
+select pg_backend_pid();
+SELECT "PG_BACKEND_PID"()
+!explain-validated-on calcite
+
+# PG_GET_COLS
+select pg_get_cols('sales_vw');
+SELECT "PG_GET_COLS"('sales_vw')
+!explain-validated-on calcite
+
+# PG_GET_LATE_BINDING_VIEW_COLS
+select pg_get_late_binding_view_cols();
+SELECT "PG_GET_LATE_BINDING_VIEW_COLS"()
+!explain-validated-on calcite
+
+# PG_LAST_COPY_COUNT
+select pg_last_copy_count();
+SELECT "PG_LAST_COPY_COUNT"()
+!explain-validated-on calcite
+
+# PG_LAST_COPY_ID
+select pg_last_copy_id();
+SELECT "PG_LAST_COPY_ID"()
+!explain-validated-on calcite
+
+# PG_LAST_UNLOAD_ID
+select pg_last_unload_id();
+SELECT "PG_LAST_UNLOAD_ID"()
+!explain-validated-on calcite
+
+# PG_LAST_QUERY_ID
+select pg_last_query_id();
+SELECT "PG_LAST_QUERY_ID"()
+!explain-validated-on calcite
+
+# PG_LAST_UNLOAD_COUNT
+select pg_last_unload_count();
+SELECT "PG_LAST_UNLOAD_COUNT"()
+!explain-validated-on calcite
+
+# SESSION_USER
+select session_user;
+SESSION_USER
+sa
+!ok
+
+# SLICE_NUM
+# Returns an integer corresponding to the slice number in the cluster where the
+# data for a row is located.
+select slice_num();
+SELECT "SLICE_NUM"()
+!explain-validated-on calcite
+
+# USER
+# Synonym for CURRENT_USER
+select user;
+USER
+sa
+!ok
+
+# VERSION
+# Returns details about the currently installed release,
+# with specific Amazon Redshift version information at the end.
+select version();
+SELECT "VERSION"()
+!explain-validated-on calcite
+
+# End redshift.iq
diff --git a/babel/src/test/resources/sql/select.iq b/babel/src/test/resources/sql/select.iq
new file mode 100755
index 000000000000..07399eb7da1e
--- /dev/null
+++ b/babel/src/test/resources/sql/select.iq
@@ -0,0 +1,63 @@
+# select.iq - Babel test for non-standard clauses in SELECT
+#
+# 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.
+#
+!use scott-babel
+!set outputformat mysql
+
+# ORDER BY column not in SELECT clause
+SELECT ename
+FROM emp, dept
+ORDER BY emp.deptno;
+
+SELECT "EMP"."ENAME"
+FROM "scott"."EMP" AS "EMP",
+ "scott"."DEPT" AS "DEPT"
+ORDER BY "EMP"."DEPTNO"
+!explain-validated-on all
+
+# LEFT SEMI JOIN (Hive only)
+SELECT *
+FROM emp LEFT SEMI JOIN dept ON emp.deptno = dept.deptno;
+
+SELECT "EMP"."EMPNO", "EMP"."ENAME", "EMP"."JOB", "EMP"."MGR", "EMP"."HIREDATE", "EMP"."SAL", "EMP"."COMM", "EMP"."DEPTNO", "DEPT"."DEPTNO" AS "DEPTNO0", "DEPT"."DNAME", "DEPT"."LOC"
+FROM "scott"."EMP" AS "EMP"
+ LEFT SEMI JOIN "scott"."DEPT" AS "DEPT" ON "EMP"."DEPTNO" = "DEPT"."DEPTNO"
+!explain-validated-on hive
+
+# Test CONNECT BY (Oracle only)
+!if (false) {
+SELECT *
+FROM emp
+START WITH mgr IS NULL
+CONNECT BY empno = PRIOR mgr;
+select(...)
+!explain-validated-on oracle
+!}
+
+# WITH RECURSIVE (Oracle, MySQL 8 onwards)
+!if (false) {
+WITH RECURSIVE t(n) AS (
+ VALUES (1)
+ UNION ALL
+ SELECT n+1 FROM t WHERE n < 100
+)
+SELECT sum(n) FROM t;
+select(...)
+!explain-validated-on mysql8+ oracle
+!}
+
+# End select.iq
diff --git a/bom/build.gradle.kts b/bom/build.gradle.kts
new file mode 100644
index 000000000000..7eaac6e53113
--- /dev/null
+++ b/bom/build.gradle.kts
@@ -0,0 +1,165 @@
+/*
+ * 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.
+ */
+plugins {
+ `java-platform`
+}
+
+val String.v: String get() = rootProject.extra["$this.version"] as String
+
+// Note: Gradle allows to declare dependency on "bom" as "api",
+// and it makes the contraints to be transitively visible
+// However Maven can't express that, so the approach is to use Gradle resolution
+// and generate pom files with resolved versions
+// See https://github.com/gradle/gradle/issues/9866
+
+fun DependencyConstraintHandlerScope.apiv(
+ notation: String,
+ versionProp: String = notation.substringAfterLast(':')
+) =
+ "api"(notation + ":" + versionProp.v)
+
+fun DependencyConstraintHandlerScope.runtimev(
+ notation: String,
+ versionProp: String = notation.substringAfterLast(':')
+) =
+ "runtime"(notation + ":" + versionProp.v)
+
+javaPlatform {
+ allowDependencies()
+}
+
+dependencies {
+ api(platform("com.fasterxml.jackson:jackson-bom:${"jackson".v}"))
+
+ // Parenthesis are needed here: https://github.com/gradle/gradle/issues/9248
+ (constraints) {
+ // api means "the dependency is for both compilation and runtime"
+ // runtime means "the dependency is only for runtime, not for compilation"
+ // In other words, marking dependency as "runtime" would avoid accidental
+ // dependency on it during compilation
+ apiv("com.alibaba.database:innodb-java-reader")
+ apiv("com.beust:jcommander")
+ apiv("org.checkerframework:checker-qual", "checkerframework")
+ apiv("com.datastax.oss:java-driver-core", "cassandra-java-driver-core")
+ apiv("com.esri.geometry:esri-geometry-api")
+ apiv("com.fasterxml.jackson.core:jackson-databind")
+ apiv("com.github.kstyrc:embedded-redis")
+ apiv("com.github.stephenc.jcip:jcip-annotations")
+ apiv("com.google.errorprone:error_prone_annotations", "errorprone")
+ apiv("com.google.errorprone:error_prone_type_annotations", "errorprone")
+ apiv("org.apache.kylin:kylin-external-guava30")
+ apiv("com.google.protobuf:protobuf-java", "protobuf")
+ apiv("com.google.uzaygezen:uzaygezen-core", "uzaygezen")
+ apiv("com.h2database:h2")
+ apiv("com.jayway.jsonpath:json-path")
+ apiv("com.joestelmach:natty")
+ apiv("com.oracle.ojdbc:ojdbc8")
+ apiv("com.teradata.tpcds:tpcds", "teradata.tpcds")
+ apiv("com.yahoo.datasketches:sketches-core")
+ apiv("commons-codec:commons-codec")
+ apiv("commons-io:commons-io")
+ apiv("de.bwaldvogel:mongo-java-server", "mongo-java-server")
+ apiv("de.bwaldvogel:mongo-java-server-core", "mongo-java-server")
+ apiv("de.bwaldvogel:mongo-java-server-memory-backend", "mongo-java-server")
+ apiv("io.prestosql.tpch:tpch")
+ apiv("javax.servlet:javax.servlet-api", "servlet")
+ apiv("joda-time:joda-time")
+ apiv("junit:junit", "junit4")
+ apiv("mysql:mysql-connector-java")
+ apiv("net.hydromatic:aggdesigner-algorithm")
+ apiv("net.hydromatic:chinook-data-hsqldb")
+ apiv("net.hydromatic:foodmart-data-hsqldb")
+ apiv("net.hydromatic:foodmart-data-json")
+ apiv("net.hydromatic:foodmart-queries")
+ apiv("net.hydromatic:quidem")
+ apiv("net.hydromatic:scott-data-hsqldb")
+ apiv("net.hydromatic:tpcds", "hydromatic.tpcds")
+ apiv("net.java.dev.jna:jna")
+ apiv("net.sf.opencsv:opencsv")
+ apiv("org.apache.calcite.avatica:avatica-core", "calcite.avatica")
+ apiv("org.apache.calcite.avatica:avatica-server", "calcite.avatica")
+ apiv("org.apache.cassandra:cassandra-all")
+ apiv("org.apache.commons:commons-dbcp2")
+ apiv("org.apache.commons:commons-lang3")
+ apiv("org.apache.commons:commons-pool2")
+ apiv("org.apache.geode:geode-core")
+ apiv("org.apache.hadoop:hadoop-client", "hadoop")
+ apiv("org.apache.hadoop:hadoop-common", "hadoop")
+ apiv("org.apache.httpcomponents:httpclient")
+ apiv("org.apache.httpcomponents:httpcore")
+ apiv("org.apache.kafka:kafka-clients")
+ apiv("org.apache.kerby:kerb-client", "kerby")
+ apiv("org.apache.kerby:kerb-core", "kerby")
+ apiv("org.apache.kerby:kerb-simplekdc", "kerby")
+ apiv("org.apache.logging.log4j:log4j-api", "log4j2")
+ apiv("org.apache.logging.log4j:log4j-core", "log4j2")
+ apiv("org.apache.logging.log4j:log4j-slf4j-impl", "log4j2")
+ apiv("org.apache.pig:pig")
+ apiv("org.apache.pig:pigunit", "pig")
+ apiv("org.apache.spark:spark-core_2.10", "spark")
+ apiv("org.apiguardian:apiguardian-api")
+ apiv("org.bouncycastle:bcpkix-jdk15on", "bouncycastle")
+ apiv("org.bouncycastle:bcprov-jdk15on", "bouncycastle")
+ apiv("net.bytebuddy:byte-buddy")
+ apiv("org.cassandraunit:cassandra-unit")
+ apiv("org.codehaus.janino:commons-compiler", "janino")
+ apiv("org.codehaus.janino:janino")
+ apiv("org.codelibs.elasticsearch.module:lang-painless", "elasticsearch")
+ apiv("org.codelibs.elasticsearch.module:scripting-painless-spi", "elasticsearch")
+ apiv("org.eclipse.jetty:jetty-http", "jetty")
+ apiv("org.eclipse.jetty:jetty-security", "jetty")
+ apiv("org.eclipse.jetty:jetty-server", "jetty")
+ apiv("org.eclipse.jetty:jetty-util", "jetty")
+ apiv("org.elasticsearch.client:elasticsearch-rest-client", "elasticsearch")
+ apiv("org.elasticsearch.plugin:transport-netty4-client", "elasticsearch")
+ apiv("org.elasticsearch:elasticsearch")
+ apiv("org.immutables:value-annotations", "immutables")
+ apiv("org.immutables:value", "immutables")
+ apiv("org.exparity:hamcrest-date")
+ apiv("org.hamcrest:hamcrest")
+ apiv("org.hamcrest:hamcrest-core", "hamcrest")
+ apiv("org.hamcrest:hamcrest-library", "hamcrest")
+ apiv("org.hsqldb:hsqldb")
+ apiv("org.incava:java-diff")
+ apiv("org.jboss:jandex")
+ apiv("org.jsoup:jsoup")
+ apiv("org.junit:junit-bom", "junit5")
+ apiv("org.mockito:mockito-core", "mockito")
+ apiv("org.mongodb:mongo-java-driver")
+ apiv("org.ow2.asm:asm")
+ apiv("org.ow2.asm:asm-all", "asm")
+ apiv("org.ow2.asm:asm-analysis", "asm")
+ apiv("org.ow2.asm:asm-commons", "asm")
+ apiv("org.ow2.asm:asm-tree", "asm")
+ apiv("org.ow2.asm:asm-util", "asm")
+ apiv("org.postgresql:postgresql")
+ apiv("org.scala-lang:scala-library")
+ apiv("org.slf4j:slf4j-api", "slf4j")
+ // TODO: https://issues.apache.org/jira/browse/CALCITE-4862
+ // Eventually we should get rid of slf4j-log4j12 dependency but currently it is not possible
+ // since certain modules (Pig, Piglet) have dependencies using directly Log4j 1.x APIs
+ runtimev("org.slf4j:slf4j-log4j12", "slf4j")
+ apiv("org.testcontainers:testcontainers")
+ apiv("redis.clients:jedis")
+ apiv("sqlline:sqlline")
+ runtimev("org.openjdk.jmh:jmh-core", "jmh")
+ apiv("org.openjdk.jmh:jmh-generator-annprocess", "jmh")
+ runtimev("xalan:xalan")
+ runtimev("xerces:xercesImpl")
+ apiv("com.google.code.findbugs:jsr305")
+ }
+}
diff --git a/build.gradle.kts b/build.gradle.kts
new file mode 100644
index 000000000000..dad20b7e7b11
--- /dev/null
+++ b/build.gradle.kts
@@ -0,0 +1,934 @@
+/*
+ * 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.
+ */
+import com.github.spotbugs.SpotBugsTask
+import com.github.vlsi.gradle.crlf.CrLfSpec
+import com.github.vlsi.gradle.crlf.LineEndings
+import com.github.vlsi.gradle.dsl.configureEach
+import com.github.vlsi.gradle.git.FindGitAttributes
+import com.github.vlsi.gradle.git.dsl.gitignore
+import com.github.vlsi.gradle.properties.dsl.lastEditYear
+import com.github.vlsi.gradle.properties.dsl.props
+import com.github.vlsi.gradle.release.RepositoryType
+import de.thetaphi.forbiddenapis.gradle.CheckForbiddenApis
+import de.thetaphi.forbiddenapis.gradle.CheckForbiddenApisExtension
+import net.ltgt.gradle.errorprone.errorprone
+import org.apache.calcite.buildtools.buildext.dsl.ParenthesisBalancer
+import org.gradle.api.tasks.testing.logging.TestExceptionFormat
+
+plugins {
+ // java-base is needed for platform(...) resolution,
+ // see https://github.com/gradle/gradle/issues/14822
+ `java-base`
+ publishing
+ // Verification
+ checkstyle
+ calcite.buildext
+ id("org.checkerframework") apply false
+ id("com.github.autostyle")
+ id("org.nosphere.apache.rat")
+ id("com.github.spotbugs")
+ id("de.thetaphi.forbiddenapis") apply false
+ id("net.ltgt.errorprone") apply false
+ id("com.github.vlsi.jandex") apply false
+ id("org.owasp.dependencycheck")
+ id("com.github.johnrengelman.shadow") apply false
+ // IDE configuration
+ id("org.jetbrains.gradle.plugin.idea-ext")
+ id("com.github.vlsi.ide")
+ // Release
+ id("com.github.vlsi.crlf")
+ id("com.github.vlsi.gradle-extensions")
+ id("com.github.vlsi.license-gather") apply false
+ id("com.github.vlsi.stage-vote-release")
+ id("com.autonomousapps.dependency-analysis") apply false
+}
+
+// define repo url
+val snapshotsRepoUrl = uri("https://repository.kyligence.io/repository/maven-snapshots/")
+val releasesRepoUrl = uri("https://repository.kyligence.io/repository/maven-releases/")
+val asfTestNexusUsername: String by properties
+val asfTestNexusPassword: String by properties
+
+allprojects {
+ repositories {
+ // At least for RAT
+ mavenLocal()
+ // achieve dependencies
+ maven {
+ name = "snapshots"
+ url = snapshotsRepoUrl
+ }
+ maven {
+ name = "releases"
+ url = releasesRepoUrl
+ }
+ mavenCentral()
+ }
+}
+
+fun reportsForHumans() = !(System.getenv()["CI"]?.toBoolean() ?: false)
+
+val lastEditYear by extra(lastEditYear())
+
+// Do not enable spotbugs by default. Execute it only when -Pspotbugs is present
+val enableSpotBugs = props.bool("spotbugs")
+val enableCheckerframework by props()
+val enableErrorprone by props()
+val enableDependencyAnalysis by props()
+val skipJandex by props()
+val skipCheckstyle by props()
+val skipAutostyle by props()
+val skipJavadoc by props()
+val enableMavenLocal by props()
+val enableGradleMetadata by props()
+val werror by props(true) // treat javac warnings as errors
+// Inherited from stage-vote-release-plugin: skipSign, useGpgCmd
+// Inherited from gradle-extensions-plugin: slowSuiteLogThreshold=0L, slowTestLogThreshold=2000L
+
+// Java versions prior to 1.8.0u202 have known issues that cause invalid bytecode in certain patterns
+// of annotation usage.
+// So we require at least 1.8.0u202
+System.getProperty("java.version").let { version ->
+ version.takeIf { it.startsWith("1.8.0_") }
+ ?.removePrefix("1.8.0_")
+ ?.toIntOrNull()
+ ?.let {
+ require(it >= 202) {
+ "Apache Calcite requires Java 1.8.0u202 or later. The current Java version is $version"
+ }
+ }
+}
+
+ide {
+ copyrightToAsf()
+ ideaInstructionsUri =
+ uri("https://calcite.apache.org/docs/howto.html#setting-up-intellij-idea")
+ doNotDetectFrameworks("android", "jruby")
+}
+
+// This task scans the project for gitignore / gitattributes, and that is reused for building
+// source/binary artifacts with the appropriate eol/executable file flags
+// It enables to automatically exclude patterns from .gitignore
+val gitProps by tasks.registering(FindGitAttributes::class) {
+ // Scanning for .gitignore and .gitattributes files in a task avoids doing that
+ // when distribution build is not required (e.g. code is just compiled)
+ root.set(rootDir)
+}
+
+val rat by tasks.getting(org.nosphere.apache.rat.RatTask::class) {
+ gitignore(gitProps)
+ verbose.set(true)
+ // Note: patterns are in non-standard syntax for RAT, so we use exclude(..) instead of excludeFile
+ exclude(rootDir.resolve(".ratignore").readLines())
+}
+
+tasks.validateBeforeBuildingReleaseArtifacts {
+ dependsOn(rat)
+}
+
+val String.v: String get() = rootProject.extra["$this.version"] as String
+
+val buildVersion = "calcite".v + releaseParams.snapshotSuffix
+
+println("Building Apache Calcite $buildVersion")
+
+releaseArtifacts {
+ fromProject(":release")
+}
+
+// Configures URLs to SVN and Nexus
+releaseParams {
+ tlp.set("Calcite")
+ componentName.set("Apache Calcite")
+ releaseTag.set("calcite-$buildVersion")
+ rcTag.set(rc.map { "calcite-$buildVersion-rc$it" })
+ sitePreviewEnabled.set(false)
+ nexus {
+ // https://github.com/marcphilipp/nexus-publish-plugin/issues/35
+ packageGroup.set("org.apache.calcite")
+ if (repositoryType.get() == RepositoryType.PROD) {
+ // org.apache.calcite at repository.apache.org
+ stagingProfileId.set("778fd0d4358bb")
+ }
+ }
+ svnDist {
+ staleRemovalFilters {
+ includes.add(Regex(".*apache-calcite-\\d.*"))
+ validates.empty()
+ validates.add(provider {
+ Regex("release/calcite/apache-calcite-${version.toString().removeSuffix("-SNAPSHOT")}")
+ })
+ }
+ }
+}
+
+val javadocAggregate by tasks.registering(Javadoc::class) {
+ group = JavaBasePlugin.DOCUMENTATION_GROUP
+ description = "Generates aggregate javadoc for all the artifacts"
+
+ val sourceSets = subprojects
+ .mapNotNull { it.extensions.findByType() }
+ .map { it.named("main") }
+
+ classpath = files(sourceSets.map { set -> set.map { it.output + it.compileClasspath } })
+ setSource(sourceSets.map { set -> set.map { it.allJava } })
+ setDestinationDir(file("$buildDir/docs/javadocAggregate"))
+}
+
+/** Similar to {@link #javadocAggregate} but includes tests.
+ * CI uses this target to validate javadoc (e.g. checking for broken links). */
+val javadocAggregateIncludingTests by tasks.registering(Javadoc::class) {
+ description = "Generates aggregate javadoc for all the artifacts"
+
+ val sourceSets = subprojects
+ .mapNotNull { it.extensions.findByType() }
+ .flatMap { listOf(it.named("main"), it.named("test")) }
+
+ classpath = files(sourceSets.map { set -> set.map { it.output + it.compileClasspath } })
+ setSource(sourceSets.map { set -> set.map { it.allJava } })
+ setDestinationDir(file("$buildDir/docs/javadocAggregateIncludingTests"))
+}
+
+val adaptersForSqlline = listOf(
+ ":babel", ":cassandra", ":druid", ":elasticsearch",
+ ":file", ":geode", ":innodb", ":kafka", ":mongodb",
+ ":pig", ":piglet", ":plus", ":redis", ":spark", ":splunk")
+
+val dataSetsForSqlline = listOf(
+ "net.hydromatic:foodmart-data-hsqldb",
+ "net.hydromatic:scott-data-hsqldb",
+ "net.hydromatic:chinook-data-hsqldb"
+)
+
+val sqllineClasspath by configurations.creating {
+ isCanBeConsumed = false
+ attributes {
+ attribute(Usage.USAGE_ATTRIBUTE, objects.named(Usage.JAVA_RUNTIME))
+ attribute(LibraryElements.LIBRARY_ELEMENTS_ATTRIBUTE, objects.named(LibraryElements.CLASSES_AND_RESOURCES))
+ attribute(TargetJvmVersion.TARGET_JVM_VERSION_ATTRIBUTE, JavaVersion.current().majorVersion.toInt())
+ attribute(Bundling.BUNDLING_ATTRIBUTE, objects.named(Bundling.EXTERNAL))
+ }
+}
+
+dependencies {
+ sqllineClasspath(platform(project(":bom")))
+ sqllineClasspath(project(":testkit"))
+ sqllineClasspath("sqlline:sqlline")
+ for (p in adaptersForSqlline) {
+ sqllineClasspath(project(p))
+ }
+ for (m in dataSetsForSqlline) {
+ sqllineClasspath(module(m))
+ }
+}
+
+val buildSqllineClasspath by tasks.registering(Jar::class) {
+ description = "Creates classpath-only jar for running SqlLine"
+ // One can debug classpath with ./gradlew dependencies --configuration sqllineClasspath
+ inputs.files(sqllineClasspath).withNormalizer(ClasspathNormalizer::class.java)
+ archiveFileName.set("sqllineClasspath.jar")
+ manifest {
+ attributes(
+ "Main-Class" to "sqlline.SqlLine",
+ "Class-Path" to provider {
+ // Class-Path is a list of URLs
+ sqllineClasspath.joinToString(" ") {
+ it.toURI().toURL().toString()
+ }
+ }
+ )
+ }
+}
+
+if (enableDependencyAnalysis) {
+ apply(plugin = "com.autonomousapps.dependency-analysis")
+ configure {
+ // See https://github.com/autonomousapps/dependency-analysis-android-gradle-plugin
+ // Most of the time the recommendations are good, however, there are cases the suggestsions
+ // are off, so we don't include the dependency analysis to CI workflow yet
+ // ./gradlew -PenableDependencyAnalysis buildHealth --no-parallel --no-daemon
+ issues {
+ all { // all projects
+ onAny {
+ severity("fail")
+ }
+ onRedundantPlugins {
+ severity("ignore")
+ }
+ }
+ }
+ }
+}
+
+val javaccGeneratedPatterns = arrayOf(
+ "org/apache/calcite/jdbc/CalciteDriverVersion.java",
+ "**/parser/**/*ParserImpl.*",
+ "**/parser/**/*ParserImplConstants.*",
+ "**/parser/**/*ParserImplTokenManager.*",
+ "**/parser/**/PigletParser.*",
+ "**/parser/**/PigletParserConstants.*",
+ "**/parser/**/ParseException.*",
+ "**/parser/**/SimpleCharStream.*",
+ "**/parser/**/Token.*",
+ "**/parser/**/TokenMgrError.*",
+ "**/org/apache/calcite/runtime/Resources.java",
+ "**/parser/**/*ParserTokenManager.*"
+)
+
+fun PatternFilterable.excludeJavaCcGenerated() {
+ exclude(*javaccGeneratedPatterns)
+}
+
+fun com.github.autostyle.gradle.BaseFormatExtension.license() {
+ licenseHeader(rootProject.ide.licenseHeader) {
+ copyrightStyle("bat", com.github.autostyle.generic.DefaultCopyrightStyle.PAAMAYIM_NEKUDOTAYIM)
+ copyrightStyle("cmd", com.github.autostyle.generic.DefaultCopyrightStyle.PAAMAYIM_NEKUDOTAYIM)
+ }
+ trimTrailingWhitespace()
+ endWithNewline()
+}
+
+allprojects {
+ group = "org.apache.calcite"
+ version = buildVersion
+
+ apply(plugin = "com.github.vlsi.gradle-extensions")
+
+ repositories {
+ // RAT and Autostyle dependencies
+ mavenCentral()
+ }
+
+ val javaUsed = file("src/main/java").isDirectory
+ if (javaUsed) {
+ apply(plugin = "java-library")
+ }
+
+ plugins.withId("java-library") {
+ dependencies {
+ "annotationProcessor"(platform(project(":bom")))
+ "implementation"(platform(project(":bom")))
+ "testAnnotationProcessor"(platform(project(":bom")))
+ }
+ }
+
+ val hasTests = file("src/test/java").isDirectory || file("src/test/kotlin").isDirectory
+ if (hasTests) {
+ // Add default tests dependencies
+ dependencies {
+ val testImplementation by configurations
+ val testRuntimeOnly by configurations
+ testImplementation(platform("org.junit:junit-bom"))
+ testImplementation("org.junit.jupiter:junit-jupiter")
+ testImplementation("org.hamcrest:hamcrest")
+ if (project.props.bool("junit4", default = false)) {
+ // Allow projects to opt-out of junit dependency, so they can be JUnit5-only
+ testImplementation("junit:junit")
+ testRuntimeOnly("org.junit.vintage:junit-vintage-engine")
+ }
+ }
+ }
+
+ if (!skipAutostyle) {
+ apply(plugin = "com.github.autostyle")
+ autostyle {
+ kotlinGradle {
+ license()
+ ktlint()
+ }
+ format("configs") {
+ filter {
+ include("**/*.sh", "**/*.bsh", "**/*.cmd", "**/*.bat")
+ include("**/*.properties", "**/*.yml")
+ include("**/*.xsd", "**/*.xsl", "**/*.xml")
+ // Autostyle does not support gitignore yet https://github.com/autostyle/autostyle/issues/13
+ exclude("bin/**", "out/**", "target/**", "gradlew*")
+ exclude(rootDir.resolve(".ratignore").readLines())
+ }
+ license()
+ }
+ if (project == rootProject) {
+ // Spotless does not exclude subprojects when using target(...)
+ // So **/*.md is enough to scan all the md files in the codebase
+ // See https://github.com/diffplug/spotless/issues/468
+ format("markdown") {
+ filter.include("**/*.md")
+ // Flot is known to have trailing whitespace, so the files
+ // are kept in their original format (e.g. to simplify diff on library upgrade)
+ endWithNewline()
+ }
+ }
+ }
+ plugins.withId("org.jetbrains.kotlin.jvm") {
+ autostyle {
+ kotlin {
+ licenseHeader(rootProject.ide.licenseHeader)
+ ktlint {
+ userData(mapOf("disabled_rules" to "import-ordering"))
+ }
+ trimTrailingWhitespace()
+ endWithNewline()
+ }
+ }
+ }
+ }
+ if (!skipCheckstyle) {
+ apply()
+ // This will be config_loc in Checkstyle (checker.xml)
+ val configLoc = File(rootDir, "src/main/config/checkstyle")
+ checkstyle {
+ toolVersion = "checkstyle".v
+ isShowViolations = true
+ configDirectory.set(configLoc)
+ configFile = configDirectory.get().file("checker.xml").asFile
+ }
+ tasks.register("checkstyleAll") {
+ dependsOn(tasks.withType())
+ }
+ tasks.configureEach {
+ // Excludes here are faster than in suppressions.xml
+ // Since here we can completely remove file from the analysis.
+ // On the other hand, supporessions.xml still analyzes the file, and
+ // then it recognizes it should suppress all the output.
+ excludeJavaCcGenerated()
+ // Workaround for https://github.com/gradle/gradle/issues/13927
+ // Absolute paths must not be used as they defeat Gradle build cache
+ // Unfortunately, Gradle passes only config_loc variable by default, so we make
+ // all the paths relative to config_loc
+ configProperties!!["cache_file"] =
+ buildDir.resolve("checkstyle/cacheFile").relativeTo(configLoc)
+ }
+ // afterEvaluate is to support late sourceSet addition (e.g. jmh sourceset)
+ afterEvaluate {
+ tasks.configureEach {
+ // Checkstyle 8.26 does not need classpath, see https://github.com/gradle/gradle/issues/14227
+ classpath = files()
+ }
+ }
+ }
+ if (!skipAutostyle || !skipCheckstyle) {
+ tasks.register("style") {
+ group = LifecycleBasePlugin.VERIFICATION_GROUP
+ description = "Formats code (license header, import order, whitespace at end of line, ...) and executes Checkstyle verifications"
+ if (!skipAutostyle) {
+ dependsOn("autostyleApply")
+ }
+ if (!skipCheckstyle) {
+ dependsOn("checkstyleAll")
+ }
+ }
+ }
+
+ tasks.configureEach {
+ // Ensure builds are reproducible
+ isPreserveFileTimestamps = false
+ isReproducibleFileOrder = true
+ dirMode = "775".toInt(8)
+ fileMode = "664".toInt(8)
+ }
+
+ // Deploy to Nexus without Signing
+ plugins.withType {
+ afterEvaluate {
+ configure {
+ // Note it would still try to sign the artifacts,
+ // however it would fail only when signing a RELEASE version fails
+ isRequired = false
+ }
+ }
+ }
+
+ tasks {
+ configureEach {
+ excludeJavaCcGenerated()
+ (options as StandardJavadocDocletOptions).apply {
+ // Please refrain from using non-ASCII chars below since the options are passed as
+ // javadoc.options file which is parsed with "default encoding"
+ noTimestamp.value = true
+ showFromProtected()
+ // javadoc: error - The code being documented uses modules but the packages
+ // defined in https://docs.oracle.com/javase/9/docs/api/ are in the unnamed module
+ source = "1.8"
+ docEncoding = "UTF-8"
+ charSet = "UTF-8"
+ encoding = "UTF-8"
+ docTitle = "Apache Calcite API"
+ windowTitle = "Apache Calcite API"
+ header = "Apache Calcite "
+ bottom =
+ "Copyright © 2012-$lastEditYear Apache Software Foundation. All Rights Reserved."
+ if (JavaVersion.current() >= JavaVersion.VERSION_1_9) {
+ addBooleanOption("html5", true)
+ links("https://docs.oracle.com/javase/9/docs/api/")
+ } else {
+ links("https://docs.oracle.com/javase/8/docs/api/")
+ }
+ }
+ }
+ }
+
+ plugins.withType {
+ configure {
+ sourceCompatibility = JavaVersion.VERSION_1_8
+ targetCompatibility = JavaVersion.VERSION_1_8
+ }
+ configure {
+ consistentResolution {
+ useCompileClasspathVersions()
+ }
+ }
+
+ repositories {
+ if (enableMavenLocal) {
+ mavenLocal()
+ }
+ mavenCentral()
+ }
+ val sourceSets: SourceSetContainer by project
+
+ apply(plugin = "de.thetaphi.forbiddenapis")
+ apply(plugin = "maven-publish")
+
+ if (!skipJandex) {
+ apply(plugin = "com.github.vlsi.jandex")
+
+ project.configure {
+ toolVersion.set("jandex".v)
+ skipIndexFileGeneration()
+ }
+ }
+
+ if (!enableGradleMetadata) {
+ tasks.withType {
+ enabled = false
+ }
+ }
+
+ if (!skipAutostyle) {
+ autostyle {
+ java {
+ filter.exclude(*javaccGeneratedPatterns +
+ "**/test/java/*.java" +
+ "**/RelRule.java" /** remove as part of CALCITE-4831 **/)
+ license()
+ if (!project.props.bool("junit4", default = false)) {
+ replace("junit5: Test", "org.junit.Test", "org.junit.jupiter.api.Test")
+ replaceRegex("junit5: Before", "org.junit.Before\\b", "org.junit.jupiter.api.BeforeEach")
+ replace("junit5: BeforeClass", "org.junit.BeforeClass", "org.junit.jupiter.api.BeforeAll")
+ replaceRegex("junit5: After", "org.junit.After\\b", "org.junit.jupiter.api.AfterEach")
+ replace("junit5: AfterClass", "org.junit.AfterClass", "org.junit.jupiter.api.AfterAll")
+ replace("junit5: Ignore", "org.junit.Ignore", "org.junit.jupiter.api.Disabled")
+ replaceRegex("junit5: @Before", "@Before\\b", "@BeforeEach")
+ replace("junit5: @BeforeClass", "@BeforeClass", "@BeforeAll")
+ replaceRegex("junit5: @After", "@After\\b", "@AfterEach")
+ replace("junit5: @AfterClass", "@AfterClass", "@AfterAll")
+ replace("junit5: @Ignore", "@Ignore", "@Disabled")
+ replace("junit5: Assert.assertThat", "org.junit.Assert.assertThat", "org.hamcrest.MatcherAssert.assertThat")
+ replace("junit5: Assert.fail", "org.junit.Assert.fail", "org.junit.jupiter.api.Assertions.fail")
+ }
+ replaceRegex("side by side comments", "(\n\\s*+[*]*+/\n)(/[/*])", "\$1\n\$2")
+ replaceRegex("jsr305 nullable -> checkerframework", "javax\\.annotation\\.Nullable", "org.checkerframework.checker.nullness.qual.Nullable")
+ replaceRegex("jsr305 nonnull -> checkerframework", "javax\\.annotation\\.Nonnull", "org.checkerframework.checker.nullness.qual.NonNull")
+ importOrder(
+ "org.apache.calcite.",
+ "org.apache.",
+ "au.com.",
+ "com.",
+ "io.",
+ "mondrian.",
+ "net.",
+ "org.",
+ "scala.",
+ "java",
+ "",
+ "static com.",
+ "static org.apache.calcite.",
+ "static org.apache.",
+ "static org.",
+ "static java",
+ "static "
+ )
+ removeUnusedImports()
+ replaceRegex("Avoid 2+ blank lines after package", "^package\\s+([^;]+)\\s*;\\n{3,}", "package \$1;\n\n")
+ replaceRegex("Avoid 2+ blank lines after import", "^import\\s+([^;]+)\\s*;\\n{3,}", "import \$1;\n\n")
+ indentWithSpaces(2)
+ replaceRegex("@Override should not be on its own line", "(@Override)\\s{2,}", "\$1 ")
+ replaceRegex("@Test should not be on its own line", "(@Test)\\s{2,}", "\$1 ")
+ replaceRegex("Newline in string should be at end of line", """\\n" *\+""", "\\\\n\"\n +")
+ replaceRegex("require message for requireNonNull", """(? should not be placed a the end of the line", "(?-m)\\s*+ *+\n \\* ", "\n *\n *
")
+ // Assume developer copy-pasted the link, and updated text only, so the url is old, and we replace it with the proper one
+ replaceRegex(">[CALCITE-...] link styles: 1", "])++CALCITE-\\d+[^>]++>\\s*+\\[?(CALCITE-\\d+)\\]?", " [\$1]")
+ // If the link was crafted manually, ensure it has [CALCITE-...] in the link text
+ replaceRegex(">[CALCITE-...] link styles: 2", " ])++(CALCITE-\\d+)[^>]++>\\s*+\\[?CALCITE-\\d+\\]?", " [\$1]")
+ custom("((() preventer", 1) { contents: String ->
+ ParenthesisBalancer.apply(contents)
+ }
+ }
+ }
+ }
+ if (enableSpotBugs) {
+ apply(plugin = "com.github.spotbugs")
+ spotbugs {
+ toolVersion = "spotbugs".v
+ reportLevel = "high"
+ // excludeFilter = file("$rootDir/src/main/config/spotbugs/spotbugs-filter.xml")
+ // By default spotbugs verifies TEST classes as well, and we do not want that
+ this.sourceSets = listOf(sourceSets["main"])
+ }
+ dependencies {
+ // Parenthesis are needed here: https://github.com/gradle/gradle/issues/9248
+ (constraints) {
+ "spotbugs"("org.ow2.asm:asm:${"asm".v}")
+ "spotbugs"("org.ow2.asm:asm-all:${"asm".v}")
+ "spotbugs"("org.ow2.asm:asm-analysis:${"asm".v}")
+ "spotbugs"("org.ow2.asm:asm-commons:${"asm".v}")
+ "spotbugs"("org.ow2.asm:asm-tree:${"asm".v}")
+ "spotbugs"("org.ow2.asm:asm-util:${"asm".v}")
+ }
+ }
+ }
+
+ configure {
+ failOnUnsupportedJava = false
+ ignoreSignaturesOfMissingClasses = true
+ suppressAnnotations.add("org.immutables.value.Generated")
+ bundledSignatures.addAll(
+ listOf(
+ "jdk-unsafe",
+ "jdk-deprecated",
+ "jdk-non-portable"
+ )
+ )
+ signaturesFiles = files("$rootDir/src/main/config/forbidden-apis/signatures.txt")
+ }
+
+ if (enableErrorprone) {
+ apply(plugin = "net.ltgt.errorprone")
+ dependencies {
+ "errorprone"("com.google.errorprone:error_prone_core:${"errorprone".v}")
+ "annotationProcessor"("com.google.guava:guava-beta-checker:1.0")
+ }
+ tasks.withType().configureEach {
+ options.errorprone {
+ disableWarningsInGeneratedCode.set(true)
+ errorproneArgs.add("-XepExcludedPaths:.*/javacc/.*")
+ enable(
+ "MethodCanBeStatic"
+ )
+ disable(
+ "ComplexBooleanConstant",
+ "EqualsGetClass",
+ "EqualsHashCode", // verified in Checkstyle
+ "OperatorPrecedence",
+ "MutableConstantField",
+ "ReferenceEquality",
+ "SameNameButDifferent",
+ "TypeParameterUnusedInFormals"
+ )
+ // Analyze issues, and enable the check
+ disable(
+ "BigDecimalEquals",
+ "DoNotCallSuggester",
+ "StringSplitter"
+ )
+ }
+ }
+ }
+ if (enableCheckerframework) {
+ apply(plugin = "org.checkerframework")
+ dependencies {
+ "checkerFramework"("org.checkerframework:checker:${"checkerframework".v}")
+ // CheckerFramework annotations might be used in the code as follows:
+ // dependencies {
+ // "compileOnly"("org.checkerframework:checker-qual")
+ // "testCompileOnly"("org.checkerframework:checker-qual")
+ // }
+ if (JavaVersion.current() == JavaVersion.VERSION_1_8) {
+ // only needed for JDK 8
+ "checkerFrameworkAnnotatedJDK"("org.checkerframework:jdk8")
+ }
+ }
+ configure {
+ skipVersionCheck = true
+ // See https://checkerframework.org/manual/#introduction
+ checkers.add("org.checkerframework.checker.nullness.NullnessChecker")
+ // Below checkers take significant time and they do not provide much value :-/
+ // checkers.add("org.checkerframework.checker.optional.OptionalChecker")
+ // checkers.add("org.checkerframework.checker.regex.RegexChecker")
+ // https://checkerframework.org/manual/#creating-debugging-options-progress
+ // extraJavacArgs.add("-Afilenames")
+ extraJavacArgs.addAll(listOf("-Xmaxerrs", "10000"))
+ // Consider Java assert statements for nullness and other checks
+ extraJavacArgs.add("-AassumeAssertionsAreEnabled")
+ // https://checkerframework.org/manual/#stub-using
+ extraJavacArgs.add("-Astubs=" +
+ fileTree("$rootDir/src/main/config/checkerframework") {
+ include("**/*.astub")
+ }.asPath
+ )
+ if (project.path == ":core") {
+ extraJavacArgs.add("-AskipDefs=^org\\.apache\\.calcite\\.sql\\.parser\\.impl\\.")
+ }
+ }
+ }
+
+ tasks {
+ configureEach {
+ manifest {
+ attributes["Bundle-License"] = "Apache-2.0"
+ attributes["Implementation-Title"] = "Apache Calcite"
+ attributes["Implementation-Version"] = project.version
+ attributes["Specification-Vendor"] = "The Apache Software Foundation"
+ attributes["Specification-Version"] = project.version
+ attributes["Specification-Title"] = "Apache Calcite"
+ attributes["Implementation-Vendor"] = "Apache Software Foundation"
+ attributes["Implementation-Vendor-Id"] = "org.apache.calcite"
+ }
+ }
+
+ configureEach {
+ excludeJavaCcGenerated()
+ exclude(
+ "**/org/apache/calcite/adapter/os/Processes${'$'}ProcessFactory.class",
+ "**/org/apache/calcite/adapter/os/OsAdapterTest.class",
+ "**/org/apache/calcite/runtime/Resources${'$'}Inst.class",
+ "**/org/apache/calcite/test/concurrent/ConcurrentTestCommandScript.class",
+ "**/org/apache/calcite/test/concurrent/ConcurrentTestCommandScript${'$'}ShellCommand.class",
+ "**/org/apache/calcite/util/Unsafe.class",
+ "**/org/apache/calcite/test/Unsafe.class"
+ )
+ }
+
+ configureEach {
+ inputs.property("java.version", System.getProperty("java.version"))
+ inputs.property("java.vm.version", System.getProperty("java.vm.version"))
+ options.encoding = "UTF-8"
+ options.compilerArgs.add("-Xlint:deprecation")
+ if (werror) {
+ options.compilerArgs.add("-Werror")
+ }
+ if (enableCheckerframework) {
+ options.forkOptions.memoryMaximumSize = "2g"
+ }
+ }
+ configureEach {
+ outputs.cacheIf("test results depend on the database configuration, so we souldn't cache it") {
+ false
+ }
+ useJUnitPlatform {
+ excludeTags("slow")
+ }
+ testLogging {
+ exceptionFormat = TestExceptionFormat.FULL
+ showStandardStreams = true
+ }
+ exclude("**/*Suite*")
+ jvmArgs("-Xmx1536m")
+ jvmArgs("-Djdk.net.URLClassPath.disableClassPathURLCheck=true")
+ // Pass the property to tests
+ fun passProperty(name: String, default: String? = null) {
+ val value = System.getProperty(name) ?: default
+ value?.let { systemProperty(name, it) }
+ }
+ passProperty("java.awt.headless")
+ passProperty("junit.jupiter.execution.parallel.enabled", "true")
+ passProperty("junit.jupiter.execution.parallel.mode.default", "concurrent")
+ passProperty("junit.jupiter.execution.timeout.default", "5 m")
+ passProperty("user.language", "TR")
+ passProperty("user.country", "tr")
+ passProperty("user.timezone", "UTC")
+ val props = System.getProperties()
+ for (e in props.propertyNames() as `java.util`.Enumeration) {
+ if (e.startsWith("calcite.") || e.startsWith("avatica.")) {
+ passProperty(e)
+ }
+ }
+ }
+ // Cannot be moved above otherwise configure each will override
+ // also the specific configurations below.
+ register("testSlow") {
+ group = LifecycleBasePlugin.VERIFICATION_GROUP
+ description = "Runs the slow unit tests."
+ useJUnitPlatform() {
+ includeTags("slow")
+ }
+ jvmArgs("-Xmx6g")
+ }
+ configureEach {
+ group = LifecycleBasePlugin.VERIFICATION_GROUP
+ if (enableSpotBugs) {
+ description = "$description (skipped by default, to enable it add -Dspotbugs)"
+ }
+ reports {
+ html.isEnabled = reportsForHumans()
+ xml.isEnabled = !reportsForHumans()
+ }
+ enabled = enableSpotBugs
+ }
+
+ afterEvaluate {
+ // Add default license/notice when missing
+ configureEach {
+ CrLfSpec(LineEndings.LF).run {
+ into("META-INF") {
+ filteringCharset = "UTF-8"
+ duplicatesStrategy = DuplicatesStrategy.EXCLUDE
+ // Note: we need "generic Apache-2.0" text without third-party items
+ // So we use the text from $rootDir/config/ since source distribution
+ // contains altered text at $rootDir/LICENSE
+ textFrom("$rootDir/src/main/config/licenses/LICENSE")
+ textFrom("$rootDir/NOTICE")
+ }
+ }
+ }
+ }
+ }
+
+ // Note: jars below do not normalize line endings.
+ // Those jars, however are not included to source/binary distributions
+ // so the normailzation is not that important
+
+ val testJar by tasks.registering(Jar::class) {
+ from(sourceSets["test"].output)
+ archiveClassifier.set("tests")
+ }
+
+ val sourcesJar by tasks.registering(Jar::class) {
+ from(sourceSets["main"].allJava)
+ archiveClassifier.set("sources")
+ }
+
+ val javadocJar by tasks.registering(Jar::class) {
+ from(tasks.named(JavaPlugin.JAVADOC_TASK_NAME))
+ archiveClassifier.set("javadoc")
+ }
+
+ val archives by configurations.getting
+
+ // Parenthesis needed to use Project#getArtifacts
+ (artifacts) {
+ archives(sourcesJar)
+ }
+
+ val archivesBaseName = "calcite-$name"
+ setProperty("archivesBaseName", archivesBaseName)
+
+ configure {
+ if (project.path == ":") {
+ // Do not publish "root" project. Java plugin is applied here for DSL purposes only
+ return@configure
+ }
+ if (!project.props.bool("nexus.publish", default = true)) {
+ // Some of the artifacts do not need to be published
+ return@configure
+ }
+ publications {
+ create(project.name) {
+ artifactId = archivesBaseName
+ version = rootProject.version.toString()
+ description = project.description
+ from(components["java"])
+
+ if (!skipJavadoc) {
+ // Eager task creation is required due to
+ // https://github.com/gradle/gradle/issues/6246
+ artifact(sourcesJar.get())
+ artifact(javadocJar.get())
+ }
+
+ // Use the resolved versions in pom.xml
+ // Gradle might have different resolution rules, so we set the versions
+ // that were used in Gradle build/test.
+ versionMapping {
+ usage(Usage.JAVA_RUNTIME) {
+ fromResolutionResult()
+ }
+ usage(Usage.JAVA_API) {
+ fromResolutionOf("runtimeClasspath")
+ }
+ }
+ pom {
+ withXml {
+ val sb = asString()
+ var s = sb.toString()
+ // compile is Maven default, so delete it
+ s = s.replace("compile ", "")
+ // Cut because all dependencies have the resolved versions
+ s = s.replace(
+ Regex(
+ ".*? ",
+ RegexOption.DOT_MATCHES_ALL
+ ),
+ ""
+ )
+ sb.setLength(0)
+ sb.append(s)
+ // Re-format the XML
+ asNode()
+ }
+ name.set(
+ (project.findProperty("artifact.name") as? String) ?: "Calcite ${project.name.capitalize()}"
+ )
+ description.set(project.description ?: "Calcite ${project.name.capitalize()}")
+ inceptionYear.set("2012")
+ url.set("https://calcite.apache.org")
+ licenses {
+ license {
+ name.set("The Apache License, Version 2.0")
+ url.set("https://www.apache.org/licenses/LICENSE-2.0.txt")
+ comments.set("A business-friendly OSS license")
+ distribution.set("repo")
+ }
+ }
+ issueManagement {
+ system.set("Jira")
+ url.set("https://issues.apache.org/jira/browse/CALCITE")
+ }
+ mailingLists {
+ mailingList {
+ name.set("Apache Calcite developers list")
+ subscribe.set("dev-subscribe@calcite.apache.org")
+ unsubscribe.set("dev-unsubscribe@calcite.apache.org")
+ post.set("dev@calcite.apache.org")
+ archive.set("https://lists.apache.org/list.html?dev@calcite.apache.org")
+ }
+ }
+ scm {
+ connection.set("scm:git:https://gitbox.apache.org/repos/asf/calcite.git")
+ developerConnection.set("scm:git:https://gitbox.apache.org/repos/asf/calcite.git")
+ url.set("https://github.com/apache/calcite")
+ tag.set("HEAD")
+ }
+ repositories {
+ val finalUrl = if (version.toString().endsWith("SNAPSHOT")) snapshotsRepoUrl else releasesRepoUrl
+ maven {
+ url = finalUrl
+ credentials {
+ username = asfTestNexusUsername
+ password = asfTestNexusPassword
+ }
+ }
+ }
+ }
+ }
+ }
+ }
+ }
+}
diff --git a/buildSrc/build.gradle.kts b/buildSrc/build.gradle.kts
new file mode 100644
index 000000000000..45462214330e
--- /dev/null
+++ b/buildSrc/build.gradle.kts
@@ -0,0 +1,83 @@
+/*
+ * 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.
+ */
+
+import com.github.vlsi.gradle.properties.dsl.props
+import org.jetbrains.kotlin.gradle.tasks.KotlinCompile
+
+plugins {
+ `embedded-kotlin`
+ `kotlin-dsl` apply false
+ id("com.github.autostyle")
+ id("com.github.vlsi.gradle-extensions")
+}
+
+repositories {
+ mavenCentral()
+ gradlePluginPortal()
+}
+
+val skipAutostyle by props()
+
+allprojects {
+ repositories {
+ mavenCentral()
+ gradlePluginPortal()
+ }
+ applyKotlinProjectConventions()
+ tasks.withType().configureEach {
+ // Ensure builds are reproducible
+ isPreserveFileTimestamps = false
+ isReproducibleFileOrder = true
+ dirMode = "775".toInt(8)
+ fileMode = "664".toInt(8)
+ }
+}
+
+fun Project.applyKotlinProjectConventions() {
+ if (project != rootProject) {
+ apply(plugin = "org.gradle.kotlin.kotlin-dsl")
+ }
+
+ tasks.withType {
+ sourceCompatibility = "unused"
+ targetCompatibility = "unused"
+ kotlinOptions {
+ jvmTarget = "1.8"
+ }
+ }
+ if (!skipAutostyle) {
+ apply(plugin = "com.github.autostyle")
+ autostyle {
+ kotlin {
+ ktlint()
+ trimTrailingWhitespace()
+ endWithNewline()
+ }
+ kotlinGradle {
+ ktlint()
+ trimTrailingWhitespace()
+ endWithNewline()
+ }
+ }
+ }
+}
+
+dependencies {
+ subprojects.forEach {
+ runtimeOnly(project(it.path))
+ }
+}
diff --git a/mongodb/src/test/resources/log4j.properties b/buildSrc/gradle.properties
similarity index 71%
rename from mongodb/src/test/resources/log4j.properties
rename to buildSrc/gradle.properties
index 834e2db6842e..767eb7a6192e 100644
--- a/mongodb/src/test/resources/log4j.properties
+++ b/buildSrc/gradle.properties
@@ -1,3 +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 regarding copyright ownership.
@@ -12,13 +13,10 @@
# 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.
+#
+org.gradle.parallel=true
+kotlin.code.style=official
-# Root logger is configured at INFO and is sent to A1
-log4j.rootLogger=INFO, A1
-
-# A1 goes to the console
-log4j.appender.A1=org.apache.log4j.ConsoleAppender
-
-# Set the pattern for each log message
-log4j.appender.A1.layout=org.apache.log4j.PatternLayout
-log4j.appender.A1.layout.ConversionPattern=%d [%t] %-5p - %m%n
+# Plugins
+com.github.autostyle.version=3.0
+com.github.vlsi.vlsi-release-plugins.version=1.52
diff --git a/buildSrc/settings.gradle.kts b/buildSrc/settings.gradle.kts
new file mode 100644
index 000000000000..1149c90d8d78
--- /dev/null
+++ b/buildSrc/settings.gradle.kts
@@ -0,0 +1,45 @@
+/*
+ * 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.
+ */
+
+pluginManagement {
+ plugins {
+ fun String.v() = extra["$this.version"].toString()
+ fun PluginDependenciesSpec.idv(id: String, key: String = id) = id(id) version key.v()
+ idv("com.github.autostyle")
+ idv("com.github.vlsi.gradle-extensions", "com.github.vlsi.vlsi-release-plugins")
+ }
+}
+
+include("javacc")
+include("fmpp")
+include("buildext")
+
+val upperCaseLetters = "\\p{Upper}".toRegex()
+
+fun String.toKebabCase() =
+ replace(upperCaseLetters) { "-${it.value.toLowerCase()}" }
+
+fun buildFileNameFor(projectDirName: String) =
+ "$projectDirName.gradle.kts"
+
+for (project in rootProject.children) {
+ val projectDirName = project.name.toKebabCase()
+ project.projectDir = file("subprojects/$projectDirName")
+ project.buildFileName = buildFileNameFor(projectDirName)
+ assert(project.projectDir.isDirectory)
+ assert(project.buildFile.isFile)
+}
diff --git a/buildSrc/subprojects/buildext/buildext.gradle.kts b/buildSrc/subprojects/buildext/buildext.gradle.kts
new file mode 100644
index 000000000000..e074f4848fbe
--- /dev/null
+++ b/buildSrc/subprojects/buildext/buildext.gradle.kts
@@ -0,0 +1,26 @@
+/*
+ * 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.
+ *
+ */
+
+gradlePlugin {
+ plugins {
+ register("buildext") {
+ id = "calcite.buildext"
+ implementationClass = "org.apache.calcite.buildtools.buildext.BuildExtPlugin"
+ }
+ }
+}
diff --git a/buildSrc/subprojects/buildext/src/main/kotlin/org/apache/calcite/buildtools/buildext/BuildExtPlugin.kt b/buildSrc/subprojects/buildext/src/main/kotlin/org/apache/calcite/buildtools/buildext/BuildExtPlugin.kt
new file mode 100644
index 000000000000..463fd08f9d80
--- /dev/null
+++ b/buildSrc/subprojects/buildext/src/main/kotlin/org/apache/calcite/buildtools/buildext/BuildExtPlugin.kt
@@ -0,0 +1,27 @@
+/*
+ * 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.calcite.buildtools.buildext
+
+import org.gradle.api.Plugin
+import org.gradle.api.Project
+
+class BuildExtPlugin : Plugin {
+ override fun apply(target: Project) {
+ }
+}
diff --git a/buildSrc/subprojects/buildext/src/main/kotlin/org/apache/calcite/buildtools/buildext/dsl/ParenthesisBalancer.kt b/buildSrc/subprojects/buildext/src/main/kotlin/org/apache/calcite/buildtools/buildext/dsl/ParenthesisBalancer.kt
new file mode 100644
index 000000000000..c1c7163783ce
--- /dev/null
+++ b/buildSrc/subprojects/buildext/src/main/kotlin/org/apache/calcite/buildtools/buildext/dsl/ParenthesisBalancer.kt
@@ -0,0 +1,101 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.calcite.buildtools.buildext.dsl
+
+import java.util.function.Function
+
+private const val SINGLE_LINE_COMMENT = "//.*+"
+private const val MULTILINE_COMMENT = "/[*](?>\\\\[*]|[*][^/]|[^*])*+[*]/"
+private const val STRING_LITERAL = "\"(?>\\\\.|[^\"])*+\""
+private const val CHAR_LITERAL = "'(?>\\\\.|[^'])'"
+
+private const val KEYWORDS = "\\b(?>for|if|return|switch|try|while)\\b"
+private const val KEYWORD_BLOCK = "$KEYWORDS *\\("
+private const val WHITESPACE = "(?:(?!$KEYWORDS|[(),\"'/]).)++"
+
+// Below Regex matches one token at a time
+// That is it breaks if (!canCastFrom(/*comment*/callBinding, throwOnFailure into the following sequence
+// "if (", "!canCastFrom", "(", "/*comment*/", "callBinding", ",", " throwOnFailure"
+// This enables to skip strings, comments, and capture the position of commas and parenthesis
+
+private val tokenizer =
+ Regex("(?>$SINGLE_LINE_COMMENT|$MULTILINE_COMMENT|$STRING_LITERAL|$CHAR_LITERAL|$KEYWORD_BLOCK|$WHITESPACE|.)")
+private val looksLikeJavadoc = Regex("^ +\\* ")
+
+// Note: if you change the logic, please remember to update the value in
+// build.gradle.kts / bumpThisNumberIfACustomStepChanges
+// Otherwise Autostyle would assume the files are up to date
+object ParenthesisBalancer : Function {
+ override fun apply(v: String): String = v.lines().map { line ->
+ if ('(' !in line || looksLikeJavadoc.containsMatchIn(line)) {
+ return@map line
+ }
+ var balance = 0
+ var seenOpen = false
+ var commaSplit = 0
+ var lastOpen = 0
+ for (m in tokenizer.findAll(line)) {
+ val range = m.range
+ if (range.last - range.first > 1) {
+ // parenthesis always take one char, so ignore long matches
+ continue
+ }
+ val c = line[range.first]
+ if (c == '(') {
+ seenOpen = true
+ if (balance == 0) {
+ lastOpen = range.first + 1
+ }
+ balance += 1
+ continue
+ } else if (!seenOpen) {
+ continue
+ }
+ if (c == ',' && balance == 0) {
+ commaSplit = range.first + 1
+ }
+ if (c == ')') {
+ balance -= 1
+ }
+ }
+ if (balance <= 1) {
+ line
+ } else {
+ val indent = line.indexOfFirst { it != ' ' }
+ val res = if (commaSplit == 0) {
+ // f1(1,f2(2,... pattern
+ // ^-- lastOpen, commaSplit=0 (no split)
+ // It is split right after ('
+ line.substring(0, lastOpen) + "\n" + " ".repeat(indent + 4) +
+ line.substring(lastOpen)
+ } else {
+ // f1(1), f2(2,... pattern
+ // ^ ^-- lastOpen
+ // '-- commaSplit
+ // It is split twice: right after the comma, and after (
+ line.substring(0, commaSplit) +
+ "\n" + " ".repeat(indent) +
+ line.substring(commaSplit, lastOpen).trimStart(' ') +
+ "\n" + " ".repeat(indent + 4) + line.substring(lastOpen)
+ }
+ // println("---\n$line\n->\n$res")
+ res
+ }
+ }.joinToString("\n")
+}
diff --git a/buildSrc/subprojects/fmpp/fmpp.gradle.kts b/buildSrc/subprojects/fmpp/fmpp.gradle.kts
new file mode 100644
index 000000000000..3937b7f22883
--- /dev/null
+++ b/buildSrc/subprojects/fmpp/fmpp.gradle.kts
@@ -0,0 +1,28 @@
+/*
+ * 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.
+ */
+
+dependencies {
+}
+
+gradlePlugin {
+ plugins {
+ register("fmpp") {
+ id = "calcite.fmpp"
+ implementationClass = "org.apache.calcite.buildtools.fmpp.FmppPlugin"
+ }
+ }
+}
diff --git a/buildSrc/subprojects/fmpp/src/main/kotlin/org/apache/calcite/buildtools/fmpp/FmppPlugin.kt b/buildSrc/subprojects/fmpp/src/main/kotlin/org/apache/calcite/buildtools/fmpp/FmppPlugin.kt
new file mode 100644
index 000000000000..1b9ce6135025
--- /dev/null
+++ b/buildSrc/subprojects/fmpp/src/main/kotlin/org/apache/calcite/buildtools/fmpp/FmppPlugin.kt
@@ -0,0 +1,41 @@
+/*
+ * 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.calcite.buildtools.fmpp
+
+import org.gradle.api.Plugin
+import org.gradle.api.Project
+
+open class FmppPlugin : Plugin {
+ companion object {
+ const val FMPP_CLASSPATH_CONFIGURATION_NAME = "fmppClaspath"
+ }
+
+ override fun apply(target: Project) {
+ target.configureFmpp()
+ }
+
+ fun Project.configureFmpp() {
+ configurations.create(FMPP_CLASSPATH_CONFIGURATION_NAME) {
+ isCanBeConsumed = false
+ }.defaultDependencies {
+ // TODO: use properties for versions
+ add(dependencies.create("org.freemarker:freemarker:2.3.29"))
+ add(dependencies.create("net.sourceforge.fmpp:fmpp:0.9.16"))
+ }
+ }
+}
diff --git a/buildSrc/subprojects/fmpp/src/main/kotlin/org/apache/calcite/buildtools/fmpp/FmppTask.kt b/buildSrc/subprojects/fmpp/src/main/kotlin/org/apache/calcite/buildtools/fmpp/FmppTask.kt
new file mode 100644
index 000000000000..2f3ed176b1dd
--- /dev/null
+++ b/buildSrc/subprojects/fmpp/src/main/kotlin/org/apache/calcite/buildtools/fmpp/FmppTask.kt
@@ -0,0 +1,85 @@
+/*
+ * 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.calcite.buildtools.fmpp
+
+import javax.inject.Inject
+import org.gradle.api.DefaultTask
+import org.gradle.api.artifacts.Configuration
+import org.gradle.api.model.ObjectFactory
+import org.gradle.api.tasks.CacheableTask
+import org.gradle.api.tasks.Classpath
+import org.gradle.api.tasks.InputDirectory
+import org.gradle.api.tasks.InputFile
+import org.gradle.api.tasks.OutputDirectory
+import org.gradle.api.tasks.PathSensitive
+import org.gradle.api.tasks.PathSensitivity
+import org.gradle.api.tasks.TaskAction
+import org.gradle.kotlin.dsl.property
+import org.gradle.kotlin.dsl.withGroovyBuilder
+
+@CacheableTask
+open class FmppTask @Inject constructor(
+ objectFactory: ObjectFactory
+) : DefaultTask() {
+ @Classpath
+ val fmppClasspath = objectFactory.property()
+ .convention(project.configurations.named(FmppPlugin.FMPP_CLASSPATH_CONFIGURATION_NAME))
+
+ @InputFile
+ @PathSensitive(PathSensitivity.NONE)
+ val config = objectFactory.fileProperty()
+
+ @InputDirectory
+ @PathSensitive(PathSensitivity.RELATIVE)
+ val templates = objectFactory.directoryProperty()
+
+ @InputFile
+ @PathSensitive(PathSensitivity.NONE)
+ val defaultConfig = objectFactory.fileProperty()
+ .convention(templates.file("../default_config.fmpp"))
+
+ @OutputDirectory
+ val output = objectFactory.directoryProperty()
+ .convention(project.layout.buildDirectory.dir("fmpp/$name"))
+
+ /**
+ * Path might contain spaces and TDD special characters, so it needs to be quoted.
+ * See http://fmpp.sourceforge.net/tdd.html
+ */
+ private fun String.tddString() =
+ "\"${toString().replace("\\", "\\\\").replace("\"", "\\\"")}\""
+
+ @TaskAction
+ fun run() {
+ project.delete(output.asFileTree)
+ ant.withGroovyBuilder {
+ "taskdef"(
+ "name" to "fmpp",
+ "classname" to "fmpp.tools.AntTask",
+ "classpath" to fmppClasspath.get().asPath
+ )
+ "fmpp"(
+ "configuration" to config.get(),
+ "sourceRoot" to templates.get().asFile,
+ "outputRoot" to output.get().asFile,
+ "data" to "tdd(${config.get().toString().tddString()}), " +
+ "default: tdd(${defaultConfig.get().toString().tddString()})"
+ )
+ }
+ }
+}
diff --git a/buildSrc/subprojects/javacc/javacc.gradle.kts b/buildSrc/subprojects/javacc/javacc.gradle.kts
new file mode 100644
index 000000000000..90caa7e5c69e
--- /dev/null
+++ b/buildSrc/subprojects/javacc/javacc.gradle.kts
@@ -0,0 +1,28 @@
+/*
+ * 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.
+ */
+
+dependencies {
+}
+
+gradlePlugin {
+ plugins {
+ register("javacc") {
+ id = "calcite.javacc"
+ implementationClass = "org.apache.calcite.buildtools.javacc.JavaCCPlugin"
+ }
+ }
+}
diff --git a/buildSrc/subprojects/javacc/src/main/kotlin/org/apache/calcite/buildtools/javacc/JavaCCPlugin.kt b/buildSrc/subprojects/javacc/src/main/kotlin/org/apache/calcite/buildtools/javacc/JavaCCPlugin.kt
new file mode 100644
index 000000000000..78f80b3b8dda
--- /dev/null
+++ b/buildSrc/subprojects/javacc/src/main/kotlin/org/apache/calcite/buildtools/javacc/JavaCCPlugin.kt
@@ -0,0 +1,47 @@
+/*
+ * 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.calcite.buildtools.javacc
+
+import org.gradle.api.Plugin
+import org.gradle.api.Project
+import org.gradle.kotlin.dsl.withType
+
+open class JavaCCPlugin : Plugin {
+ companion object {
+ const val JAVACC_CLASSPATH_CONFIGURATION_NAME = "javaccClaspath"
+ const val GENERATE_SOURCES_TASK_NAME = "generateSources"
+ }
+
+ override fun apply(target: Project) {
+ target.configureJavaCC()
+ }
+
+ fun Project.configureJavaCC() {
+ configurations.create(JAVACC_CLASSPATH_CONFIGURATION_NAME) {
+ isCanBeConsumed = false
+ }.defaultDependencies {
+ // TODO: use properties for versions
+ add(dependencies.create("net.java.dev.javacc:javacc:4.0")) // 7.0.5"))
+ }
+
+ tasks.register(GENERATE_SOURCES_TASK_NAME) {
+ description = "Generates sources (e.g. JavaCC)"
+ dependsOn(tasks.withType())
+ }
+ }
+}
diff --git a/buildSrc/subprojects/javacc/src/main/kotlin/org/apache/calcite/buildtools/javacc/JavaCCTask.kt b/buildSrc/subprojects/javacc/src/main/kotlin/org/apache/calcite/buildtools/javacc/JavaCCTask.kt
new file mode 100644
index 000000000000..340b7d8cbdf8
--- /dev/null
+++ b/buildSrc/subprojects/javacc/src/main/kotlin/org/apache/calcite/buildtools/javacc/JavaCCTask.kt
@@ -0,0 +1,73 @@
+/*
+ * 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.calcite.buildtools.javacc
+
+import javax.inject.Inject
+import org.gradle.api.DefaultTask
+import org.gradle.api.artifacts.Configuration
+import org.gradle.api.model.ObjectFactory
+import org.gradle.api.tasks.CacheableTask
+import org.gradle.api.tasks.Classpath
+import org.gradle.api.tasks.Input
+import org.gradle.api.tasks.InputFiles
+import org.gradle.api.tasks.OutputDirectory
+import org.gradle.api.tasks.PathSensitive
+import org.gradle.api.tasks.PathSensitivity
+import org.gradle.api.tasks.TaskAction
+import org.gradle.kotlin.dsl.property
+
+@CacheableTask
+open class JavaCCTask @Inject constructor(
+ objectFactory: ObjectFactory
+) : DefaultTask() {
+ @Classpath
+ val javaCCClasspath = objectFactory.property()
+ .convention(project.configurations.named(JavaCCPlugin.JAVACC_CLASSPATH_CONFIGURATION_NAME))
+
+ @InputFiles
+ @PathSensitive(PathSensitivity.NONE)
+ // We expect one file only, however there's https://github.com/gradle/gradle/issues/12627
+ val inputFile = objectFactory.fileCollection()
+
+ @Input
+ val lookAhead = objectFactory.property().convention(1)
+
+ @Input
+ val static = objectFactory.property().convention(false)
+
+ @OutputDirectory
+ val output = objectFactory.directoryProperty()
+ .convention(project.layout.buildDirectory.dir("javacc/$name"))
+
+ @Input
+ val packageName = objectFactory.property()
+
+ @TaskAction
+ fun run() {
+ project.delete(output.asFileTree)
+ project.javaexec {
+ classpath = javaCCClasspath.get()
+ // The class is in the top-level package
+ main = "javacc"
+ args("-STATIC=${static.get()}")
+ args("-LOOKAHEAD:${lookAhead.get()}")
+ args("-OUTPUT_DIRECTORY:${output.get()}/${packageName.get().replace('.', '/')}")
+ args(inputFile.singleFile)
+ }
+ }
+}
diff --git a/cassandra/build.gradle.kts b/cassandra/build.gradle.kts
new file mode 100644
index 000000000000..bb00dfb4f0a7
--- /dev/null
+++ b/cassandra/build.gradle.kts
@@ -0,0 +1,88 @@
+/*
+ * 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.
+ */
+import com.github.vlsi.gradle.ide.dsl.settings
+import com.github.vlsi.gradle.ide.dsl.taskTriggers
+
+plugins {
+ id("com.github.vlsi.ide")
+}
+
+dependencies {
+ api(project(":core"))
+ api(project(":linq4j"))
+
+ api("com.datastax.oss:java-driver-core")
+ api("org.apache.kylin:kylin-external-guava30")
+ api("org.slf4j:slf4j-api")
+
+ implementation("org.apache.calcite.avatica:avatica-core")
+
+ testImplementation(project(":testkit"))
+ testImplementation("org.apache.cassandra:cassandra-all") {
+ exclude("org.slf4j", "log4j-over-slf4j")
+ .because("log4j is already present in the classpath")
+ exclude("ch.qos.logback", "logback-core")
+ .because("conflicts with log4j-slf4j-impl")
+ exclude("ch.qos.logback", "logback-classic")
+ .because("conflicts with log4j-slf4j-impl")
+ }
+ testImplementation("org.cassandraunit:cassandra-unit") {
+ exclude("ch.qos.logback", "logback-core")
+ .because("conflicts with log4j-slf4j-impl")
+ exclude("ch.qos.logback", "logback-classic")
+ .because("conflicts with log4j-slf4j-impl")
+ }
+ testRuntimeOnly("net.java.dev.jna:jna")
+
+ annotationProcessor("org.immutables:value")
+ compileOnly("org.immutables:value-annotations")
+ compileOnly("com.google.code.findbugs:jsr305")
+ testRuntimeOnly("org.apache.logging.log4j:log4j-slf4j-impl")
+}
+
+fun JavaCompile.configureAnnotationSet(sourceSet: SourceSet) {
+ source = sourceSet.java
+ classpath = sourceSet.compileClasspath
+ options.compilerArgs.add("-proc:only")
+ org.gradle.api.plugins.internal.JvmPluginsHelper.configureAnnotationProcessorPath(sourceSet, sourceSet.java, options, project)
+ destinationDirectory.set(temporaryDir)
+
+ // only if we aren't running compileJava, since doing twice fails (in some places)
+ onlyIf { !project.gradle.taskGraph.hasTask(sourceSet.getCompileTaskName("java")) }
+}
+
+val annotationProcessorMain by tasks.registering(JavaCompile::class) {
+ configureAnnotationSet(sourceSets.main.get())
+}
+
+ide {
+ // generate annotation processed files on project import/sync.
+ // adds to idea path but skip don't add to SourceSet since that triggers checkstyle
+ fun generatedSource(compile: TaskProvider) {
+ project.rootProject.configure {
+ project {
+ settings {
+ taskTriggers {
+ afterSync(compile.get())
+ }
+ }
+ }
+ }
+ }
+
+ generatedSource(annotationProcessorMain)
+}
diff --git a/cassandra/gradle.properties b/cassandra/gradle.properties
new file mode 100644
index 000000000000..32e45ba329cb
--- /dev/null
+++ b/cassandra/gradle.properties
@@ -0,0 +1,18 @@
+#
+# 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.
+#
+description=Cassandra adapter for Calcite
+artifact.name=Calcite Cassandra
diff --git a/cassandra/pom.xml b/cassandra/pom.xml
deleted file mode 100644
index d98ce6cd3dcd..000000000000
--- a/cassandra/pom.xml
+++ /dev/null
@@ -1,143 +0,0 @@
-
-
-
- 4.0.0
-
- org.apache.calcite
- calcite
- 1.13.0
-
-
- calcite-cassandra
- jar
- 1.13.0
- Calcite Cassandra
- Cassandra adapter for Calcite
-
-
- ${project.basedir}/..
-
-
-
-
-
- org.apache.calcite.avatica
- avatica-core
-
-
- org.apache.calcite
- calcite-core
- jar
-
-
- org.apache.calcite
- calcite-core
- test-jar
- test
-
-
- org.apache.calcite
- calcite-linq4j
-
-
-
- com.google.guava
- guava
-
-
- junit
- junit
- test
-
-
- com.datastax.cassandra
- cassandra-driver-core
-
-
- org.slf4j
- slf4j-api
-
-
- org.slf4j
- slf4j-log4j12
- test
-
-
-
-
-
-
-
- maven-dependency-plugin
- ${maven-dependency-plugin.version}
-
-
- analyze
-
- analyze-only
-
-
- true
-
-
- org.slf4j:slf4j-api
- org.slf4j:slf4j-log4j12
-
-
-
-
-
-
- org.apache.maven.plugins
- maven-jar-plugin
-
-
-
- test-jar
-
-
-
-
-
- org.apache.maven.plugins
- maven-release-plugin
-
-
-
- org.apache.maven.plugins
- maven-source-plugin
-
-
- attach-sources
- verify
-
- jar-no-fork
- test-jar-no-fork
-
-
-
-
-
-
-
-
diff --git a/cassandra/src/main/java/org/apache/calcite/adapter/cassandra/CassandraEnumerator.java b/cassandra/src/main/java/org/apache/calcite/adapter/cassandra/CassandraEnumerator.java
index a7436fd1b27a..1cf187a5de87 100644
--- a/cassandra/src/main/java/org/apache/calcite/adapter/cassandra/CassandraEnumerator.java
+++ b/cassandra/src/main/java/org/apache/calcite/adapter/cassandra/CassandraEnumerator.java
@@ -16,33 +16,44 @@
*/
package org.apache.calcite.adapter.cassandra;
+import org.apache.calcite.avatica.util.ByteString;
import org.apache.calcite.linq4j.Enumerator;
import org.apache.calcite.rel.type.RelDataTypeFactory;
import org.apache.calcite.rel.type.RelDataTypeField;
import org.apache.calcite.rel.type.RelDataTypeSystem;
import org.apache.calcite.rel.type.RelProtoDataType;
import org.apache.calcite.sql.type.SqlTypeFactoryImpl;
-import org.apache.calcite.sql.type.SqlTypeName;
-import com.datastax.driver.core.DataType;
-import com.datastax.driver.core.ResultSet;
-import com.datastax.driver.core.Row;
+import com.datastax.oss.driver.api.core.cql.ResultSet;
+import com.datastax.oss.driver.api.core.cql.Row;
+import com.datastax.oss.driver.api.core.data.TupleValue;
+import com.datastax.oss.driver.api.core.type.codec.registry.CodecRegistry;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+import java.nio.ByteBuffer;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalTime;
+import java.util.Date;
import java.util.Iterator;
+import java.util.LinkedHashSet;
import java.util.List;
+import java.util.Objects;
+import java.util.stream.IntStream;
/** Enumerator that reads from a Cassandra column family. */
class CassandraEnumerator implements Enumerator {
- private Iterator iterator;
- private Row current;
- private List fieldTypes;
+ private final Iterator iterator;
+ private final List fieldTypes;
+ @Nullable private Row current;
/** Creates a CassandraEnumerator.
*
- * @param results Cassandra result set ({@link com.datastax.driver.core.ResultSet})
+ * @param results Cassandra result set ({@link com.datastax.oss.driver.api.core.cql.ResultSet})
* @param protoRowType The type of resulting rows
*/
- public CassandraEnumerator(ResultSet results, RelProtoDataType protoRowType) {
+ CassandraEnumerator(ResultSet results, RelProtoDataType protoRowType) {
this.iterator = results.iterator();
this.current = null;
@@ -51,19 +62,19 @@ public CassandraEnumerator(ResultSet results, RelProtoDataType protoRowType) {
this.fieldTypes = protoRowType.apply(typeFactory).getFieldList();
}
- /** Produce the next row from the results
+ /** Produces the next row from the results.
*
* @return A new row from the results
*/
- public Object current() {
+ @Override public Object current() {
if (fieldTypes.size() == 1) {
// If we just have one field, produce it directly
- return currentRowField(0, fieldTypes.get(0).getType().getSqlTypeName());
+ return currentRowField(0);
} else {
// Build an array with all fields in this row
Object[] row = new Object[fieldTypes.size()];
for (int i = 0; i < fieldTypes.size(); i++) {
- row[i] = currentRowField(i, fieldTypes.get(i).getType().getSqlTypeName());
+ row[i] = currentRowField(i);
}
return row;
@@ -73,28 +84,58 @@ public Object current() {
/** Get a field for the current row from the underlying object.
*
* @param index Index of the field within the Row object
- * @param typeName Type of the field in this row
*/
- private Object currentRowField(int index, SqlTypeName typeName) {
- DataType type = current.getColumnDefinitions().getType(index);
- if (type == DataType.ascii() || type == DataType.text() || type == DataType.varchar()) {
- return current.getString(index);
- } else if (type == DataType.cint() || type == DataType.varint()) {
- return current.getInt(index);
- } else if (type == DataType.bigint()) {
- return current.getLong(index);
- } else if (type == DataType.cdouble()) {
- return current.getDouble(index);
- } else if (type == DataType.cfloat()) {
- return current.getFloat(index);
- } else if (type == DataType.uuid() || type == DataType.timeuuid()) {
- return current.getUUID(index).toString();
- } else {
- return null;
+ private @Nullable Object currentRowField(int index) {
+ assert current != null;
+ final Object o = current.get(index,
+ CodecRegistry.DEFAULT.codecFor(
+ current.getColumnDefinitions().get(index).getType()));
+
+ return convertToEnumeratorObject(o);
+ }
+
+ /** Convert an object into the expected internal representation.
+ *
+ * @param obj Object to convert, if needed
+ */
+ private @Nullable Object convertToEnumeratorObject(@Nullable Object obj) {
+ if (obj instanceof ByteBuffer) {
+ ByteBuffer buf = (ByteBuffer) obj;
+ byte [] bytes = new byte[buf.remaining()];
+ buf.get(bytes, 0, bytes.length);
+ return new ByteString(bytes);
+ } else if (obj instanceof LocalDate) {
+ // converts dates to the expected numeric format
+ return ((LocalDate) obj).toEpochDay();
+ } else if (obj instanceof Date) {
+ @SuppressWarnings("JdkObsolete")
+ long milli = ((Date) obj).toInstant().toEpochMilli();
+ return milli;
+ } else if (obj instanceof Instant) {
+ return ((Instant) obj).toEpochMilli();
+ } else if (obj instanceof LocalTime) {
+ return ((LocalTime) obj).toNanoOfDay();
+ } else if (obj instanceof LinkedHashSet) {
+ // MULTISET is handled as an array
+ return ((LinkedHashSet>) obj).toArray();
+ } else if (obj instanceof TupleValue) {
+ // STRUCT can be handled as an array
+ final TupleValue tupleValue = (TupleValue) obj;
+ int numComponents = tupleValue.getType().getComponentTypes().size();
+ return IntStream.range(0, numComponents)
+ .mapToObj(i ->
+ tupleValue.get(i,
+ CodecRegistry.DEFAULT.codecFor(
+ tupleValue.getType().getComponentTypes().get(i)))
+ ).map(this::convertToEnumeratorObject)
+ .map(Objects::requireNonNull) // "null" cannot appear inside collections
+ .toArray();
}
+
+ return obj;
}
- public boolean moveNext() {
+ @Override public boolean moveNext() {
if (iterator.hasNext()) {
current = iterator.next();
return true;
@@ -103,13 +144,11 @@ public boolean moveNext() {
}
}
- public void reset() {
+ @Override public void reset() {
throw new UnsupportedOperationException();
}
- public void close() {
+ @Override public void close() {
// Nothing to do here
}
}
-
-// End CassandraEnumerator.java
diff --git a/cassandra/src/main/java/org/apache/calcite/adapter/cassandra/CassandraFilter.java b/cassandra/src/main/java/org/apache/calcite/adapter/cassandra/CassandraFilter.java
index ba8aa9cbc69e..aeb749a6d288 100644
--- a/cassandra/src/main/java/org/apache/calcite/adapter/cassandra/CassandraFilter.java
+++ b/cassandra/src/main/java/org/apache/calcite/adapter/cassandra/CassandraFilter.java
@@ -28,19 +28,29 @@
import org.apache.calcite.rel.core.Filter;
import org.apache.calcite.rel.metadata.RelMetadataQuery;
import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeField;
import org.apache.calcite.rex.RexCall;
import org.apache.calcite.rex.RexInputRef;
import org.apache.calcite.rex.RexLiteral;
import org.apache.calcite.rex.RexNode;
import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.util.DateString;
+import org.apache.calcite.util.TimestampString;
import org.apache.calcite.util.Util;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+import java.text.SimpleDateFormat;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashSet;
import java.util.List;
+import java.util.Objects;
import java.util.Set;
+import static org.apache.calcite.util.DateTimeStringUtils.ISO_DATETIME_FRACTIONAL_SECOND_FORMAT;
+import static org.apache.calcite.util.DateTimeStringUtils.getDateFormatter;
+
/**
* Implementation of a {@link org.apache.calcite.rel.core.Filter}
* relational expression in Cassandra.
@@ -49,9 +59,9 @@ public class CassandraFilter extends Filter implements CassandraRel {
private final List partitionKeys;
private Boolean singlePartition;
private final List clusteringKeys;
- private List implicitFieldCollations;
- private RelCollation implicitCollation;
- private String match;
+ private final List implicitFieldCollations;
+ private final RelCollation implicitCollation;
+ private final String match;
public CassandraFilter(
RelOptCluster cluster,
@@ -65,7 +75,7 @@ public CassandraFilter(
this.partitionKeys = partitionKeys;
this.singlePartition = false;
- this.clusteringKeys = new ArrayList(clusteringKeys);
+ this.clusteringKeys = new ArrayList<>(clusteringKeys);
this.implicitFieldCollations = implicitFieldCollations;
Translator translator =
@@ -79,18 +89,18 @@ public CassandraFilter(
assert getConvention() == child.getConvention();
}
- @Override public RelOptCost computeSelfCost(RelOptPlanner planner,
+ @Override public @Nullable RelOptCost computeSelfCost(RelOptPlanner planner,
RelMetadataQuery mq) {
return super.computeSelfCost(planner, mq).multiplyBy(0.1);
}
- public CassandraFilter copy(RelTraitSet traitSet, RelNode input,
+ @Override public CassandraFilter copy(RelTraitSet traitSet, RelNode input,
RexNode condition) {
return new CassandraFilter(getCluster(), traitSet, input, condition,
partitionKeys, clusteringKeys, implicitFieldCollations);
}
- public void implement(Implementor implementor) {
+ @Override public void implement(Implementor implementor) {
implementor.visitChild(0, getInput());
implementor.add(null, Collections.singletonList(match));
}
@@ -124,7 +134,7 @@ static class Translator {
List implicitFieldCollations) {
this.rowType = rowType;
this.fieldNames = CassandraRules.cassandraFieldNames(rowType);
- this.partitionKeys = new HashSet(partitionKeys);
+ this.partitionKeys = new HashSet<>(partitionKeys);
this.clusteringKeys = clusteringKeys;
this.restrictedClusteringKeys = 0;
this.implicitFieldCollations = implicitFieldCollations;
@@ -149,7 +159,7 @@ public RelCollation getImplicitCollation() {
}
// Pull out the correct fields along with their original collations
- List fieldCollations = new ArrayList();
+ List fieldCollations = new ArrayList<>();
for (int i = restrictedClusteringKeys; i < clusteringKeys.size(); i++) {
int fieldIndex = fieldNames.indexOf(clusteringKeys.get(i));
RelFieldCollation.Direction direction = implicitFieldCollations.get(i).getDirection();
@@ -174,16 +184,27 @@ private String translateMatch(RexNode condition) {
}
}
- /** Conver the value of a literal to a string.
+ /** Returns the value of the literal.
*
* @param literal Literal to translate
- * @return String representation of the literal
+ * @return The value of the literal in the form of the actual type.
*/
- private static String literalValue(RexLiteral literal) {
- Object value = literal.getValue2();
- StringBuilder buf = new StringBuilder();
- buf.append(value);
- return buf.toString();
+ private static Object literalValue(RexLiteral literal) {
+ Comparable> value = RexLiteral.value(literal);
+ switch (literal.getTypeName()) {
+ case TIMESTAMP:
+ case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
+ assert value instanceof TimestampString;
+ final SimpleDateFormat dateFormatter =
+ getDateFormatter(ISO_DATETIME_FRACTIONAL_SECOND_FORMAT);
+ return dateFormatter.format(literal.getValue2());
+ case DATE:
+ assert value instanceof DateString;
+ return value.toString();
+ default:
+ Object val = literal.getValue3();
+ return val == null ? "null" : val;
+ }
}
/** Translate a conjunctive predicate to a CQL string.
@@ -192,7 +213,7 @@ private static String literalValue(RexLiteral literal) {
* @return CQL string for the predicate
*/
private String translateAnd(RexNode condition) {
- List predicates = new ArrayList();
+ List predicates = new ArrayList<>();
for (RexNode node : RelOptUtil.conjunctions(condition)) {
predicates.add(translateMatch2(node));
}
@@ -237,7 +258,7 @@ private String translateBinary(String op, String rop, RexCall call) {
}
/** Translates a call to a binary operator. Returns null on failure. */
- private String translateBinary2(String op, RexNode left, RexNode right) {
+ private @Nullable String translateBinary2(String op, RexNode left, RexNode right) {
switch (right.getKind()) {
case LITERAL:
break;
@@ -271,7 +292,9 @@ private String translateOp2(String op, String name, RexLiteral right) {
Object value = literalValue(right);
String valueString = value.toString();
if (value instanceof String) {
- SqlTypeName typeName = rowType.getField(name, true, false).getType().getSqlTypeName();
+ RelDataTypeField field =
+ Objects.requireNonNull(rowType.getField(name, true, false));
+ SqlTypeName typeName = field.getType().getSqlTypeName();
if (typeName != SqlTypeName.CHAR) {
valueString = "'" + valueString + "'";
}
@@ -280,5 +303,3 @@ private String translateOp2(String op, String name, RexLiteral right) {
}
}
}
-
-// End CassandraFilter.java
diff --git a/cassandra/src/main/java/org/apache/calcite/adapter/cassandra/CassandraLimit.java b/cassandra/src/main/java/org/apache/calcite/adapter/cassandra/CassandraLimit.java
index cca7e19d2072..ee5f6aa77652 100644
--- a/cassandra/src/main/java/org/apache/calcite/adapter/cassandra/CassandraLimit.java
+++ b/cassandra/src/main/java/org/apache/calcite/adapter/cassandra/CassandraLimit.java
@@ -27,24 +27,26 @@
import org.apache.calcite.rex.RexLiteral;
import org.apache.calcite.rex.RexNode;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
import java.util.List;
/**
* Implementation of limits in Cassandra.
*/
public class CassandraLimit extends SingleRel implements CassandraRel {
- public final RexNode offset;
- public final RexNode fetch;
+ public final @Nullable RexNode offset;
+ public final @Nullable RexNode fetch;
public CassandraLimit(RelOptCluster cluster, RelTraitSet traitSet,
- RelNode input, RexNode offset, RexNode fetch) {
+ RelNode input, @Nullable RexNode offset, @Nullable RexNode fetch) {
super(cluster, traitSet, input);
this.offset = offset;
this.fetch = fetch;
assert getConvention() == input.getConvention();
}
- @Override public RelOptCost computeSelfCost(RelOptPlanner planner,
+ @Override public @Nullable RelOptCost computeSelfCost(RelOptPlanner planner,
RelMetadataQuery mq) {
// We do this so we get the limit for free
return planner.getCostFactory().makeZeroCost();
@@ -54,18 +56,20 @@ public CassandraLimit(RelOptCluster cluster, RelTraitSet traitSet,
return new CassandraLimit(getCluster(), traitSet, sole(newInputs), offset, fetch);
}
- public void implement(Implementor implementor) {
+ @Override public void implement(Implementor implementor) {
implementor.visitChild(0, getInput());
- if (offset != null) { implementor.offset = RexLiteral.intValue(offset); }
- if (fetch != null) { implementor.fetch = RexLiteral.intValue(fetch); }
+ if (offset != null) {
+ implementor.offset = RexLiteral.intValue(offset);
+ }
+ if (fetch != null) {
+ implementor.fetch = RexLiteral.intValue(fetch);
+ }
}
- public RelWriter explainTerms(RelWriter pw) {
+ @Override public RelWriter explainTerms(RelWriter pw) {
super.explainTerms(pw);
pw.itemIf("offset", offset, offset != null);
pw.itemIf("fetch", fetch, fetch != null);
return pw;
}
}
-
-// End CassandraLimit.java
diff --git a/cassandra/src/main/java/org/apache/calcite/adapter/cassandra/CassandraMethod.java b/cassandra/src/main/java/org/apache/calcite/adapter/cassandra/CassandraMethod.java
index b2035e56242d..a19943e52f92 100644
--- a/cassandra/src/main/java/org/apache/calcite/adapter/cassandra/CassandraMethod.java
+++ b/cassandra/src/main/java/org/apache/calcite/adapter/cassandra/CassandraMethod.java
@@ -18,7 +18,7 @@
import org.apache.calcite.linq4j.tree.Types;
-import com.google.common.collect.ImmutableMap;
+import org.apache.kylin.guava30.shaded.common.collect.ImmutableMap;
import java.lang.reflect.Method;
import java.util.List;
@@ -30,6 +30,7 @@ public enum CassandraMethod {
CASSANDRA_QUERYABLE_QUERY(CassandraTable.CassandraQueryable.class, "query",
List.class, List.class, List.class, List.class, Integer.class, Integer.class);
+ @SuppressWarnings("ImmutableEnumChecker")
public final Method method;
public static final ImmutableMap MAP;
@@ -43,9 +44,7 @@ public enum CassandraMethod {
MAP = builder.build();
}
- CassandraMethod(Class clazz, String methodName, Class... argumentTypes) {
+ CassandraMethod(Class> clazz, String methodName, Class>... argumentTypes) {
this.method = Types.lookupMethod(clazz, methodName, argumentTypes);
}
}
-
-// End CassandraMethod.java
diff --git a/cassandra/src/main/java/org/apache/calcite/adapter/cassandra/CassandraProject.java b/cassandra/src/main/java/org/apache/calcite/adapter/cassandra/CassandraProject.java
index 5e55e461ffb7..e801590bc07f 100644
--- a/cassandra/src/main/java/org/apache/calcite/adapter/cassandra/CassandraProject.java
+++ b/cassandra/src/main/java/org/apache/calcite/adapter/cassandra/CassandraProject.java
@@ -16,7 +16,6 @@
*/
package org.apache.calcite.adapter.cassandra;
-import org.apache.calcite.adapter.java.JavaTypeFactory;
import org.apache.calcite.plan.RelOptCluster;
import org.apache.calcite.plan.RelOptCost;
import org.apache.calcite.plan.RelOptPlanner;
@@ -28,6 +27,10 @@
import org.apache.calcite.rex.RexNode;
import org.apache.calcite.util.Pair;
+import org.apache.kylin.guava30.shaded.common.collect.ImmutableList;
+
+import org.checkerframework.checker.nullness.qual.Nullable;
+
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
@@ -39,7 +42,7 @@
public class CassandraProject extends Project implements CassandraRel {
public CassandraProject(RelOptCluster cluster, RelTraitSet traitSet,
RelNode input, List extends RexNode> projects, RelDataType rowType) {
- super(cluster, traitSet, input, projects, rowType);
+ super(cluster, traitSet, ImmutableList.of(), input, projects, rowType);
assert getConvention() == CassandraRel.CONVENTION;
assert getConvention() == input.getConvention();
}
@@ -50,19 +53,19 @@ public CassandraProject(RelOptCluster cluster, RelTraitSet traitSet,
rowType);
}
- @Override public RelOptCost computeSelfCost(RelOptPlanner planner,
+ @Override public @Nullable RelOptCost computeSelfCost(RelOptPlanner planner,
RelMetadataQuery mq) {
return super.computeSelfCost(planner, mq).multiplyBy(0.1);
}
- public void implement(Implementor implementor) {
+ @Override public void implement(Implementor implementor) {
implementor.visitChild(0, getInput());
final CassandraRules.RexToCassandraTranslator translator =
new CassandraRules.RexToCassandraTranslator(
- (JavaTypeFactory) getCluster().getTypeFactory(),
CassandraRules.cassandraFieldNames(getInput().getRowType()));
- final Map fields = new LinkedHashMap();
+ final Map fields = new LinkedHashMap<>();
for (Pair pair : getNamedProjects()) {
+ assert pair.left != null;
final String name = pair.right;
final String originalName = pair.left.accept(translator);
fields.put(originalName, name);
@@ -70,5 +73,3 @@ public void implement(Implementor implementor) {
implementor.add(fields, null);
}
}
-
-// End CassandraProject.java
diff --git a/cassandra/src/main/java/org/apache/calcite/adapter/cassandra/CassandraRel.java b/cassandra/src/main/java/org/apache/calcite/adapter/cassandra/CassandraRel.java
index b74919ded647..adb20bac9b6a 100644
--- a/cassandra/src/main/java/org/apache/calcite/adapter/cassandra/CassandraRel.java
+++ b/cassandra/src/main/java/org/apache/calcite/adapter/cassandra/CassandraRel.java
@@ -20,6 +20,8 @@
import org.apache.calcite.plan.RelOptTable;
import org.apache.calcite.rel.RelNode;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
import java.util.ArrayList;
import java.util.LinkedHashMap;
import java.util.List;
@@ -37,21 +39,21 @@ public interface CassandraRel extends RelNode {
/** Callback for the implementation process that converts a tree of
* {@link CassandraRel} nodes into a CQL query. */
class Implementor {
- final Map selectFields = new LinkedHashMap();
- final List whereClause = new ArrayList();
+ final Map selectFields = new LinkedHashMap<>();
+ final List whereClause = new ArrayList<>();
int offset = 0;
int fetch = -1;
- final List order = new ArrayList();
+ final List order = new ArrayList<>();
- RelOptTable table;
- CassandraTable cassandraTable;
+ @Nullable RelOptTable table;
+ @Nullable CassandraTable cassandraTable;
/** Adds newly projected fields and restricted predicates.
*
* @param fields New fields to be projected from a query
* @param predicates New predicates to be applied to the query
*/
- public void add(Map fields, List predicates) {
+ public void add(@Nullable Map fields, @Nullable List predicates) {
if (fields != null) {
selectFields.putAll(fields);
}
@@ -70,5 +72,3 @@ public void visitChild(int ordinal, RelNode input) {
}
}
}
-
-// End CassandraRel.java
diff --git a/cassandra/src/main/java/org/apache/calcite/adapter/cassandra/CassandraRules.java b/cassandra/src/main/java/org/apache/calcite/adapter/cassandra/CassandraRules.java
index 095df16a7058..ce38ed3d18d2 100644
--- a/cassandra/src/main/java/org/apache/calcite/adapter/cassandra/CassandraRules.java
+++ b/cassandra/src/main/java/org/apache/calcite/adapter/cassandra/CassandraRules.java
@@ -16,13 +16,13 @@
*/
package org.apache.calcite.adapter.cassandra;
+import org.apache.calcite.adapter.enumerable.EnumerableConvention;
import org.apache.calcite.adapter.enumerable.EnumerableLimit;
-import org.apache.calcite.adapter.java.JavaTypeFactory;
import org.apache.calcite.plan.Convention;
import org.apache.calcite.plan.RelOptRule;
import org.apache.calcite.plan.RelOptRuleCall;
-import org.apache.calcite.plan.RelOptRuleOperand;
import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.plan.RelRule;
import org.apache.calcite.plan.RelTraitSet;
import org.apache.calcite.rel.RelCollation;
import org.apache.calcite.rel.RelCollations;
@@ -37,13 +37,11 @@
import org.apache.calcite.rex.RexInputRef;
import org.apache.calcite.rex.RexNode;
import org.apache.calcite.rex.RexVisitorImpl;
-import org.apache.calcite.runtime.PredicateImpl;
import org.apache.calcite.sql.SqlKind;
import org.apache.calcite.sql.validate.SqlValidatorUtil;
-import org.apache.calcite.util.Pair;
-import com.google.common.base.Predicate;
-import com.google.common.base.Predicates;
+import org.checkerframework.checker.nullness.qual.Nullable;
+import org.immutables.value.Value;
import java.util.HashSet;
import java.util.List;
@@ -55,13 +53,30 @@
* calling convention.
*/
public class CassandraRules {
+
private CassandraRules() {}
- public static final RelOptRule[] RULES = {
- CassandraFilterRule.INSTANCE,
- CassandraProjectRule.INSTANCE,
- CassandraSortRule.INSTANCE,
- CassandraLimitRule.INSTANCE
+ public static final CassandraFilterRule FILTER =
+ CassandraFilterRule.Config.DEFAULT.toRule();
+ public static final CassandraProjectRule PROJECT =
+ CassandraProjectRule.DEFAULT_CONFIG.toRule(CassandraProjectRule.class);
+ public static final CassandraSortRule SORT =
+ CassandraSortRule.Config.DEFAULT.toRule();
+ public static final CassandraLimitRule LIMIT =
+ CassandraLimitRule.Config.DEFAULT.toRule();
+
+ /** Rule to convert a relational expression from
+ * {@link CassandraRel#CONVENTION} to {@link EnumerableConvention}. */
+ public static final CassandraToEnumerableConverterRule TO_ENUMERABLE =
+ CassandraToEnumerableConverterRule.DEFAULT_CONFIG
+ .toRule(CassandraToEnumerableConverterRule.class);
+
+ @SuppressWarnings("MutablePublicArray")
+ protected static final RelOptRule[] RULES = {
+ FILTER,
+ PROJECT,
+ SORT,
+ LIMIT
};
static List cassandraFieldNames(final RelDataType rowType) {
@@ -72,13 +87,11 @@ static List cassandraFieldNames(final RelDataType rowType) {
/** Translator from {@link RexNode} to strings in Cassandra's expression
* language. */
static class RexToCassandraTranslator extends RexVisitorImpl {
- private final JavaTypeFactory typeFactory;
private final List inFields;
- protected RexToCassandraTranslator(JavaTypeFactory typeFactory,
+ protected RexToCassandraTranslator(
List inFields) {
super(true);
- this.typeFactory = typeFactory;
this.inFields = inFields;
}
@@ -90,42 +103,22 @@ protected RexToCassandraTranslator(JavaTypeFactory typeFactory,
/** Base class for planner rules that convert a relational expression to
* Cassandra calling convention. */
abstract static class CassandraConverterRule extends ConverterRule {
- protected final Convention out;
-
- public CassandraConverterRule(
- Class extends RelNode> clazz,
- String description) {
- this(clazz, Predicates.alwaysTrue(), description);
- }
-
- public CassandraConverterRule(
- Class clazz,
- Predicate super R> predicate,
- String description) {
- super(clazz, predicate, Convention.NONE, CassandraRel.CONVENTION, description);
- this.out = CassandraRel.CONVENTION;
+ CassandraConverterRule(Config config) {
+ super(config);
}
}
/**
* Rule to convert a {@link org.apache.calcite.rel.logical.LogicalFilter} to a
* {@link CassandraFilter}.
+ *
+ * @see #FILTER
*/
- private static class CassandraFilterRule extends RelOptRule {
- private static final Predicate PREDICATE =
- new PredicateImpl() {
- public boolean test(LogicalFilter input) {
- // TODO: Check for an equality predicate on the partition key
- // Right now this just checks if we have a single top-level AND
- return RelOptUtil.disjunctions(input.getCondition()).size() == 1;
- }
- };
-
- private static final CassandraFilterRule INSTANCE = new CassandraFilterRule();
-
- private CassandraFilterRule() {
- super(operand(LogicalFilter.class, operand(CassandraTableScan.class, none())),
- "CassandraFilterRule");
+ public static class CassandraFilterRule
+ extends RelRule {
+ /** Creates a CassandraFilterRule. */
+ protected CassandraFilterRule(CassandraFilterRuleConfig config) {
+ super(config);
}
@Override public boolean matches(RelOptRuleCall call) {
@@ -135,8 +128,11 @@ private CassandraFilterRule() {
// Get field names from the scan operation
CassandraTableScan scan = call.rel(1);
- Pair, List> keyFields = scan.cassandraTable.getKeyFields();
- Set partitionKeys = new HashSet(keyFields.left);
+
+ List partitionKeys = scan.cassandraTable.getPartitionKeys();
+ List clusteringKeys = scan.cassandraTable.getClusteringKeys();
+ Set partitionKeysSet = new HashSet<>(scan.cassandraTable.getPartitionKeys());
+
List fieldNames = CassandraRules.cassandraFieldNames(filter.getInput().getRowType());
List disjunctions = RelOptUtil.disjunctions(condition);
@@ -146,14 +142,14 @@ private CassandraFilterRule() {
// Check that all conjunctions are primary key equalities
condition = disjunctions.get(0);
for (RexNode predicate : RelOptUtil.conjunctions(condition)) {
- if (!isEqualityOnKey(predicate, fieldNames, partitionKeys, keyFields.right)) {
+ if (!isEqualityOnKey(predicate, fieldNames, partitionKeysSet, clusteringKeys)) {
return false;
}
}
}
- // Either all of the partition keys must be specified or none
- return partitionKeys.size() == keyFields.left.size() || partitionKeys.size() == 0;
+ // Either all the partition keys must be specified or none
+ return partitionKeysSet.size() == partitionKeys.size() || partitionKeysSet.isEmpty();
}
/** Check if the node is a supported predicate (primary key equality).
@@ -164,7 +160,7 @@ private CassandraFilterRule() {
* @param clusteringKeys Names of primary key columns
* @return True if the node represents an equality predicate on a primary key
*/
- private boolean isEqualityOnKey(RexNode node, List fieldNames,
+ private static boolean isEqualityOnKey(RexNode node, List fieldNames,
Set partitionKeys, List clusteringKeys) {
if (node.getKind() != SqlKind.EQUALS) {
return false;
@@ -175,7 +171,7 @@ private boolean isEqualityOnKey(RexNode node, List fieldNames,
final RexNode right = call.operands.get(1);
String key = compareFieldWithLiteral(left, right, fieldNames);
if (key == null) {
- key = compareFieldWithLiteral(right, left, fieldNames);
+ key = compareFieldWithLiteral(left, right, fieldNames);
}
if (key != null) {
return partitionKeys.remove(key) || clusteringKeys.contains(key);
@@ -191,23 +187,22 @@ private boolean isEqualityOnKey(RexNode node, List fieldNames,
* @param fieldNames Names of all columns in the table
* @return The field being compared or null if there is no key equality
*/
- private String compareFieldWithLiteral(RexNode left, RexNode right, List fieldNames) {
+ private static @Nullable String compareFieldWithLiteral(
+ RexNode left, RexNode right, List fieldNames) {
// FIXME Ignore casts for new and assume they aren't really necessary
if (left.isA(SqlKind.CAST)) {
left = ((RexCall) left).getOperands().get(0);
}
if (left.isA(SqlKind.INPUT_REF) && right.isA(SqlKind.LITERAL)) {
- final RexInputRef left1 = (RexInputRef) left;
- String name = fieldNames.get(left1.getIndex());
- return name;
+ RexInputRef left1 = (RexInputRef) left;
+ return fieldNames.get(left1.getIndex());
} else {
return null;
}
}
- /** @see org.apache.calcite.rel.convert.ConverterRule */
- public void onMatch(RelOptRuleCall call) {
+ @Override public void onMatch(RelOptRuleCall call) {
LogicalFilter filter = call.rel(0);
CassandraTableScan scan = call.rel(1);
if (filter.getTraitSet().contains(Convention.NONE)) {
@@ -218,29 +213,57 @@ public void onMatch(RelOptRuleCall call) {
}
}
- public RelNode convert(LogicalFilter filter, CassandraTableScan scan) {
+ @Nullable RelNode convert(LogicalFilter filter, CassandraTableScan scan) {
final RelTraitSet traitSet = filter.getTraitSet().replace(CassandraRel.CONVENTION);
- final Pair, List> keyFields = scan.cassandraTable.getKeyFields();
+ final List partitionKeys = scan.cassandraTable.getPartitionKeys();
+ final List clusteringKeys = scan.cassandraTable.getClusteringKeys();
+
return new CassandraFilter(
filter.getCluster(),
traitSet,
convert(filter.getInput(), CassandraRel.CONVENTION),
filter.getCondition(),
- keyFields.left,
- keyFields.right,
+ partitionKeys,
+ clusteringKeys,
scan.cassandraTable.getClusteringOrder());
}
+
+ /** Deprecated in favor of {@link CassandraFilterRuleConfig}. **/
+ @Deprecated
+ public interface Config extends CassandraFilterRuleConfig { }
+
+ /** Rule configuration. */
+ @Value.Immutable
+ public interface CassandraFilterRuleConfig extends RelRule.Config {
+ CassandraFilterRuleConfig DEFAULT = ImmutableCassandraFilterRuleConfig.builder()
+ .withOperandSupplier(b0 ->
+ b0.operand(LogicalFilter.class)
+ .oneInput(b1 -> b1.operand(CassandraTableScan.class)
+ .noInputs()))
+ .build();
+
+ @Override default CassandraFilterRule toRule() {
+ return new CassandraFilterRule(this);
+ }
+ }
+
}
/**
* Rule to convert a {@link org.apache.calcite.rel.logical.LogicalProject}
* to a {@link CassandraProject}.
+ *
+ * @see #PROJECT
*/
- private static class CassandraProjectRule extends CassandraConverterRule {
- private static final CassandraProjectRule INSTANCE = new CassandraProjectRule();
-
- private CassandraProjectRule() {
- super(LogicalProject.class, "CassandraProjectRule");
+ public static class CassandraProjectRule extends CassandraConverterRule {
+ /** Default configuration. */
+ private static final Config DEFAULT_CONFIG = Config.INSTANCE
+ .withConversion(LogicalProject.class, Convention.NONE,
+ CassandraRel.CONVENTION, "CassandraProjectRule")
+ .withRuleFactory(CassandraProjectRule::new);
+
+ protected CassandraProjectRule(Config config) {
+ super(config);
}
@Override public boolean matches(RelOptRuleCall call) {
@@ -254,7 +277,7 @@ private CassandraProjectRule() {
return true;
}
- public RelNode convert(RelNode rel) {
+ @Override public RelNode convert(RelNode rel) {
final LogicalProject project = (LogicalProject) rel;
final RelTraitSet traitSet = project.getTraitSet().replace(out);
return new CassandraProject(project.getCluster(), traitSet,
@@ -266,30 +289,14 @@ public RelNode convert(RelNode rel) {
/**
* Rule to convert a {@link org.apache.calcite.rel.core.Sort} to a
* {@link CassandraSort}.
+ *
+ * @see #SORT
*/
- private static class CassandraSortRule extends RelOptRule {
- private static final Predicate SORT_PREDICATE =
- new PredicateImpl() {
- public boolean test(Sort input) {
- // Limits are handled by CassandraLimit
- return input.offset == null && input.fetch == null;
- }
- };
- private static final Predicate FILTER_PREDICATE =
- new PredicateImpl() {
- public boolean test(CassandraFilter input) {
- // We can only use implicit sorting within a single partition
- return input.isSinglePartition();
- }
- };
- private static final RelOptRuleOperand CASSANDRA_OP =
- operand(CassandraToEnumerableConverter.class,
- operand(CassandraFilter.class, null, FILTER_PREDICATE, any()));
-
- private static final CassandraSortRule INSTANCE = new CassandraSortRule();
-
- private CassandraSortRule() {
- super(operand(Sort.class, null, SORT_PREDICATE, CASSANDRA_OP), "CassandraSortRule");
+ public static class CassandraSortRule
+ extends RelRule {
+ /** Creates a CassandraSortRule. */
+ protected CassandraSortRule(CassandraSortRuleConfig config) {
+ super(config);
}
public RelNode convert(Sort sort, CassandraFilter filter) {
@@ -301,7 +308,7 @@ public RelNode convert(Sort sort, CassandraFilter filter) {
sort.getCollation());
}
- public boolean matches(RelOptRuleCall call) {
+ @Override public boolean matches(RelOptRuleCall call) {
final Sort sort = call.rel(0);
final CassandraFilter filter = call.rel(2);
return collationsCompatible(sort.getCollation(), filter.getImplicitCollation());
@@ -311,7 +318,7 @@ public boolean matches(RelOptRuleCall call) {
*
* @return True if it is possible to achieve this sort in Cassandra
*/
- private boolean collationsCompatible(RelCollation sortCollation,
+ private static boolean collationsCompatible(RelCollation sortCollation,
RelCollation implicitCollation) {
List sortFieldCollations = sortCollation.getFieldCollations();
List implicitFieldCollations = implicitCollation.getFieldCollations();
@@ -319,12 +326,12 @@ private boolean collationsCompatible(RelCollation sortCollation,
if (sortFieldCollations.size() > implicitFieldCollations.size()) {
return false;
}
- if (sortFieldCollations.size() == 0) {
+ if (sortFieldCollations.isEmpty()) {
return true;
}
// Check if we need to reverse the order of the implicit collation
- boolean reversed = reverseDirection(sortFieldCollations.get(0).getDirection())
+ boolean reversed = sortFieldCollations.get(0).getDirection().reverse().lax()
== implicitFieldCollations.get(0).getDirection();
for (int i = 0; i < sortFieldCollations.size(); i++) {
@@ -342,7 +349,7 @@ private boolean collationsCompatible(RelCollation sortCollation,
RelFieldCollation.Direction sortDirection = sorted.getDirection();
RelFieldCollation.Direction implicitDirection = implied.getDirection();
if ((!reversed && sortDirection != implicitDirection)
- || (reversed && reverseDirection(sortDirection) != implicitDirection)) {
+ || (reversed && sortDirection.reverse().lax() != implicitDirection)) {
return false;
}
}
@@ -350,44 +357,53 @@ private boolean collationsCompatible(RelCollation sortCollation,
return true;
}
- /** Find the reverse of a given collation direction.
- *
- * @return Reverse of the input direction
- */
- private RelFieldCollation.Direction reverseDirection(RelFieldCollation.Direction direction) {
- switch (direction) {
- case ASCENDING:
- case STRICTLY_ASCENDING:
- return RelFieldCollation.Direction.DESCENDING;
- case DESCENDING:
- case STRICTLY_DESCENDING:
- return RelFieldCollation.Direction.ASCENDING;
- default:
- return null;
- }
+ @Override public void onMatch(RelOptRuleCall call) {
+ Sort sort = call.rel(0);
+ CassandraFilter filter = call.rel(2);
+ call.transformTo(convert(sort, filter));
}
- /** @see org.apache.calcite.rel.convert.ConverterRule */
- public void onMatch(RelOptRuleCall call) {
- final Sort sort = call.rel(0);
- CassandraFilter filter = call.rel(2);
- final RelNode converted = convert(sort, filter);
- if (converted != null) {
- call.transformTo(converted);
+ /** Deprecated in favor of CassandraSortRuleConfig. **/
+ @Deprecated
+ public interface Config extends CassandraSortRuleConfig { }
+
+ /** Rule configuration. */
+ @Value.Immutable
+ public interface CassandraSortRuleConfig extends RelRule.Config {
+ CassandraSortRuleConfig DEFAULT = ImmutableCassandraSortRuleConfig.builder()
+ .withOperandSupplier(b0 ->
+ b0.operand(Sort.class)
+ // Limits are handled by CassandraLimit
+ .predicate(sort ->
+ sort.offset == null && sort.fetch == null)
+ .oneInput(b1 ->
+ b1.operand(CassandraToEnumerableConverter.class)
+ .oneInput(b2 ->
+ b2.operand(CassandraFilter.class)
+ // We can only use implicit sorting within a
+ // single partition
+ .predicate(
+ CassandraFilter::isSinglePartition)
+ .anyInputs()))).build();
+
+ @Override default CassandraSortRule toRule() {
+ return new CassandraSortRule(this);
}
}
}
/**
- * Rule to convert a {@link org.apache.calcite.adapter.enumerable.EnumerableLimit} to a
+ * Rule to convert a
+ * {@link org.apache.calcite.adapter.enumerable.EnumerableLimit} to a
* {@link CassandraLimit}.
+ *
+ * @see #LIMIT
*/
- private static class CassandraLimitRule extends RelOptRule {
- private static final CassandraLimitRule INSTANCE = new CassandraLimitRule();
-
- private CassandraLimitRule() {
- super(operand(EnumerableLimit.class, operand(CassandraToEnumerableConverter.class, any())),
- "CassandraLimitRule");
+ public static class CassandraLimitRule
+ extends RelRule {
+ /** Creates a CassandraLimitRule. */
+ protected CassandraLimitRule(CassandraLimitRuleConfig config) {
+ super(config);
}
public RelNode convert(EnumerableLimit limit) {
@@ -397,15 +413,28 @@ public RelNode convert(EnumerableLimit limit) {
convert(limit.getInput(), CassandraRel.CONVENTION), limit.offset, limit.fetch);
}
- /** @see org.apache.calcite.rel.convert.ConverterRule */
- public void onMatch(RelOptRuleCall call) {
- final EnumerableLimit limit = call.rel(0);
- final RelNode converted = convert(limit);
- if (converted != null) {
- call.transformTo(converted);
+ @Override public void onMatch(RelOptRuleCall call) {
+ EnumerableLimit limit = call.rel(0);
+ call.transformTo(convert(limit));
+ }
+
+ /** Deprecated in favor of CassandraLimitRuleConfig. **/
+ @Deprecated
+ public interface Config extends CassandraLimitRuleConfig { }
+
+ /** Rule configuration. */
+ @Value.Immutable
+ public interface CassandraLimitRuleConfig extends RelRule.Config {
+ CassandraLimitRuleConfig DEFAULT = ImmutableCassandraLimitRuleConfig.builder()
+ .withOperandSupplier(b0 ->
+ b0.operand(EnumerableLimit.class)
+ .oneInput(b1 ->
+ b1.operand(CassandraToEnumerableConverter.class)
+ .anyInputs())).build();
+
+ @Override default CassandraLimitRule toRule() {
+ return new CassandraLimitRule(this);
}
}
}
}
-
-// End CassandraRules.java
diff --git a/cassandra/src/main/java/org/apache/calcite/adapter/cassandra/CassandraSchema.java b/cassandra/src/main/java/org/apache/calcite/adapter/cassandra/CassandraSchema.java
index b2f5a4e3fd75..0f7643f33fe0 100644
--- a/cassandra/src/main/java/org/apache/calcite/adapter/cassandra/CassandraSchema.java
+++ b/cassandra/src/main/java/org/apache/calcite/adapter/cassandra/CassandraSchema.java
@@ -19,7 +19,7 @@
import org.apache.calcite.avatica.util.Casing;
import org.apache.calcite.jdbc.CalciteSchema;
import org.apache.calcite.rel.RelFieldCollation;
-import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.type.RelDataType;
import org.apache.calcite.rel.type.RelDataTypeFactory;
import org.apache.calcite.rel.type.RelDataTypeImpl;
import org.apache.calcite.rel.type.RelDataTypeSystem;
@@ -29,104 +29,103 @@
import org.apache.calcite.schema.Table;
import org.apache.calcite.schema.impl.AbstractSchema;
import org.apache.calcite.schema.impl.MaterializedViewTable;
-import org.apache.calcite.sql.SqlDialect;
import org.apache.calcite.sql.SqlSelect;
import org.apache.calcite.sql.SqlWriter;
+import org.apache.calcite.sql.SqlWriterConfig;
import org.apache.calcite.sql.parser.SqlParseException;
import org.apache.calcite.sql.parser.SqlParser;
import org.apache.calcite.sql.pretty.SqlPrettyWriter;
import org.apache.calcite.sql.type.SqlTypeFactoryImpl;
import org.apache.calcite.sql.type.SqlTypeName;
import org.apache.calcite.util.Pair;
-import org.apache.calcite.util.Util;
import org.apache.calcite.util.trace.CalciteTrace;
-import com.datastax.driver.core.AbstractTableMetadata;
-import com.datastax.driver.core.Cluster;
-import com.datastax.driver.core.ClusteringOrder;
-import com.datastax.driver.core.ColumnMetadata;
-import com.datastax.driver.core.DataType;
-import com.datastax.driver.core.KeyspaceMetadata;
-import com.datastax.driver.core.MaterializedViewMetadata;
-import com.datastax.driver.core.Session;
-import com.datastax.driver.core.TableMetadata;
-import com.google.common.base.Function;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableMap;
+import org.apache.kylin.guava30.shaded.common.collect.ImmutableMap;
+
+import com.datastax.oss.driver.api.core.CqlIdentifier;
+import com.datastax.oss.driver.api.core.CqlSession;
+import com.datastax.oss.driver.api.core.cql.Row;
+import com.datastax.oss.driver.api.core.metadata.schema.ClusteringOrder;
+import com.datastax.oss.driver.api.core.metadata.schema.ColumnMetadata;
+import com.datastax.oss.driver.api.core.metadata.schema.KeyspaceMetadata;
+import com.datastax.oss.driver.api.core.metadata.schema.RelationMetadata;
+import com.datastax.oss.driver.api.core.metadata.schema.TableMetadata;
+import com.datastax.oss.driver.api.core.metadata.schema.ViewMetadata;
+import com.datastax.oss.driver.api.core.type.DataType;
+import com.datastax.oss.driver.api.core.type.ListType;
+import com.datastax.oss.driver.api.core.type.MapType;
+import com.datastax.oss.driver.api.core.type.SetType;
+import com.datastax.oss.driver.api.core.type.TupleType;
import org.slf4j.Logger;
-import java.io.PrintWriter;
-import java.io.StringWriter;
import java.util.ArrayList;
+import java.util.Collection;
import java.util.List;
import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
/**
- * Schema mapped onto a Cassandra column family
+ * Schema mapped onto a Cassandra column family.
*/
public class CassandraSchema extends AbstractSchema {
- final Session session;
+ final CqlSession session;
final String keyspace;
private final SchemaPlus parentSchema;
final String name;
final Hook.Closeable hook;
- protected static final Logger LOGGER = CalciteTrace.getPlannerTracer();
+ static final CqlToSqlTypeConversionRules CQL_TO_SQL_TYPE =
+ CqlToSqlTypeConversionRules.instance();
- /**
- * Creates a Cassandra schema.
- *
- * @param host Cassandra host, e.g. "localhost"
- * @param keyspace Cassandra keyspace name, e.g. "twissandra"
- */
- public CassandraSchema(String host, String keyspace, SchemaPlus parentSchema, String name) {
- this(host, keyspace, null, null, parentSchema, name);
- }
+ protected static final Logger LOGGER = CalciteTrace.getPlannerTracer();
/**
* Creates a Cassandra schema.
*
- * @param host Cassandra host, e.g. "localhost"
- * @param keyspace Cassandra keyspace name, e.g. "twissandra"
- * @param username Cassandra username
- * @param password Cassandra password
+ * @param session a Cassandra session
+ * @param parentSchema the parent schema
+ * @param name the schema name
*/
- public CassandraSchema(String host, String keyspace, String username, String password,
- SchemaPlus parentSchema, String name) {
+ public CassandraSchema(CqlSession session, SchemaPlus parentSchema, String name) {
super();
-
- this.keyspace = keyspace;
- try {
- Cluster cluster;
- if (username != null && password != null) {
- cluster = Cluster.builder().addContactPoint(host)
- .withCredentials(username, password).build();
- } else {
- cluster = Cluster.builder().addContactPoint(host).build();
- }
-
- this.session = cluster.connect(keyspace);
- } catch (Exception e) {
- throw new RuntimeException(e);
- }
+ this.session = session;
+ this.keyspace = session.getKeyspace()
+ .orElseThrow(() -> new RuntimeException("No keyspace for session " + session.getName()))
+ .asInternal();
this.parentSchema = parentSchema;
this.name = name;
+ this.hook = prepareHook();
+ }
- this.hook = Hook.TRIMMED.add(new Function() {
- public Void apply(RelNode node) {
- CassandraSchema.this.addMaterializedViews();
- return null;
- }
+ @SuppressWarnings("deprecation")
+ private Hook.Closeable prepareHook() {
+ // It adds a global hook, so it should probably be replaced with a thread-local hook
+ return Hook.TRIMMED.add(node -> {
+ CassandraSchema.this.addMaterializedViews();
});
}
RelProtoDataType getRelDataType(String columnFamily, boolean view) {
- List columns;
+ Map columns;
+ CqlIdentifier tableName = CqlIdentifier.fromInternal(columnFamily);
if (view) {
- columns = getKeyspace().getMaterializedView(columnFamily).getColumns();
+ Optional optionalViewMetadata = getKeyspace().getView(tableName);
+ if (optionalViewMetadata.isPresent()) {
+ columns = optionalViewMetadata.get().getColumns();
+ } else {
+ throw new IllegalStateException("Unknown view " + tableName + " in keyspace " + keyspace);
+ }
} else {
- columns = getKeyspace().getTable(columnFamily).getColumns();
+ Optional optionalTableMetadata = getKeyspace().getTable(tableName);
+ if (optionalTableMetadata.isPresent()) {
+ columns = optionalTableMetadata.get().getColumns();
+ } else {
+ throw new IllegalStateException("Unknown table " + tableName + " in keyspace " + keyspace);
+ }
}
// Temporary type factory, just for the duration of this method. Allowable
@@ -134,78 +133,92 @@ RelProtoDataType getRelDataType(String columnFamily, boolean view) {
// proto-type will be copied into a real type factory.
final RelDataTypeFactory typeFactory =
new SqlTypeFactoryImpl(RelDataTypeSystem.DEFAULT);
- final RelDataTypeFactory.FieldInfoBuilder fieldInfo = typeFactory.builder();
- for (ColumnMetadata column : columns) {
- final String columnName = column.getName();
- final DataType type = column.getType();
-
- // TODO: This mapping of types can be done much better
- SqlTypeName typeName = SqlTypeName.ANY;
- if (type == DataType.uuid() || type == DataType.timeuuid()) {
- // We currently rely on this in CassandraFilter to detect UUID columns.
- // That is, these fixed length literals should be unquoted in CQL.
- typeName = SqlTypeName.CHAR;
- } else if (type == DataType.ascii() || type == DataType.text()
- || type == DataType.varchar()) {
- typeName = SqlTypeName.VARCHAR;
- } else if (type == DataType.cint() || type == DataType.varint()) {
- typeName = SqlTypeName.INTEGER;
- } else if (type == DataType.bigint()) {
- typeName = SqlTypeName.BIGINT;
- } else if (type == DataType.cdouble() || type == DataType.cfloat()
- || type == DataType.decimal()) {
- typeName = SqlTypeName.DOUBLE;
+ final RelDataTypeFactory.Builder fieldInfo = typeFactory.builder();
+ for (ColumnMetadata column : columns.values()) {
+ final DataType dataType = column.getType();
+ final String columnName = column.getName().asInternal();
+
+ if (dataType instanceof ListType) {
+ SqlTypeName arrayInnerType = CQL_TO_SQL_TYPE.lookup(
+ ((ListType) dataType).getElementType());
+
+ fieldInfo.add(columnName,
+ typeFactory.createArrayType(
+ typeFactory.createSqlType(arrayInnerType), -1))
+ .nullable(true);
+ } else if (dataType instanceof SetType) {
+ SqlTypeName multiSetInnerType = CQL_TO_SQL_TYPE.lookup(
+ ((SetType) dataType).getElementType());
+
+ fieldInfo.add(columnName,
+ typeFactory.createMultisetType(
+ typeFactory.createSqlType(multiSetInnerType), -1)
+ ).nullable(true);
+ } else if (dataType instanceof MapType) {
+ MapType columnType = (MapType) dataType;
+ SqlTypeName keyType = CQL_TO_SQL_TYPE.lookup(columnType.getKeyType());
+ SqlTypeName valueType = CQL_TO_SQL_TYPE.lookup(columnType.getValueType());
+
+ fieldInfo.add(columnName,
+ typeFactory.createMapType(
+ typeFactory.createSqlType(keyType),
+ typeFactory.createSqlType(valueType))
+ ).nullable(true);
+ } else if (dataType instanceof TupleType) {
+ List typeArgs = ((TupleType) dataType).getComponentTypes();
+ List> typesList =
+ IntStream.range(0, typeArgs.size())
+ .mapToObj(
+ i -> new Pair<>(
+ Integer.toString(i + 1), // 1 indexed (as ARRAY)
+ typeFactory.createSqlType(
+ CQL_TO_SQL_TYPE.lookup(typeArgs.get(i)))))
+ .collect(Collectors.toList());
+
+ fieldInfo.add(columnName,
+ typeFactory.createStructType(typesList))
+ .nullable(true);
+ } else {
+ SqlTypeName typeName = CQL_TO_SQL_TYPE.lookup(dataType);
+ fieldInfo.add(columnName, typeName).nullable(true);
}
-
- fieldInfo.add(columnName, typeFactory.createSqlType(typeName)).nullable(true);
}
return RelDataTypeImpl.proto(fieldInfo.build());
}
- /**
- * Get all primary key columns from the underlying CQL table
+ /** Returns the partition key columns from the underlying CQL table.
*
- * @return A list of field names that are part of the partition and clustering keys
+ * @return A list of field names that are part of the partition keys
*/
- Pair, List> getKeyFields(String columnFamily, boolean view) {
- AbstractTableMetadata table;
- if (view) {
- table = getKeyspace().getMaterializedView(columnFamily);
- } else {
- table = getKeyspace().getTable(columnFamily);
- }
-
- List partitionKey = table.getPartitionKey();
- List pKeyFields = new ArrayList();
- for (ColumnMetadata column : partitionKey) {
- pKeyFields.add(column.getName());
- }
-
- List clusteringKey = table.getClusteringColumns();
- List cKeyFields = new ArrayList();
- for (ColumnMetadata column : clusteringKey) {
- cKeyFields.add(column.getName());
- }
+ List getPartitionKeys(String columnFamily, boolean isView) {
+ RelationMetadata table = getRelationMetadata(columnFamily, isView);
+ return table.getPartitionKey().stream()
+ .map(ColumnMetadata::getName)
+ .map(CqlIdentifier::asInternal)
+ .collect(Collectors.toList());
+ }
- return Pair.of((List) ImmutableList.copyOf(pKeyFields),
- (List) ImmutableList.copyOf(cKeyFields));
+ /** Returns the clustering keys from the underlying CQL table.
+ *
+ * @return A list of field names that are part of the clustering keys
+ */
+ List getClusteringKeys(String columnFamily, boolean isView) {
+ RelationMetadata table = getRelationMetadata(columnFamily, isView);
+ return table.getClusteringColumns().keySet().stream()
+ .map(ColumnMetadata::getName)
+ .map(CqlIdentifier::asInternal)
+ .collect(Collectors.toList());
}
/** Get the collation of all clustering key columns.
*
* @return A RelCollations representing the collation of all clustering keys
*/
- public List getClusteringOrder(String columnFamily, boolean view) {
- AbstractTableMetadata table;
- if (view) {
- table = getKeyspace().getMaterializedView(columnFamily);
- } else {
- table = getKeyspace().getTable(columnFamily);
- }
-
- List clusteringOrder = table.getClusteringOrder();
- List keyCollations = new ArrayList();
+ public List getClusteringOrder(String columnFamily, boolean isView) {
+ RelationMetadata table = getRelationMetadata(columnFamily, isView);
+ Collection clusteringOrder = table.getClusteringColumns().values();
+ List keyCollations = new ArrayList<>();
int i = 0;
for (ClusteringOrder order : clusteringOrder) {
@@ -226,71 +239,95 @@ public List getClusteringOrder(String columnFamily, boolean v
return keyCollations;
}
- /** Add all materialized views defined in the schema to this column family
- */
+ private RelationMetadata getRelationMetadata(String columnFamily, boolean isView) {
+ String tableName = CqlIdentifier.fromInternal(columnFamily).asCql(false);
+
+ if (isView) {
+ return getKeyspace().getView(tableName)
+ .orElseThrow(
+ () -> new RuntimeException(
+ "Unknown view " + columnFamily + " in keyspace " + keyspace));
+ }
+ return getKeyspace().getTable(tableName)
+ .orElseThrow(
+ () -> new RuntimeException(
+ "Unknown table " + columnFamily + " in keyspace " + keyspace));
+ }
+
+ /** Adds all materialized views defined in the schema to this column family. */
private void addMaterializedViews() {
- // Close the hook use to get us here
+ // Close the hook used to get us here
hook.close();
- for (MaterializedViewMetadata view : getKeyspace().getMaterializedViews()) {
- String tableName = view.getBaseTable().getName();
+ for (ViewMetadata view : getKeyspace().getViews().values()) {
+ String tableName = view.getBaseTable().asInternal();
StringBuilder queryBuilder = new StringBuilder("SELECT ");
// Add all the selected columns to the query
- List columnNames = new ArrayList();
- for (ColumnMetadata column : view.getColumns()) {
- columnNames.add("\"" + column.getName() + "\"");
- }
- queryBuilder.append(Util.toString(columnNames, "", ", ", ""));
+ String columnsList = view.getColumns().values().stream()
+ .map(c -> c.getName().asInternal())
+ .collect(Collectors.joining(", "));
+ queryBuilder.append(columnsList);
- queryBuilder.append(" FROM \"" + tableName + "\"");
+ queryBuilder.append(" FROM ")
+ .append(tableName);
// Get the where clause from the system schema
String whereQuery = "SELECT where_clause from system_schema.views "
- + "WHERE keyspace_name='" + keyspace + "' AND view_name='" + view.getName() + "'";
- queryBuilder.append(" WHERE " + session.execute(whereQuery).one().getString(0));
+ + "WHERE keyspace_name='" + keyspace + "' AND view_name='"
+ + view.getName().asInternal() + "'";
+
+ Row whereClauseRow = Objects.requireNonNull(session.execute(whereQuery).one());
+
+ queryBuilder.append(" WHERE ")
+ .append(whereClauseRow.getString(0));
// Parse and unparse the view query to get properly quoted field names
String query = queryBuilder.toString();
- SqlParser.ConfigBuilder configBuilder = SqlParser.configBuilder();
- configBuilder.setUnquotedCasing(Casing.UNCHANGED);
+ SqlParser.Config parserConfig = SqlParser.config()
+ .withUnquotedCasing(Casing.UNCHANGED);
SqlSelect parsedQuery;
try {
- parsedQuery = (SqlSelect) SqlParser.create(query, configBuilder.build()).parseQuery();
+ parsedQuery = (SqlSelect) SqlParser.create(query, parserConfig).parseQuery();
} catch (SqlParseException e) {
LOGGER.warn("Could not parse query {} for CQL view {}.{}",
- query, keyspace, view.getName());
+ query, keyspace, view.getName().asInternal());
continue;
}
- StringWriter stringWriter = new StringWriter(query.length());
- PrintWriter printWriter = new PrintWriter(stringWriter);
- SqlWriter writer = new SqlPrettyWriter(SqlDialect.CALCITE, true, printWriter);
+ final StringBuilder buf = new StringBuilder(query.length());
+ final SqlWriterConfig config = SqlPrettyWriter.config()
+ .withAlwaysUseParentheses(true);
+ final SqlWriter writer = new SqlPrettyWriter(config, buf);
parsedQuery.unparse(writer, 0, 0);
- query = stringWriter.toString();
+ query = buf.toString();
// Add the view for this query
String viewName = "$" + getTableNames().size();
SchemaPlus schema = parentSchema.getSubSchema(name);
+ if (schema == null) {
+ throw new IllegalStateException("Cannot find schema " + name
+ + " in parent schema " + parentSchema.getName());
+ }
CalciteSchema calciteSchema = CalciteSchema.from(schema);
List viewPath = calciteSchema.path(viewName);
schema.add(viewName,
MaterializedViewTable.create(calciteSchema, query,
- null, viewPath, view.getName(), true));
+ null, viewPath, view.getName().asInternal(), true));
}
}
@Override protected Map getTableMap() {
final ImmutableMap.Builder builder = ImmutableMap.builder();
- for (TableMetadata table : getKeyspace().getTables()) {
- String tableName = table.getName();
+ for (TableMetadata table : getKeyspace().getTables().values()) {
+ String tableName = table.getName().asInternal();
builder.put(tableName, new CassandraTable(this, tableName));
- for (MaterializedViewMetadata view : table.getViews()) {
- String viewName = view.getName();
+ for (ViewMetadata view : getKeyspace().getViewsOnTable(table.getName()).values()) {
+ String viewName = view.getName().asInternal();
builder.put(viewName, new CassandraTable(this, viewName, true));
}
}
@@ -298,8 +335,7 @@ private void addMaterializedViews() {
}
private KeyspaceMetadata getKeyspace() {
- return session.getCluster().getMetadata().getKeyspace(keyspace);
+ return session.getMetadata().getKeyspace(keyspace).orElseThrow(
+ () -> new RuntimeException("Keyspace " + keyspace + " not found"));
}
}
-
-// End CassandraSchema.java
diff --git a/cassandra/src/main/java/org/apache/calcite/adapter/cassandra/CassandraSchemaFactory.java b/cassandra/src/main/java/org/apache/calcite/adapter/cassandra/CassandraSchemaFactory.java
index 136b81b01f31..f0db89a384dc 100644
--- a/cassandra/src/main/java/org/apache/calcite/adapter/cassandra/CassandraSchemaFactory.java
+++ b/cassandra/src/main/java/org/apache/calcite/adapter/cassandra/CassandraSchemaFactory.java
@@ -19,26 +19,92 @@
import org.apache.calcite.schema.Schema;
import org.apache.calcite.schema.SchemaFactory;
import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.util.trace.CalciteTrace;
+import org.apache.kylin.guava30.shaded.common.collect.ImmutableSet;
+
+import com.datastax.oss.driver.api.core.CqlSession;
+
+import org.slf4j.Logger;
+
+import java.net.InetSocketAddress;
import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.stream.Collectors;
/**
- * Factory that creates a {@link CassandraSchema}
+ * Factory that creates a {@link CassandraSchema}.
*/
@SuppressWarnings("UnusedDeclaration")
public class CassandraSchemaFactory implements SchemaFactory {
+
+ private static final int DEFAULT_CASSANDRA_PORT = 9042;
+ private static final Map, CqlSession> INFO_TO_SESSION =
+ new ConcurrentHashMap<>();
+ private static final Set SESSION_DEFINING_KEYS = ImmutableSet.of(
+ "host", "port", "keyspace", "username", "password");
+ protected static final Logger LOGGER = CalciteTrace.getPlannerTracer();
+
public CassandraSchemaFactory() {
+ super();
}
- public Schema create(SchemaPlus parentSchema, String name,
+ @Override public Schema create(SchemaPlus parentSchema, String name,
Map operand) {
- Map map = (Map) operand;
- String host = (String) map.get("host");
- String keyspace = (String) map.get("keyspace");
- String username = (String) map.get("username");
- String password = (String) map.get("password");
- return new CassandraSchema(host, keyspace, username, password, parentSchema, name);
+
+ final Map sessionMap = projectMapOverKeys(operand, SESSION_DEFINING_KEYS);
+
+ INFO_TO_SESSION.computeIfAbsent(sessionMap, m -> {
+ String host = (String) m.get("host");
+ String keyspace = (String) m.get("keyspace");
+ String username = (String) m.get("username");
+ String password = (String) m.get("password");
+ int port = getPort(m);
+
+ if (LOGGER.isDebugEnabled()) {
+ LOGGER.debug("Creating session for info {}", m);
+ }
+ try {
+ if (username != null && password != null) {
+ return CqlSession.builder()
+ .addContactPoint(new InetSocketAddress(host, port))
+ .withAuthCredentials(username, password)
+ .withKeyspace(keyspace)
+ .withLocalDatacenter("datacenter1")
+ .build();
+ } else {
+ return CqlSession.builder()
+ .addContactPoint(new InetSocketAddress(host, port))
+ .withKeyspace(keyspace)
+ .withLocalDatacenter("datacenter1")
+ .build();
+ }
+ } catch (Exception e) {
+ throw new RuntimeException(e);
+ }
+ });
+
+ return new CassandraSchema(INFO_TO_SESSION.get(sessionMap), parentSchema, name);
}
-}
-// End CassandraSchemaFactory.java
+ private static Map projectMapOverKeys(
+ Map map, Set keysToKeep) {
+ return map.entrySet().stream()
+ .filter(e -> keysToKeep.contains(e.getKey()))
+ .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+ }
+
+ private static int getPort(Map map) {
+ if (map.containsKey("port")) {
+ Object portObj = map.get("port");
+ if (portObj instanceof String) {
+ return Integer.parseInt((String) portObj);
+ } else {
+ return (int) portObj;
+ }
+ } else {
+ return DEFAULT_CASSANDRA_PORT;
+ }
+ }
+}
diff --git a/cassandra/src/main/java/org/apache/calcite/adapter/cassandra/CassandraSort.java b/cassandra/src/main/java/org/apache/calcite/adapter/cassandra/CassandraSort.java
index 8487815e8b33..04bd4781cc66 100644
--- a/cassandra/src/main/java/org/apache/calcite/adapter/cassandra/CassandraSort.java
+++ b/cassandra/src/main/java/org/apache/calcite/adapter/cassandra/CassandraSort.java
@@ -28,6 +28,8 @@
import org.apache.calcite.rel.type.RelDataTypeField;
import org.apache.calcite.rex.RexNode;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
import java.util.ArrayList;
import java.util.List;
@@ -44,7 +46,7 @@ public CassandraSort(RelOptCluster cluster, RelTraitSet traitSet,
assert getConvention() == child.getConvention();
}
- @Override public RelOptCost computeSelfCost(RelOptPlanner planner,
+ @Override public @Nullable RelOptCost computeSelfCost(RelOptPlanner planner,
RelMetadataQuery mq) {
RelOptCost cost = super.computeSelfCost(planner, mq);
if (!collation.getFieldCollations().isEmpty()) {
@@ -55,24 +57,28 @@ public CassandraSort(RelOptCluster cluster, RelTraitSet traitSet,
}
@Override public Sort copy(RelTraitSet traitSet, RelNode input,
- RelCollation newCollation, RexNode offset, RexNode fetch) {
+ RelCollation newCollation, @Nullable RexNode offset, @Nullable RexNode fetch) {
return new CassandraSort(getCluster(), traitSet, input, collation);
}
- public void implement(Implementor implementor) {
+ @Override public void implement(Implementor implementor) {
implementor.visitChild(0, getInput());
List sortCollations = collation.getFieldCollations();
- List fieldOrder = new ArrayList();
+ List fieldOrder = new ArrayList<>();
if (!sortCollations.isEmpty()) {
// Construct a series of order clauses from the desired collation
final List fields = getRowType().getFieldList();
for (RelFieldCollation fieldCollation : sortCollations) {
final String name =
fields.get(fieldCollation.getFieldIndex()).getName();
- String direction = "ASC";
- if (fieldCollation.getDirection().equals(RelFieldCollation.Direction.DESCENDING)) {
+ final String direction;
+ switch (fieldCollation.getDirection()) {
+ case DESCENDING:
direction = "DESC";
+ break;
+ default:
+ direction = "ASC";
}
fieldOrder.add(name + " " + direction);
}
@@ -81,5 +87,3 @@ public void implement(Implementor implementor) {
}
}
}
-
-// End CassandraSort.java
diff --git a/cassandra/src/main/java/org/apache/calcite/adapter/cassandra/CassandraTable.java b/cassandra/src/main/java/org/apache/calcite/adapter/cassandra/CassandraTable.java
index 25786fb40be6..f5e883abe8a7 100644
--- a/cassandra/src/main/java/org/apache/calcite/adapter/cassandra/CassandraTable.java
+++ b/cassandra/src/main/java/org/apache/calcite/adapter/cassandra/CassandraTable.java
@@ -36,70 +36,67 @@
import org.apache.calcite.schema.TranslatableTable;
import org.apache.calcite.schema.impl.AbstractTableQueryable;
import org.apache.calcite.sql.type.SqlTypeFactoryImpl;
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.apache.calcite.util.Pair;
import org.apache.calcite.util.Util;
-import com.datastax.driver.core.ResultSet;
-import com.datastax.driver.core.Session;
+import org.apache.kylin.guava30.shaded.common.collect.ImmutableList;
+
+import com.datastax.oss.driver.api.core.CqlSession;
+import com.datastax.oss.driver.api.core.cql.ResultSet;
+
+import org.checkerframework.checker.nullness.qual.Nullable;
-import java.util.Collections;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
+import java.util.Objects;
/**
- * Table based on a Cassandra column family
+ * Table based on a Cassandra column family.
*/
public class CassandraTable extends AbstractQueryableTable
implements TranslatableTable {
- RelProtoDataType protoRowType;
- Pair, List> keyFields;
+ final RelProtoDataType protoRowType;
+ List partitionKeys;
+ List clusteringKeys;
List clusteringOrder;
- private final CassandraSchema schema;
private final String columnFamily;
- private final boolean view;
- public CassandraTable(CassandraSchema schema, String columnFamily, boolean view) {
+ public CassandraTable(CassandraSchema schema, String columnFamily, boolean isView) {
super(Object[].class);
- this.schema = schema;
this.columnFamily = columnFamily;
- this.view = view;
+ this.protoRowType = schema.getRelDataType(columnFamily, isView);
+ this.partitionKeys = schema.getPartitionKeys(columnFamily, isView);
+ this.clusteringKeys = schema.getClusteringKeys(columnFamily, isView);
+ this.clusteringOrder = schema.getClusteringOrder(columnFamily, isView);
}
public CassandraTable(CassandraSchema schema, String columnFamily) {
this(schema, columnFamily, false);
}
- public String toString() {
+ @Override public String toString() {
return "CassandraTable {" + columnFamily + "}";
}
- public RelDataType getRowType(RelDataTypeFactory typeFactory) {
- if (protoRowType == null) {
- protoRowType = schema.getRelDataType(columnFamily, view);
- }
+ @Override public RelDataType getRowType(RelDataTypeFactory typeFactory) {
return protoRowType.apply(typeFactory);
}
- public Pair, List> getKeyFields() {
- if (keyFields == null) {
- keyFields = schema.getKeyFields(columnFamily, view);
- }
- return keyFields;
+ public List getPartitionKeys() {
+ return partitionKeys;
+ }
+
+ public List getClusteringKeys() {
+ return clusteringKeys;
}
public List getClusteringOrder() {
- if (clusteringOrder == null) {
- clusteringOrder = schema.getClusteringOrder(columnFamily, view);
- }
return clusteringOrder;
}
- public Enumerable query(final Session session) {
- return query(session, Collections.>emptyList(),
- Collections.>emptyList(),
- Collections.emptyList(), Collections.emptyList(), 0, -1);
+ public Enumerable query(final CqlSession session) {
+ return query(session, ImmutableList.of(), ImmutableList.of(),
+ ImmutableList.of(), ImmutableList.of(), 0, -1);
}
/** Executes a CQL query on the underlying table.
@@ -109,21 +106,20 @@ public Enumerable query(final Session session) {
* @param predicates A list of predicates which should be used in the query
* @return Enumerator of results
*/
- public Enumerable query(final Session session, List> fields,
+ public Enumerable query(final CqlSession session, List> fields,
final List> selectFields, List predicates,
List order, final Integer offset, final Integer fetch) {
// Build the type of the resulting row based on the provided fields
final RelDataTypeFactory typeFactory =
new SqlTypeFactoryImpl(RelDataTypeSystem.DEFAULT);
- final RelDataTypeFactory.FieldInfoBuilder fieldInfo = typeFactory.builder();
+ final RelDataTypeFactory.Builder fieldInfo = typeFactory.builder();
final RelDataType rowType = getRowType(typeFactory);
- Function1 addField = new Function1() {
- public Void apply(String fieldName) {
- SqlTypeName typeName = rowType.getField(fieldName, true, false).getType().getSqlTypeName();
- fieldInfo.add(fieldName, typeFactory.createSqlType(typeName)).nullable(true);
- return null;
- }
+ Function1 addField = fieldName -> {
+ RelDataType relDataType = Objects.requireNonNull(
+ rowType.getField(fieldName, true, false)).getType();
+ fieldInfo.add(fieldName, relDataType).nullable(true);
+ return null;
};
if (selectFields.isEmpty()) {
@@ -143,26 +139,24 @@ public Void apply(String fieldName) {
if (selectFields.isEmpty()) {
selectString = "*";
} else {
- selectString = Util.toString(new Iterable() {
- public Iterator iterator() {
- final Iterator> selectIterator =
- selectFields.iterator();
-
- return new Iterator() {
- @Override public boolean hasNext() {
- return selectIterator.hasNext();
- }
-
- @Override public String next() {
- Map.Entry entry = selectIterator.next();
- return entry.getKey() + " AS " + entry.getValue();
- }
-
- @Override public void remove() {
- throw new UnsupportedOperationException();
- }
- };
- }
+ selectString = Util.toString(() -> {
+ final Iterator