{ return SqlStdOperatorTable.IMMEDIATELY_PRECEDES; }
@@ -8016,6 +8113,7 @@ SqlPostfixOperator PostfixRowOperator() :
| < ARRAY_CONCAT_AGG: "ARRAY_CONCAT_AGG" >
| < ARRAY_MAX_CARDINALITY: "ARRAY_MAX_CARDINALITY" >
| < AS: "AS" >
+| < ASOF: "ASOF" >
| < ASC: "ASC" >
| < ASENSITIVE: "ASENSITIVE" >
| < ASSERTION: "ASSERTION" >
@@ -8316,10 +8414,12 @@ SqlPostfixOperator PostfixRowOperator() :
| < MATCH: "MATCH" >
| < MATCHED: "MATCHED" >
| < MATCHES: "MATCHES" >
+| < MATCH_CONDITION: "MATCH_CONDITION">
| < MATCH_NUMBER: "MATCH_NUMBER">
| < MATCH_RECOGNIZE: "MATCH_RECOGNIZE">
| < MAX: "MAX" >
| < MAXVALUE: "MAXVALUE" >
+| < MEASURE: "MEASURE" >
| < MEASURES: "MEASURES" >
| < MEMBER: "MEMBER" >
| < MERGE: "MERGE" > { beforeTableName(); }
@@ -8663,6 +8763,7 @@ SqlPostfixOperator PostfixRowOperator() :
| < UNPIVOT: "UNPIVOT" >
| < UNNAMED: "UNNAMED" >
| < UNNEST: "UNNEST" >
+| < UNSIGNED: "UNSIGNED" >
| < UPDATE: "UPDATE" > { beforeTableName(); }
| < UPPER: "UPPER" >
| < UPSERT: "UPSERT" >
@@ -8676,6 +8777,7 @@ SqlPostfixOperator PostfixRowOperator() :
| < UTF8: "UTF8" >
| < UTF16: "UTF16" >
| < UTF32: "UTF32" >
+| < UUID: "UUID" >
| < VALUE: "VALUE" >
| < VALUES: "VALUES" > { afterTableName(); }
| < VALUE_OF: "VALUE_OF" >
@@ -8905,6 +9007,8 @@ void NonReservedKeyWord2of3() :
| < DOUBLE_QUOTE: "\"" >
| < VERTICAL_BAR: "|" >
| < CARET: "^" >
+| < AMPERSAND: "&" >
+| < LEFTSHIFT: "<<" >
| < DOLLAR: "$" >
<#list (parser.binaryOperatorsTokens!default.parser.binaryOperatorsTokens) as operator>
| ${operator}
diff --git a/flink-table/flink-sql-parser/src/main/java/org/apache/calcite/sql/SqlCollectionTypeNameSpec.java b/flink-table/flink-sql-parser/src/main/java/org/apache/calcite/sql/SqlCollectionTypeNameSpec.java
new file mode 100644
index 0000000000000..3ca5081b410b1
--- /dev/null
+++ b/flink-table/flink-sql-parser/src/main/java/org/apache/calcite/sql/SqlCollectionTypeNameSpec.java
@@ -0,0 +1,155 @@
+/*
+ * 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;
+
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.sql.validate.SqlValidator;
+import org.apache.calcite.util.Litmus;
+import org.apache.calcite.util.Util;
+
+import java.util.Objects;
+
+import static java.util.Objects.requireNonNull;
+
+/**
+ * A sql type name specification of collection type.
+ *
+ * The grammar definition in SQL-2011 IWD 9075-2:201?(E) 6.1 <collection type> is as
+ * following:
+ *
+ *
+ *
+ *
+ * <collection type> ::=
+ * <array type>
+ * | <multiset type>
+ *
+ * <array type> ::=
+ * <data type> ARRAY
+ * [ <left bracket or trigraph>
+ * <maximum cardinality>
+ * <right bracket or trigraph> ]
+ *
+ * <maximum cardinality> ::=
+ * <unsigned integer>
+ *
+ * <multiset type> ::=
+ * <data type> MULTISET
+ *
+ *
+ *
+ *
+ * This class is intended to describe SQL collection type. It can describe either simple
+ * collection type like "int array" or nested collection type like "int array array" or "int array
+ * multiset". For nested collection type, the element type name of this {@code
+ * SqlCollectionTypeNameSpec} is also a {@code SqlCollectionTypeNameSpec}.
+ */
+public class SqlCollectionTypeNameSpec extends SqlTypeNameSpec {
+ private final SqlTypeNameSpec elementTypeName;
+ private final boolean elementTypeNullable;
+ private final SqlTypeName collectionTypeName;
+
+ /**
+ * Creates a {@code SqlCollectionTypeNameSpec}.
+ *
+ * @param elementTypeName Type of the collection element
+ * @param elementTypeNullable Type of the collection element is nullable
+ * @param collectionTypeName Collection type name
+ * @param pos Parser position, must not be null
+ */
+ public SqlCollectionTypeNameSpec(
+ SqlTypeNameSpec elementTypeName,
+ boolean elementTypeNullable,
+ SqlTypeName collectionTypeName,
+ SqlParserPos pos) {
+ super(new SqlIdentifier(collectionTypeName.name(), pos), pos);
+ this.elementTypeName = requireNonNull(elementTypeName, "elementTypeName");
+ this.elementTypeNullable = elementTypeNullable;
+ this.collectionTypeName = requireNonNull(collectionTypeName, "collectionTypeName");
+ }
+
+ /**
+ * Creates a {@code SqlCollectionTypeNameSpec}.
+ *
+ * @param elementTypeName Type of the collection element
+ * @param collectionTypeName Collection type name
+ * @param pos Parser position, must not be null
+ */
+ public SqlCollectionTypeNameSpec(
+ SqlTypeNameSpec elementTypeName, SqlTypeName collectionTypeName, SqlParserPos pos) {
+ this(elementTypeName, true, collectionTypeName, pos);
+ }
+
+ public SqlTypeNameSpec getElementTypeName() {
+ return elementTypeName;
+ }
+
+ @Override
+ public RelDataType deriveType(SqlValidator validator) {
+ // FLINK MODIFICATION BEGIN CALCITE-1466
+ final RelDataType type = elementTypeName.deriveType(validator);
+ // FLINK MODIFICATION END
+ return createCollectionType(type, validator.getTypeFactory());
+ }
+
+ @Override
+ public void unparse(SqlWriter writer, int leftPrec, int rightPrec) {
+ elementTypeName.unparse(writer, leftPrec, rightPrec);
+ writer.keyword(collectionTypeName.name());
+ }
+
+ @Override
+ public boolean equalsDeep(SqlTypeNameSpec spec, Litmus litmus) {
+ if (!(spec instanceof SqlCollectionTypeNameSpec)) {
+ return litmus.fail("{} != {}", this, spec);
+ }
+ SqlCollectionTypeNameSpec that = (SqlCollectionTypeNameSpec) spec;
+ if (!this.elementTypeName.equalsDeep(that.elementTypeName, litmus)) {
+ return litmus.fail("{} != {}", this, spec);
+ }
+ if (!Objects.equals(this.collectionTypeName, that.collectionTypeName)) {
+ return litmus.fail("{} != {}", this, spec);
+ }
+ return litmus.succeed();
+ }
+
+ // ~ Tools ------------------------------------------------------------------
+
+ /**
+ * Create collection data type.
+ *
+ * @param elementType Type of the collection element
+ * @param typeFactory Type factory
+ * @return The collection data type, or throw exception if the collection type name does not
+ * belong to {@code SqlTypeName} enumerations
+ */
+ private RelDataType createCollectionType(
+ RelDataType elementType, RelDataTypeFactory typeFactory) {
+ switch (collectionTypeName) {
+ case MULTISET:
+ return typeFactory.createMultisetType(elementType, -1);
+ case ARRAY:
+ return typeFactory.createArrayType(elementType, -1);
+
+ default:
+ throw Util.unexpected(collectionTypeName);
+ }
+ }
+}
diff --git a/flink-table/flink-sql-parser/src/main/java/org/apache/calcite/sql/SqlJoin.java b/flink-table/flink-sql-parser/src/main/java/org/apache/calcite/sql/SqlJoin.java
deleted file mode 100644
index 455255cd6cc81..0000000000000
--- a/flink-table/flink-sql-parser/src/main/java/org/apache/calcite/sql/SqlJoin.java
+++ /dev/null
@@ -1,293 +0,0 @@
-/*
- * 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;
-
-import org.apache.calcite.sql.parser.SqlParserPos;
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.apache.calcite.sql.util.SqlString;
-import org.apache.calcite.util.ImmutableNullableList;
-import org.apache.calcite.util.Util;
-import org.checkerframework.checker.nullness.qual.Nullable;
-
-import java.util.List;
-import java.util.function.UnaryOperator;
-
-import static com.google.common.base.Preconditions.checkArgument;
-import static java.util.Objects.requireNonNull;
-
-/** Parse tree node representing a {@code JOIN} clause. */
-public class SqlJoin extends SqlCall {
- static final SqlJoinOperator COMMA_OPERATOR = new SqlJoinOperator("COMMA-JOIN", 18);
- public static final SqlJoinOperator OPERATOR = new SqlJoinOperator("JOIN", 18);
-
- SqlNode left;
-
- /** Operand says whether this is a natural join. Must be constant TRUE or FALSE. */
- SqlLiteral natural;
-
- /** Value must be a {@link SqlLiteral}, one of the integer codes for {@link JoinType}. */
- SqlLiteral joinType;
-
- SqlNode right;
-
- /**
- * Value must be a {@link SqlLiteral}, one of the integer codes for {@link JoinConditionType}.
- */
- SqlLiteral conditionType;
-
- @Nullable SqlNode condition;
-
- // ~ Constructors -----------------------------------------------------------
-
- public SqlJoin(
- SqlParserPos pos,
- SqlNode left,
- SqlLiteral natural,
- SqlLiteral joinType,
- SqlNode right,
- SqlLiteral conditionType,
- @Nullable SqlNode condition) {
- super(pos);
- this.left = left;
- this.natural = requireNonNull(natural, "natural");
- this.joinType = requireNonNull(joinType, "joinType");
- this.right = right;
- this.conditionType = requireNonNull(conditionType, "conditionType");
- this.condition = condition;
-
- checkArgument(natural.getTypeName() == SqlTypeName.BOOLEAN);
- conditionType.getValueAs(JoinConditionType.class);
- joinType.getValueAs(JoinType.class);
- }
-
- // ~ Methods ----------------------------------------------------------------
-
- @Override
- public SqlOperator getOperator() {
- //noinspection SwitchStatementWithTooFewBranches
- switch (getJoinType()) {
- case COMMA:
- return COMMA_OPERATOR;
- default:
- return OPERATOR;
- }
- }
-
- @Override
- public SqlKind getKind() {
- return SqlKind.JOIN;
- }
-
- @SuppressWarnings("nullness")
- @Override
- public List getOperandList() {
- return ImmutableNullableList.of(left, natural, joinType, right, conditionType, condition);
- }
-
- @SuppressWarnings("assignment.type.incompatible")
- @Override
- public void setOperand(int i, @Nullable SqlNode operand) {
- switch (i) {
- case 0:
- left = operand;
- break;
- case 1:
- natural = (SqlLiteral) operand;
- break;
- case 2:
- joinType = (SqlLiteral) operand;
- break;
- case 3:
- right = operand;
- break;
- case 4:
- conditionType = (SqlLiteral) operand;
- break;
- case 5:
- condition = operand;
- break;
- default:
- throw new AssertionError(i);
- }
- }
-
- public final @Nullable SqlNode getCondition() {
- return condition;
- }
-
- /** Returns a {@link JoinConditionType}, never null. */
- public final JoinConditionType getConditionType() {
- return conditionType.getValueAs(JoinConditionType.class);
- }
-
- public SqlLiteral getConditionTypeNode() {
- return conditionType;
- }
-
- /** Returns a {@link JoinType}, never null. */
- public final JoinType getJoinType() {
- return joinType.getValueAs(JoinType.class);
- }
-
- public SqlLiteral getJoinTypeNode() {
- return joinType;
- }
-
- public final SqlNode getLeft() {
- return left;
- }
-
- public void setLeft(SqlNode left) {
- this.left = left;
- }
-
- public final boolean isNatural() {
- return natural.booleanValue();
- }
-
- public final SqlLiteral isNaturalNode() {
- return natural;
- }
-
- public final SqlNode getRight() {
- return right;
- }
-
- public void setRight(SqlNode right) {
- this.right = right;
- }
-
- /**
- * Describes the syntax of the SQL {@code JOIN} operator.
- *
- * A variant describes the comma operator, which has lower precedence.
- */
- public static class SqlJoinOperator extends SqlOperator {
- private static final SqlWriter.FrameType FRAME_TYPE =
- SqlWriter.FrameTypeEnum.create("USING");
-
- // ~ Constructors -----------------------------------------------------------
-
- private SqlJoinOperator(String name, int prec) {
- super(name, SqlKind.JOIN, prec, true, null, null, null);
- }
-
- // ~ Methods ----------------------------------------------------------------
-
- @Override
- public SqlSyntax getSyntax() {
- return SqlSyntax.SPECIAL;
- }
-
- @SuppressWarnings("argument.type.incompatible")
- @Override
- public SqlCall createCall(
- @Nullable SqlLiteral functionQualifier,
- SqlParserPos pos,
- @Nullable SqlNode... operands) {
- assert functionQualifier == null;
- return new SqlJoin(
- pos,
- operands[0],
- (SqlLiteral) operands[1],
- (SqlLiteral) operands[2],
- operands[3],
- (SqlLiteral) operands[4],
- operands[5]);
- }
-
- @Override
- public void unparse(SqlWriter writer, SqlCall call, int leftPrec, int rightPrec) {
- final SqlJoin join = (SqlJoin) call;
-
- join.left.unparse(writer, leftPrec, getLeftPrec());
- switch (join.getJoinType()) {
- case COMMA:
- writer.sep(",", true);
- break;
- case CROSS:
- writer.sep(join.isNatural() ? "NATURAL CROSS JOIN" : "CROSS JOIN");
- break;
- case FULL:
- writer.sep(join.isNatural() ? "NATURAL FULL JOIN" : "FULL JOIN");
- break;
- case INNER:
- writer.sep(join.isNatural() ? "NATURAL INNER JOIN" : "INNER JOIN");
- break;
- case LEFT:
- writer.sep(join.isNatural() ? "NATURAL LEFT JOIN" : "LEFT JOIN");
- break;
- case LEFT_SEMI_JOIN:
- writer.sep(join.isNatural() ? "NATURAL LEFT SEMI JOIN" : "LEFT SEMI JOIN");
- break;
- case LEFT_ANTI_JOIN:
- writer.sep(join.isNatural() ? "NATURAL LEFT ANTI JOIN" : "LEFT ANTI JOIN");
- break;
- case RIGHT:
- writer.sep(join.isNatural() ? "NATURAL RIGHT JOIN" : "RIGHT JOIN");
- break;
- default:
- throw Util.unexpected(join.getJoinType());
- }
- join.right.unparse(writer, getRightPrec(), rightPrec);
- SqlNode joinCondition = join.condition;
- if (joinCondition != null) {
- switch (join.getConditionType()) {
- case USING:
- // No need for an extra pair of parens -- the condition is a
- // list. The result is something like "USING (deptno, gender)".
- writer.keyword("USING");
- assert joinCondition instanceof SqlNodeList
- : "joinCondition should be SqlNodeList, got " + joinCondition;
- final SqlWriter.Frame frame = writer.startList(FRAME_TYPE, "(", ")");
- joinCondition.unparse(writer, 0, 0);
- writer.endList(frame);
- break;
-
- case ON:
- writer.keyword("ON");
- joinCondition.unparse(writer, leftPrec, rightPrec);
- break;
-
- default:
- throw Util.unexpected(join.getConditionType());
- }
- }
- }
- }
-
- @Override
- public SqlString toSqlString(UnaryOperator transform) {
- SqlNode selectWrapper =
- new SqlSelect(
- SqlParserPos.ZERO,
- SqlNodeList.EMPTY,
- SqlNodeList.SINGLETON_STAR,
- this,
- null,
- null,
- null,
- SqlNodeList.EMPTY,
- null,
- null,
- null,
- null,
- SqlNodeList.EMPTY);
- return selectWrapper.toSqlString(transform);
- }
-}
diff --git a/flink-table/flink-sql-parser/src/main/java/org/apache/calcite/sql/SqlMapTypeNameSpec.java b/flink-table/flink-sql-parser/src/main/java/org/apache/calcite/sql/SqlMapTypeNameSpec.java
index 827302de6a2a2..616dff23490f7 100644
--- a/flink-table/flink-sql-parser/src/main/java/org/apache/calcite/sql/SqlMapTypeNameSpec.java
+++ b/flink-table/flink-sql-parser/src/main/java/org/apache/calcite/sql/SqlMapTypeNameSpec.java
@@ -57,9 +57,12 @@ public SqlDataTypeSpec getValType() {
@Override
public RelDataType deriveType(SqlValidator validator) {
- return validator
- .getTypeFactory()
- .createMapType(keyType.deriveType(validator), valType.deriveType(validator));
+ boolean keyCanBeNullable =
+ validator.getTypeFactory().getTypeSystem().mapKeysCanBeNullable();
+ RelDataType kType = keyType.deriveType(validator, keyCanBeNullable);
+
+ RelDataType valueType = valType.deriveType(validator, true);
+ return validator.getTypeFactory().createMapType(kType, valueType);
}
@Override
diff --git a/flink-table/flink-sql-parser/src/main/java/org/apache/calcite/sql/type/SqlTypeFamily.java b/flink-table/flink-sql-parser/src/main/java/org/apache/calcite/sql/type/SqlTypeFamily.java
deleted file mode 100644
index b755fc4d6f46f..0000000000000
--- a/flink-table/flink-sql-parser/src/main/java/org/apache/calcite/sql/type/SqlTypeFamily.java
+++ /dev/null
@@ -1,292 +0,0 @@
-/*
- * 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.type;
-
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableMap;
-import org.apache.calcite.avatica.util.TimeUnit;
-import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rel.type.RelDataTypeFactory;
-import org.apache.calcite.rel.type.RelDataTypeFamily;
-import org.apache.calcite.sql.SqlIntervalQualifier;
-import org.apache.calcite.sql.SqlWindow;
-import org.apache.calcite.sql.parser.SqlParserPos;
-import org.checkerframework.checker.nullness.qual.Nullable;
-
-import java.sql.Types;
-import java.util.Collection;
-import java.util.List;
-import java.util.Map;
-
-/**
- * SqlTypeFamily provides SQL type categorization.
- *
- * The primary family categorization is a complete disjoint partitioning of SQL types
- * into families, where two types are members of the same primary family iff instances of the two
- * types can be the operands of an SQL equality predicate such as WHERE v1 = v2.
- * Primary families are returned by RelDataType.getFamily().
- *
- *
There is also a secondary family categorization which overlaps with the primary
- * categorization. It is used in type strategies for more specific or more general categorization
- * than the primary families. Secondary families are never returned by RelDataType.getFamily().
- *
- *
This class was copied over from Calcite to support variant type(CALCITE-4918). When upgrading
- * to Calcite 1.39.0 version, please remove the entire class.
- */
-public enum SqlTypeFamily implements RelDataTypeFamily {
- // Primary families.
- CHARACTER,
- BINARY,
- NUMERIC,
- DATE,
- TIME,
- TIMESTAMP,
- BOOLEAN,
- INTERVAL_YEAR_MONTH,
- INTERVAL_DAY_TIME,
-
- // Secondary families.
-
- STRING,
- APPROXIMATE_NUMERIC,
- EXACT_NUMERIC,
- DECIMAL,
- INTEGER,
- DATETIME,
- DATETIME_INTERVAL,
- MULTISET,
- ARRAY,
- MAP,
- NULL,
- ANY,
- CURSOR,
- COLUMN_LIST,
- GEO,
- VARIANT,
- FUNCTION,
- /** Like ANY, but do not even validate the operand. It may not be an expression. */
- IGNORE;
-
- private static final Map JDBC_TYPE_TO_FAMILY =
- ImmutableMap.builder()
- // Not present:
- // SqlTypeName.MULTISET shares Types.ARRAY with SqlTypeName.ARRAY;
- // SqlTypeName.MAP has no corresponding JDBC type
- // SqlTypeName.COLUMN_LIST has no corresponding JDBC type
- .put(Types.BIT, NUMERIC)
- .put(Types.TINYINT, NUMERIC)
- .put(Types.SMALLINT, NUMERIC)
- .put(Types.BIGINT, NUMERIC)
- .put(Types.INTEGER, NUMERIC)
- .put(Types.NUMERIC, NUMERIC)
- .put(Types.DECIMAL, NUMERIC)
- .put(Types.FLOAT, NUMERIC)
- .put(Types.REAL, NUMERIC)
- .put(Types.DOUBLE, NUMERIC)
- .put(Types.CHAR, CHARACTER)
- .put(Types.VARCHAR, CHARACTER)
- .put(Types.LONGVARCHAR, CHARACTER)
- .put(Types.CLOB, CHARACTER)
- .put(Types.BINARY, BINARY)
- .put(Types.VARBINARY, BINARY)
- .put(Types.LONGVARBINARY, BINARY)
- .put(Types.BLOB, BINARY)
- .put(Types.DATE, DATE)
- .put(Types.TIME, TIME)
- .put(ExtraSqlTypes.TIME_WITH_TIMEZONE, TIME)
- .put(Types.TIMESTAMP, TIMESTAMP)
- .put(ExtraSqlTypes.TIMESTAMP_WITH_TIMEZONE, TIMESTAMP)
- .put(Types.BOOLEAN, BOOLEAN)
- .put(ExtraSqlTypes.REF_CURSOR, CURSOR)
- .put(Types.ARRAY, ARRAY)
- .put(Types.JAVA_OBJECT, VARIANT)
- .build();
-
- /**
- * Gets the primary family containing a JDBC type.
- *
- * @param jdbcType the JDBC type of interest
- * @return containing family
- */
- public static @Nullable SqlTypeFamily getFamilyForJdbcType(int jdbcType) {
- return JDBC_TYPE_TO_FAMILY.get(jdbcType);
- }
-
- /**
- * For this type family, returns the allow types of the difference between two values of this
- * family.
- *
- * Equivalently, given an {@code ORDER BY} expression with one key, returns the allowable
- * type families of the difference between two keys.
- *
- *
Example 1. For {@code ORDER BY empno}, a NUMERIC, the difference between two {@code empno}
- * values is also NUMERIC.
- *
- *
Example 2. For {@code ORDER BY hireDate}, a DATE, the difference between two {@code
- * hireDate} values might be an INTERVAL_DAY_TIME or INTERVAL_YEAR_MONTH.
- *
- *
The result determines whether a {@link SqlWindow} with a {@code RANGE} is valid (for
- * example, {@code OVER (ORDER BY empno RANGE 10} is valid because {@code 10} is numeric); and
- * whether a call to {@link org.apache.calcite.sql.fun.SqlStdOperatorTable#PERCENTILE_CONT
- * PERCENTILE_CONT} is valid (for example, {@code PERCENTILE_CONT(0.25)} ORDER BY (hireDate)} is
- * valid because {@code hireDate} values may be interpolated by adding values of type {@code
- * INTERVAL_DAY_TIME}.
- */
- public List allowableDifferenceTypes() {
- switch (this) {
- case NUMERIC:
- return ImmutableList.of(NUMERIC);
- case DATE:
- case TIME:
- case TIMESTAMP:
- return ImmutableList.of(INTERVAL_DAY_TIME, INTERVAL_YEAR_MONTH);
- default:
- return ImmutableList.of();
- }
- }
-
- /** Returns the collection of {@link SqlTypeName}s included in this family. */
- public Collection getTypeNames() {
- switch (this) {
- case CHARACTER:
- return SqlTypeName.CHAR_TYPES;
- case BINARY:
- return SqlTypeName.BINARY_TYPES;
- case NUMERIC:
- return SqlTypeName.NUMERIC_TYPES;
- case DECIMAL:
- return ImmutableList.of(SqlTypeName.DECIMAL);
- case DATE:
- return ImmutableList.of(SqlTypeName.DATE);
- case TIME:
- return ImmutableList.of(
- SqlTypeName.TIME,
- SqlTypeName.TIME_WITH_LOCAL_TIME_ZONE,
- SqlTypeName.TIME_TZ);
- case TIMESTAMP:
- return ImmutableList.of(
- SqlTypeName.TIMESTAMP,
- SqlTypeName.TIMESTAMP_WITH_LOCAL_TIME_ZONE,
- SqlTypeName.TIMESTAMP_TZ);
- case BOOLEAN:
- return SqlTypeName.BOOLEAN_TYPES;
- case INTERVAL_YEAR_MONTH:
- return SqlTypeName.YEAR_INTERVAL_TYPES;
- case INTERVAL_DAY_TIME:
- return SqlTypeName.DAY_INTERVAL_TYPES;
- case STRING:
- return SqlTypeName.STRING_TYPES;
- case APPROXIMATE_NUMERIC:
- return SqlTypeName.APPROX_TYPES;
- case EXACT_NUMERIC:
- return SqlTypeName.EXACT_TYPES;
- case INTEGER:
- return SqlTypeName.INT_TYPES;
- case DATETIME:
- return SqlTypeName.DATETIME_TYPES;
- case DATETIME_INTERVAL:
- return SqlTypeName.INTERVAL_TYPES;
- case GEO:
- return SqlTypeName.GEOMETRY_TYPES;
- case MULTISET:
- return ImmutableList.of(SqlTypeName.MULTISET);
- case ARRAY:
- return ImmutableList.of(SqlTypeName.ARRAY);
- case MAP:
- return ImmutableList.of(SqlTypeName.MAP);
- case NULL:
- return ImmutableList.of(SqlTypeName.NULL);
- case ANY:
- return SqlTypeName.ALL_TYPES;
- case CURSOR:
- return ImmutableList.of(SqlTypeName.CURSOR);
- case COLUMN_LIST:
- return ImmutableList.of(SqlTypeName.COLUMN_LIST);
- case FUNCTION:
- return ImmutableList.of(SqlTypeName.FUNCTION);
- case VARIANT:
- return ImmutableList.of(SqlTypeName.VARIANT);
- default:
- throw new IllegalArgumentException();
- }
- }
-
- /** Return the default {@link RelDataType} that belongs to this family. */
- public @Nullable RelDataType getDefaultConcreteType(RelDataTypeFactory factory) {
- switch (this) {
- case CHARACTER:
- return factory.createSqlType(SqlTypeName.VARCHAR);
- case BINARY:
- return factory.createSqlType(SqlTypeName.VARBINARY);
- case NUMERIC:
- return SqlTypeUtil.getMaxPrecisionScaleDecimal(factory);
- case DATE:
- return factory.createSqlType(SqlTypeName.DATE);
- case TIME:
- return factory.createSqlType(SqlTypeName.TIME);
- case TIMESTAMP:
- return factory.createSqlType(SqlTypeName.TIMESTAMP);
- case BOOLEAN:
- return factory.createSqlType(SqlTypeName.BOOLEAN);
- case STRING:
- return factory.createSqlType(SqlTypeName.VARCHAR);
- case APPROXIMATE_NUMERIC:
- return factory.createSqlType(SqlTypeName.DOUBLE);
- case EXACT_NUMERIC:
- return SqlTypeUtil.getMaxPrecisionScaleDecimal(factory);
- case INTEGER:
- return factory.createSqlType(SqlTypeName.BIGINT);
- case DECIMAL:
- return factory.createSqlType(SqlTypeName.DECIMAL);
- case DATETIME:
- return factory.createSqlType(SqlTypeName.TIMESTAMP);
- case INTERVAL_DAY_TIME:
- return factory.createSqlIntervalType(
- new SqlIntervalQualifier(TimeUnit.DAY, TimeUnit.SECOND, SqlParserPos.ZERO));
- case INTERVAL_YEAR_MONTH:
- return factory.createSqlIntervalType(
- new SqlIntervalQualifier(TimeUnit.YEAR, TimeUnit.MONTH, SqlParserPos.ZERO));
- case GEO:
- return factory.createSqlType(SqlTypeName.GEOMETRY);
- case MULTISET:
- return factory.createMultisetType(factory.createSqlType(SqlTypeName.ANY), -1);
- case ARRAY:
- return factory.createArrayType(factory.createSqlType(SqlTypeName.ANY), -1);
- case MAP:
- return factory.createMapType(
- factory.createSqlType(SqlTypeName.ANY),
- factory.createSqlType(SqlTypeName.ANY));
- case NULL:
- return factory.createSqlType(SqlTypeName.NULL);
- case CURSOR:
- return factory.createSqlType(SqlTypeName.CURSOR);
- case COLUMN_LIST:
- return factory.createSqlType(SqlTypeName.COLUMN_LIST);
- case FUNCTION:
- return factory.createFunctionSqlType(
- factory.createStructType(ImmutableList.of(), ImmutableList.of()),
- factory.createSqlType(SqlTypeName.ANY));
- default:
- return null;
- }
- }
-
- public boolean contains(RelDataType type) {
- return SqlTypeUtil.isOfSameTypeName(getTypeNames(), type);
- }
-}
diff --git a/flink-table/flink-sql-parser/src/main/java/org/apache/calcite/sql/type/SqlTypeName.java b/flink-table/flink-sql-parser/src/main/java/org/apache/calcite/sql/type/SqlTypeName.java
deleted file mode 100644
index 5f76b4493b1a6..0000000000000
--- a/flink-table/flink-sql-parser/src/main/java/org/apache/calcite/sql/type/SqlTypeName.java
+++ /dev/null
@@ -1,1078 +0,0 @@
-/*
- * 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.type;
-
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.Iterables;
-import com.google.common.collect.Sets;
-import org.apache.calcite.avatica.util.TimeUnit;
-import org.apache.calcite.sql.SqlLiteral;
-import org.apache.calcite.sql.parser.SqlParserPos;
-import org.apache.calcite.util.DateString;
-import org.apache.calcite.util.TimeString;
-import org.apache.calcite.util.TimestampString;
-import org.apache.calcite.util.Util;
-import org.checkerframework.checker.nullness.qual.Nullable;
-
-import java.math.BigDecimal;
-import java.sql.Types;
-import java.util.Arrays;
-import java.util.Calendar;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
-/**
- * Enumeration of the type names which can be used to construct a SQL type. Rationale for this
- * class's existence (instead of just using the standard java.sql.Type ordinals):
- *
- *
- * - {@link Types} does not include all SQL2003 data-types;
- *
- SqlTypeName provides a type-safe enumeration;
- *
- SqlTypeName provides a place to hang extra information such as whether the type carries
- * precision and scale.
- *
- *
- * This class was copied over from Calcite to support variant type(CALCITE-4918). When upgrading
- * to Calcite 1.39.0 version, please remove the entire class.
- */
-public enum SqlTypeName {
- BOOLEAN(PrecScale.NO_NO, false, Types.BOOLEAN, SqlTypeFamily.BOOLEAN),
- TINYINT(PrecScale.NO_NO, false, Types.TINYINT, SqlTypeFamily.NUMERIC),
- SMALLINT(PrecScale.NO_NO, false, Types.SMALLINT, SqlTypeFamily.NUMERIC),
- INTEGER(PrecScale.NO_NO, false, Types.INTEGER, SqlTypeFamily.NUMERIC),
- BIGINT(PrecScale.NO_NO, false, Types.BIGINT, SqlTypeFamily.NUMERIC),
- DECIMAL(
- PrecScale.NO_NO | PrecScale.YES_NO | PrecScale.YES_YES,
- false,
- Types.DECIMAL,
- SqlTypeFamily.NUMERIC),
- FLOAT(PrecScale.NO_NO, false, Types.FLOAT, SqlTypeFamily.NUMERIC),
- REAL(PrecScale.NO_NO, false, Types.REAL, SqlTypeFamily.NUMERIC),
- DOUBLE(PrecScale.NO_NO, false, Types.DOUBLE, SqlTypeFamily.NUMERIC),
- DATE(PrecScale.NO_NO, false, Types.DATE, SqlTypeFamily.DATE),
- TIME(PrecScale.NO_NO | PrecScale.YES_NO, false, Types.TIME, SqlTypeFamily.TIME),
- TIME_WITH_LOCAL_TIME_ZONE(
- PrecScale.NO_NO | PrecScale.YES_NO, false, Types.TIME, SqlTypeFamily.TIME),
- TIME_TZ(PrecScale.NO_NO | PrecScale.YES_NO, false, Types.TIME, SqlTypeFamily.TIME),
- TIMESTAMP(PrecScale.NO_NO | PrecScale.YES_NO, false, Types.TIMESTAMP, SqlTypeFamily.TIMESTAMP),
- TIMESTAMP_WITH_LOCAL_TIME_ZONE(
- PrecScale.NO_NO | PrecScale.YES_NO, false, Types.TIMESTAMP, SqlTypeFamily.TIMESTAMP),
- TIMESTAMP_TZ(
- PrecScale.NO_NO | PrecScale.YES_NO, false, Types.TIMESTAMP, SqlTypeFamily.TIMESTAMP),
- INTERVAL_YEAR(PrecScale.NO_NO, false, Types.OTHER, SqlTypeFamily.INTERVAL_YEAR_MONTH),
- INTERVAL_YEAR_MONTH(PrecScale.NO_NO, false, Types.OTHER, SqlTypeFamily.INTERVAL_YEAR_MONTH),
- INTERVAL_MONTH(PrecScale.NO_NO, false, Types.OTHER, SqlTypeFamily.INTERVAL_YEAR_MONTH),
- INTERVAL_DAY(
- PrecScale.NO_NO | PrecScale.YES_NO | PrecScale.YES_YES,
- false,
- Types.OTHER,
- SqlTypeFamily.INTERVAL_DAY_TIME),
- INTERVAL_DAY_HOUR(
- PrecScale.NO_NO | PrecScale.YES_NO | PrecScale.YES_YES,
- false,
- Types.OTHER,
- SqlTypeFamily.INTERVAL_DAY_TIME),
- INTERVAL_DAY_MINUTE(
- PrecScale.NO_NO | PrecScale.YES_NO | PrecScale.YES_YES,
- false,
- Types.OTHER,
- SqlTypeFamily.INTERVAL_DAY_TIME),
- INTERVAL_DAY_SECOND(
- PrecScale.NO_NO | PrecScale.YES_NO | PrecScale.YES_YES,
- false,
- Types.OTHER,
- SqlTypeFamily.INTERVAL_DAY_TIME),
- INTERVAL_HOUR(
- PrecScale.NO_NO | PrecScale.YES_NO | PrecScale.YES_YES,
- false,
- Types.OTHER,
- SqlTypeFamily.INTERVAL_DAY_TIME),
- INTERVAL_HOUR_MINUTE(
- PrecScale.NO_NO | PrecScale.YES_NO | PrecScale.YES_YES,
- false,
- Types.OTHER,
- SqlTypeFamily.INTERVAL_DAY_TIME),
- INTERVAL_HOUR_SECOND(
- PrecScale.NO_NO | PrecScale.YES_NO | PrecScale.YES_YES,
- false,
- Types.OTHER,
- SqlTypeFamily.INTERVAL_DAY_TIME),
- INTERVAL_MINUTE(
- PrecScale.NO_NO | PrecScale.YES_NO | PrecScale.YES_YES,
- false,
- Types.OTHER,
- SqlTypeFamily.INTERVAL_DAY_TIME),
- INTERVAL_MINUTE_SECOND(
- PrecScale.NO_NO | PrecScale.YES_NO | PrecScale.YES_YES,
- false,
- Types.OTHER,
- SqlTypeFamily.INTERVAL_DAY_TIME),
- INTERVAL_SECOND(
- PrecScale.NO_NO | PrecScale.YES_NO | PrecScale.YES_YES,
- false,
- Types.OTHER,
- SqlTypeFamily.INTERVAL_DAY_TIME),
- CHAR(PrecScale.NO_NO | PrecScale.YES_NO, false, Types.CHAR, SqlTypeFamily.CHARACTER),
- VARCHAR(PrecScale.NO_NO | PrecScale.YES_NO, false, Types.VARCHAR, SqlTypeFamily.CHARACTER),
- BINARY(PrecScale.NO_NO | PrecScale.YES_NO, false, Types.BINARY, SqlTypeFamily.BINARY),
- VARBINARY(PrecScale.NO_NO | PrecScale.YES_NO, false, Types.VARBINARY, SqlTypeFamily.BINARY),
- NULL(PrecScale.NO_NO, true, Types.NULL, SqlTypeFamily.NULL),
- UNKNOWN(PrecScale.NO_NO, true, Types.NULL, SqlTypeFamily.NULL),
- ANY(
- PrecScale.NO_NO | PrecScale.YES_NO | PrecScale.YES_YES,
- true,
- Types.JAVA_OBJECT,
- SqlTypeFamily.ANY),
- SYMBOL(PrecScale.NO_NO, true, Types.OTHER, null),
- MULTISET(PrecScale.NO_NO, false, Types.ARRAY, SqlTypeFamily.MULTISET),
- ARRAY(PrecScale.NO_NO, false, Types.ARRAY, SqlTypeFamily.ARRAY),
- MAP(PrecScale.NO_NO, false, Types.OTHER, SqlTypeFamily.MAP),
- DISTINCT(PrecScale.NO_NO, false, Types.DISTINCT, null),
- STRUCTURED(PrecScale.NO_NO, false, Types.STRUCT, null),
- ROW(PrecScale.NO_NO, false, Types.STRUCT, null),
- OTHER(PrecScale.NO_NO, false, Types.OTHER, null),
- CURSOR(PrecScale.NO_NO, false, ExtraSqlTypes.REF_CURSOR, SqlTypeFamily.CURSOR),
- COLUMN_LIST(PrecScale.NO_NO, false, Types.OTHER + 2, SqlTypeFamily.COLUMN_LIST),
- DYNAMIC_STAR(
- PrecScale.NO_NO | PrecScale.YES_NO | PrecScale.YES_YES,
- true,
- Types.JAVA_OBJECT,
- SqlTypeFamily.ANY),
- /**
- * Spatial type. Though not standard, it is common to several DBs, so we do not flag it
- * 'special' (internal).
- */
- GEOMETRY(PrecScale.NO_NO, false, ExtraSqlTypes.GEOMETRY, SqlTypeFamily.GEO),
- MEASURE(PrecScale.NO_NO, true, Types.OTHER, SqlTypeFamily.ANY),
- FUNCTION(PrecScale.NO_NO, true, Types.OTHER, SqlTypeFamily.FUNCTION),
- SARG(PrecScale.NO_NO, true, Types.OTHER, SqlTypeFamily.ANY),
- /**
- * VARIANT data type, a dynamically-typed value that can have at runtime any of the other data
- * types in this table.
- */
- VARIANT(PrecScale.NO_NO, false, Types.OTHER, SqlTypeFamily.VARIANT);
-
- public static final int MAX_DATETIME_PRECISION = 3;
-
- // Minimum and default interval precisions are defined by SQL2003
- // Maximum interval precisions are implementation dependent,
- // but must be at least the default value
- public static final int DEFAULT_INTERVAL_START_PRECISION = 2;
- public static final int DEFAULT_INTERVAL_FRACTIONAL_SECOND_PRECISION = 6;
- public static final int MIN_INTERVAL_START_PRECISION = 1;
- public static final int MIN_INTERVAL_FRACTIONAL_SECOND_PRECISION = 0;
- public static final int MAX_INTERVAL_START_PRECISION = 10;
- public static final int MAX_INTERVAL_FRACTIONAL_SECOND_PRECISION = 9;
-
- // Cached map of enum values
- private static final Map VALUES_MAP =
- Util.enumConstants(SqlTypeName.class);
-
- // categorizations used by SqlTypeFamily definitions
-
- // you probably want to use JDK 1.5 support for treating enumeration
- // as collection instead; this is only here to support
- // SqlTypeFamily.ANY
- public static final List ALL_TYPES =
- ImmutableList.of(
- BOOLEAN,
- INTEGER,
- VARCHAR,
- DATE,
- TIME,
- TIMESTAMP,
- NULL,
- DECIMAL,
- ANY,
- CHAR,
- BINARY,
- VARBINARY,
- TINYINT,
- SMALLINT,
- BIGINT,
- REAL,
- DOUBLE,
- SYMBOL,
- INTERVAL_YEAR,
- INTERVAL_YEAR_MONTH,
- INTERVAL_MONTH,
- INTERVAL_DAY,
- INTERVAL_DAY_HOUR,
- INTERVAL_DAY_MINUTE,
- INTERVAL_DAY_SECOND,
- INTERVAL_HOUR,
- INTERVAL_HOUR_MINUTE,
- INTERVAL_HOUR_SECOND,
- INTERVAL_MINUTE,
- INTERVAL_MINUTE_SECOND,
- INTERVAL_SECOND,
- TIME_WITH_LOCAL_TIME_ZONE,
- TIME_TZ,
- TIMESTAMP_WITH_LOCAL_TIME_ZONE,
- TIMESTAMP_TZ,
- FLOAT,
- MULTISET,
- DISTINCT,
- STRUCTURED,
- ROW,
- CURSOR,
- COLUMN_LIST,
- VARIANT);
-
- public static final List BOOLEAN_TYPES = ImmutableList.of(BOOLEAN);
-
- public static final List BINARY_TYPES = ImmutableList.of(BINARY, VARBINARY);
-
- public static final List INT_TYPES =
- ImmutableList.of(TINYINT, SMALLINT, INTEGER, BIGINT);
-
- public static final List EXACT_TYPES =
- combine(INT_TYPES, ImmutableList.of(DECIMAL));
-
- public static final List APPROX_TYPES = ImmutableList.of(FLOAT, REAL, DOUBLE);
-
- public static final List NUMERIC_TYPES = combine(EXACT_TYPES, APPROX_TYPES);
-
- public static final List FRACTIONAL_TYPES =
- combine(APPROX_TYPES, ImmutableList.of(DECIMAL));
-
- public static final List CHAR_TYPES = ImmutableList.of(CHAR, VARCHAR);
-
- public static final List STRING_TYPES = combine(CHAR_TYPES, BINARY_TYPES);
-
- public static final List GEOMETRY_TYPES = ImmutableList.of(GEOMETRY);
-
- public static final List DATETIME_TYPES =
- ImmutableList.of(
- DATE,
- TIME,
- TIME_WITH_LOCAL_TIME_ZONE,
- TIME_TZ,
- TIMESTAMP,
- TIMESTAMP_WITH_LOCAL_TIME_ZONE,
- TIMESTAMP_TZ);
-
- /** Types that contain time zone information. */
- public static final List TZ_TYPES =
- ImmutableList.of(
- TIME_WITH_LOCAL_TIME_ZONE,
- TIME_TZ,
- TIMESTAMP_WITH_LOCAL_TIME_ZONE,
- TIMESTAMP_TZ);
-
- public static final Set YEAR_INTERVAL_TYPES =
- Sets.immutableEnumSet(
- SqlTypeName.INTERVAL_YEAR,
- SqlTypeName.INTERVAL_YEAR_MONTH,
- SqlTypeName.INTERVAL_MONTH);
-
- public static final Set DAY_INTERVAL_TYPES =
- Sets.immutableEnumSet(
- SqlTypeName.INTERVAL_DAY,
- SqlTypeName.INTERVAL_DAY_HOUR,
- SqlTypeName.INTERVAL_DAY_MINUTE,
- SqlTypeName.INTERVAL_DAY_SECOND,
- SqlTypeName.INTERVAL_HOUR,
- SqlTypeName.INTERVAL_HOUR_MINUTE,
- SqlTypeName.INTERVAL_HOUR_SECOND,
- SqlTypeName.INTERVAL_MINUTE,
- SqlTypeName.INTERVAL_MINUTE_SECOND,
- SqlTypeName.INTERVAL_SECOND);
-
- public static final Set INTERVAL_TYPES =
- Sets.immutableEnumSet(Iterables.concat(YEAR_INTERVAL_TYPES, DAY_INTERVAL_TYPES));
-
- /** The possible types of a time frame argument to a function such as {@code TIMESTAMP_DIFF}. */
- public static final Set TIME_FRAME_TYPES =
- Sets.immutableEnumSet(Iterables.concat(INTERVAL_TYPES, ImmutableList.of(SYMBOL)));
-
- private static final Map JDBC_TYPE_TO_NAME =
- ImmutableMap.builder()
- .put(Types.TINYINT, TINYINT)
- .put(Types.SMALLINT, SMALLINT)
- .put(Types.BIGINT, BIGINT)
- .put(Types.INTEGER, INTEGER)
- .put(Types.NUMERIC, DECIMAL) // REVIEW
- .put(Types.DECIMAL, DECIMAL)
- .put(Types.FLOAT, FLOAT)
- .put(Types.REAL, REAL)
- .put(Types.DOUBLE, DOUBLE)
- .put(Types.CHAR, CHAR)
- .put(Types.VARCHAR, VARCHAR)
-
- // TODO: provide real support for these eventually
- .put(ExtraSqlTypes.NCHAR, CHAR)
- .put(ExtraSqlTypes.NVARCHAR, VARCHAR)
-
- // TODO: additional types not yet supported. See ExtraSqlTypes.
- // .put(Types.LONGVARCHAR, Longvarchar)
- // .put(Types.CLOB, Clob)
- // .put(Types.LONGVARBINARY, Longvarbinary)
- // .put(Types.BLOB, Blob)
- // .put(Types.LONGNVARCHAR, Longnvarchar)
- // .put(Types.NCLOB, Nclob)
- // .put(Types.ROWID, Rowid)
- // .put(Types.SQLXML, Sqlxml)
-
- .put(Types.BINARY, BINARY)
- .put(Types.VARBINARY, VARBINARY)
- .put(Types.DATE, DATE)
- .put(Types.TIME, TIME)
- .put(Types.TIMESTAMP, TIMESTAMP)
- .put(Types.BIT, BOOLEAN)
- .put(Types.BOOLEAN, BOOLEAN)
- .put(Types.DISTINCT, DISTINCT)
- .put(Types.STRUCT, STRUCTURED)
- .put(Types.ARRAY, ARRAY)
- .build();
-
- /** Bitwise-or of flags indicating allowable precision/scale combinations. */
- private final int signatures;
-
- /**
- * Returns true if not of a "pure" standard sql type. "Inpure" types are {@link #ANY}, {@link
- * #NULL} and {@link #SYMBOL}
- */
- private final boolean special;
-
- private final int jdbcOrdinal;
- private final @Nullable SqlTypeFamily family;
-
- SqlTypeName(int signatures, boolean special, int jdbcType, @Nullable SqlTypeFamily family) {
- this.signatures = signatures;
- this.special = special;
- this.jdbcOrdinal = jdbcType;
- this.family = family;
- }
-
- /**
- * Looks up a type name from its name.
- *
- * @return Type name, or null if not found
- */
- public static @Nullable SqlTypeName get(String name) {
- if (false) {
- // The following code works OK, but the spurious exceptions are
- // annoying.
- try {
- return SqlTypeName.valueOf(name);
- } catch (IllegalArgumentException e) {
- return null;
- }
- }
- return VALUES_MAP.get(name);
- }
-
- /**
- * Returns the SqlTypeName value whose name or {@link #getSpaceName()} matches the given name,
- * or throws {@link IllegalArgumentException}; never returns null.
- */
- public static SqlTypeName lookup(String tag) {
- // Special handling for TIME WITH TIME ZONE and
- // TIMESTAMP WITH TIME ZONE, whose type names are TIME_TZ and TIMESTAMP_TZ.
- // We know that the type name is always uppercase, because it is
- // inserted in the tag by the parser.
- final String tag1 = tag.replace("WITH TIME ZONE", "TZ");
- final String tag2 = tag1.replace(' ', '_');
- return valueOf(tag2);
- }
-
- public boolean allowsNoPrecNoScale() {
- return (signatures & PrecScale.NO_NO) != 0;
- }
-
- public boolean allowsPrecNoScale() {
- return (signatures & PrecScale.YES_NO) != 0;
- }
-
- public boolean allowsPrec() {
- return allowsPrecScale(true, true) || allowsPrecScale(true, false);
- }
-
- public boolean allowsScale() {
- return allowsPrecScale(true, true);
- }
-
- /**
- * Returns whether this type can be specified with a given combination of precision and scale.
- * For example,
- *
- *
- * Varchar.allowsPrecScale(true, false) returns
- * true, because the VARCHAR type allows a precision parameter, as in VARCHAR(10)
- * .
- * Varchar.allowsPrecScale(true, true) returns
- * false, because the VARCHAR type does not allow a precision and a scale parameter, as
- * in VARCHAR(10, 4).
- * allowsPrecScale(false, true) returns false for every type.
- *
- *
- * @param precision Whether the precision/length field is part of the type specification
- * @param scale Whether the scale field is part of the type specification
- * @return Whether this combination of precision/scale is valid
- */
- public boolean allowsPrecScale(boolean precision, boolean scale) {
- int mask =
- precision
- ? (scale ? PrecScale.YES_YES : PrecScale.YES_NO)
- : (scale ? 0 : PrecScale.NO_NO);
- return (signatures & mask) != 0;
- }
-
- public boolean isSpecial() {
- return special;
- }
-
- /** Returns the ordinal from {@link java.sql.Types} corresponding to this SqlTypeName. */
- public int getJdbcOrdinal() {
- return jdbcOrdinal;
- }
-
- private static List combine(List list0, List list1) {
- return ImmutableList.builder().addAll(list0).addAll(list1).build();
- }
-
- /**
- * Returns the default scale for this type if supported, otherwise -1 if scale is either
- * unsupported or must be specified explicitly.
- */
- public int getDefaultScale() {
- switch (this) {
- case DECIMAL:
- return 0;
- case INTERVAL_YEAR:
- case INTERVAL_YEAR_MONTH:
- case INTERVAL_MONTH:
- case INTERVAL_DAY:
- case INTERVAL_DAY_HOUR:
- case INTERVAL_DAY_MINUTE:
- case INTERVAL_DAY_SECOND:
- case INTERVAL_HOUR:
- case INTERVAL_HOUR_MINUTE:
- case INTERVAL_HOUR_SECOND:
- case INTERVAL_MINUTE:
- case INTERVAL_MINUTE_SECOND:
- case INTERVAL_SECOND:
- return DEFAULT_INTERVAL_FRACTIONAL_SECOND_PRECISION;
- default:
- return -1;
- }
- }
-
- /**
- * Gets the SqlTypeFamily containing this SqlTypeName.
- *
- * @return containing family, or null for none (SYMBOL, DISTINCT, STRUCTURED, ROW, OTHER)
- */
- public @Nullable SqlTypeFamily getFamily() {
- return family;
- }
-
- /**
- * Gets the SqlTypeName corresponding to a JDBC type.
- *
- * @param jdbcType the JDBC type of interest
- * @return corresponding SqlTypeName, or null if the type is not known
- */
- public static @Nullable SqlTypeName getNameForJdbcType(int jdbcType) {
- return JDBC_TYPE_TO_NAME.get(jdbcType);
- }
-
- /**
- * Returns the limit of this datatype. For example,
- *
- *
- * Datatype limits
- *
- * | Datatype |
- * sign |
- * limit |
- * beyond |
- * precision |
- * scale |
- * Returns |
- *
- *
- * | Integer |
- * true |
- * true |
- * false |
- * -1 |
- * -1 |
- * 2147483647 (2 ^ 31 -1 = MAXINT) |
- *
- *
- * | Integer |
- * true |
- * true |
- * true |
- * -1 |
- * -1 |
- * 2147483648 (2 ^ 31 = MAXINT + 1) |
- *
- *
- * | Integer |
- * false |
- * true |
- * false |
- * -1 |
- * -1 |
- * -2147483648 (-2 ^ 31 = MININT) |
- *
- *
- * | Boolean |
- * true |
- * true |
- * false |
- * -1 |
- * -1 |
- * TRUE |
- *
- *
- * | Varchar |
- * true |
- * true |
- * false |
- * 10 |
- * -1 |
- * 'ZZZZZZZZZZ' |
- *
- *
- *
- * @param sign If true, returns upper limit, otherwise lower limit
- * @param limit If true, returns value at or near to overflow; otherwise value at or near to
- * underflow
- * @param beyond If true, returns the value just beyond the limit, otherwise the value at the
- * limit
- * @param precision Precision, or -1 if not applicable
- * @param scale Scale, or -1 if not applicable
- * @return Limit value
- */
- public @Nullable Object getLimit(
- boolean sign, Limit limit, boolean beyond, int precision, int scale) {
- assert allowsPrecScale(precision != -1, scale != -1) : this;
- if (limit == Limit.ZERO) {
- if (beyond) {
- return null;
- }
- sign = true;
- }
- Calendar calendar;
-
- switch (this) {
- case BOOLEAN:
- switch (limit) {
- case ZERO:
- return false;
- case UNDERFLOW:
- return null;
- case OVERFLOW:
- if (beyond || !sign) {
- return null;
- } else {
- return true;
- }
- default:
- throw Util.unexpected(limit);
- }
-
- case TINYINT:
- return getNumericLimit(2, 8, sign, limit, beyond);
-
- case SMALLINT:
- return getNumericLimit(2, 16, sign, limit, beyond);
-
- case INTEGER:
- return getNumericLimit(2, 32, sign, limit, beyond);
-
- case BIGINT:
- return getNumericLimit(2, 64, sign, limit, beyond);
-
- case DECIMAL:
- BigDecimal decimal = getNumericLimit(10, precision, sign, limit, beyond);
- if (decimal == null) {
- return null;
- }
-
- // Decimal values must fit into 64 bits. So, the maximum value of
- // a DECIMAL(19, 0) is 2^63 - 1, not 10^19 - 1.
- switch (limit) {
- case OVERFLOW:
- final BigDecimal other =
- (BigDecimal) BIGINT.getLimit(sign, limit, beyond, -1, -1);
- if (other != null && decimal.compareTo(other) == (sign ? 1 : -1)) {
- decimal = other;
- }
- break;
- default:
- break;
- }
-
- // Apply scale.
- if (scale == 0) {
- // do nothing
- } else if (scale > 0) {
- decimal = decimal.divide(BigDecimal.TEN.pow(scale));
- } else {
- decimal = decimal.multiply(BigDecimal.TEN.pow(-scale));
- }
- return decimal;
-
- case CHAR:
- case VARCHAR:
- if (!sign) {
- return null; // this type does not have negative values
- }
- StringBuilder buf = new StringBuilder();
- switch (limit) {
- case ZERO:
- break;
- case UNDERFLOW:
- if (beyond) {
- // There is no value between the empty string and the
- // smallest non-empty string.
- return null;
- }
- buf.append("a");
- break;
- case OVERFLOW:
- for (int i = 0; i < precision; ++i) {
- buf.append("Z");
- }
- if (beyond) {
- buf.append("Z");
- }
- break;
- default:
- break;
- }
- return buf.toString();
-
- case BINARY:
- case VARBINARY:
- if (!sign) {
- return null; // this type does not have negative values
- }
- byte[] bytes;
- switch (limit) {
- case ZERO:
- bytes = new byte[0];
- break;
- case UNDERFLOW:
- if (beyond) {
- // There is no value between the empty string and the
- // smallest value.
- return null;
- }
- bytes = new byte[] {0x00};
- break;
- case OVERFLOW:
- bytes = new byte[precision + (beyond ? 1 : 0)];
- Arrays.fill(bytes, (byte) 0xff);
- break;
- default:
- throw Util.unexpected(limit);
- }
- return bytes;
-
- case DATE:
- calendar = Util.calendar();
- switch (limit) {
- case ZERO:
- // The epoch.
- calendar.set(Calendar.YEAR, 1970);
- calendar.set(Calendar.MONTH, 0);
- calendar.set(Calendar.DAY_OF_MONTH, 1);
- break;
- case UNDERFLOW:
- return null;
- case OVERFLOW:
- if (beyond) {
- // It is impossible to represent an invalid year as a date
- // literal. SQL dates are represented as 'yyyy-mm-dd', and
- // 1 <= yyyy <= 9999 is valid. There is no year 0: the year
- // before 1AD is 1BC, so SimpleDateFormat renders the day
- // before 0001-01-01 (AD) as 0001-12-31 (BC), which looks
- // like a valid date.
- return null;
- }
-
- // "SQL:2003 6.1 Access Rules 6" says that year is
- // between 1 and 9999, and days/months are the valid Gregorian
- // calendar values for these years.
- if (sign) {
- calendar.set(Calendar.YEAR, 9999);
- calendar.set(Calendar.MONTH, 11);
- calendar.set(Calendar.DAY_OF_MONTH, 31);
- } else {
- calendar.set(Calendar.YEAR, 1);
- calendar.set(Calendar.MONTH, 0);
- calendar.set(Calendar.DAY_OF_MONTH, 1);
- }
- break;
- default:
- break;
- }
- calendar.set(Calendar.HOUR_OF_DAY, 0);
- calendar.set(Calendar.MINUTE, 0);
- calendar.set(Calendar.SECOND, 0);
- return calendar;
-
- case TIME:
- if (!sign) {
- return null; // this type does not have negative values
- }
- if (beyond) {
- return null; // invalid values are impossible to represent
- }
- calendar = Util.calendar();
- switch (limit) {
- case ZERO:
-
- // The epoch.
- calendar.set(Calendar.HOUR_OF_DAY, 0);
- calendar.set(Calendar.MINUTE, 0);
- calendar.set(Calendar.SECOND, 0);
- calendar.set(Calendar.MILLISECOND, 0);
- break;
- case UNDERFLOW:
- return null;
- case OVERFLOW:
- calendar.set(Calendar.HOUR_OF_DAY, 23);
- calendar.set(Calendar.MINUTE, 59);
- calendar.set(Calendar.SECOND, 59);
- int millis =
- (precision >= 3)
- ? 999
- : ((precision == 2) ? 990 : ((precision == 1) ? 900 : 0));
- calendar.set(Calendar.MILLISECOND, millis);
- break;
- default:
- break;
- }
- return calendar;
-
- case TIMESTAMP:
- calendar = Util.calendar();
- switch (limit) {
- case ZERO:
- // The epoch.
- calendar.set(Calendar.YEAR, 1970);
- calendar.set(Calendar.MONTH, 0);
- calendar.set(Calendar.DAY_OF_MONTH, 1);
- calendar.set(Calendar.HOUR_OF_DAY, 0);
- calendar.set(Calendar.MINUTE, 0);
- calendar.set(Calendar.SECOND, 0);
- calendar.set(Calendar.MILLISECOND, 0);
- break;
- case UNDERFLOW:
- return null;
- case OVERFLOW:
- if (beyond) {
- // It is impossible to represent an invalid year as a date
- // literal. SQL dates are represented as 'yyyy-mm-dd', and
- // 1 <= yyyy <= 9999 is valid. There is no year 0: the year
- // before 1AD is 1BC, so SimpleDateFormat renders the day
- // before 0001-01-01 (AD) as 0001-12-31 (BC), which looks
- // like a valid date.
- return null;
- }
-
- // "SQL:2003 6.1 Access Rules 6" says that year is
- // between 1 and 9999, and days/months are the valid Gregorian
- // calendar values for these years.
- if (sign) {
- calendar.set(Calendar.YEAR, 9999);
- calendar.set(Calendar.MONTH, 11);
- calendar.set(Calendar.DAY_OF_MONTH, 31);
- calendar.set(Calendar.HOUR_OF_DAY, 23);
- calendar.set(Calendar.MINUTE, 59);
- calendar.set(Calendar.SECOND, 59);
- int millis =
- (precision >= 3)
- ? 999
- : ((precision == 2)
- ? 990
- : ((precision == 1) ? 900 : 0));
- calendar.set(Calendar.MILLISECOND, millis);
- } else {
- calendar.set(Calendar.YEAR, 1);
- calendar.set(Calendar.MONTH, 0);
- calendar.set(Calendar.DAY_OF_MONTH, 1);
- calendar.set(Calendar.HOUR_OF_DAY, 0);
- calendar.set(Calendar.MINUTE, 0);
- calendar.set(Calendar.SECOND, 0);
- calendar.set(Calendar.MILLISECOND, 0);
- }
- break;
- default:
- break;
- }
- return calendar;
-
- default:
- throw Util.unexpected(this);
- }
- }
-
- /**
- * Returns the minimum precision (or length) allowed for this type, or -1 if precision/length
- * are not applicable for this type.
- *
- * @return Minimum allowed precision
- */
- public int getMinPrecision() {
- switch (this) {
- case DECIMAL:
- case VARCHAR:
- case CHAR:
- case VARBINARY:
- case BINARY:
- case TIME:
- case TIME_WITH_LOCAL_TIME_ZONE:
- case TIME_TZ:
- case TIMESTAMP:
- case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
- case TIMESTAMP_TZ:
- return 1;
- case INTERVAL_YEAR:
- case INTERVAL_YEAR_MONTH:
- case INTERVAL_MONTH:
- case INTERVAL_DAY:
- case INTERVAL_DAY_HOUR:
- case INTERVAL_DAY_MINUTE:
- case INTERVAL_DAY_SECOND:
- case INTERVAL_HOUR:
- case INTERVAL_HOUR_MINUTE:
- case INTERVAL_HOUR_SECOND:
- case INTERVAL_MINUTE:
- case INTERVAL_MINUTE_SECOND:
- case INTERVAL_SECOND:
- return MIN_INTERVAL_START_PRECISION;
- default:
- return -1;
- }
- }
-
- /**
- * Returns the minimum scale (or fractional second precision in the case of intervals) allowed
- * for this type, or -1 if precision/length are not applicable for this type.
- *
- * @return Minimum allowed scale
- */
- public int getMinScale() {
- switch (this) {
- // TODO: Minimum numeric scale for decimal
- case INTERVAL_YEAR:
- case INTERVAL_YEAR_MONTH:
- case INTERVAL_MONTH:
- case INTERVAL_DAY:
- case INTERVAL_DAY_HOUR:
- case INTERVAL_DAY_MINUTE:
- case INTERVAL_DAY_SECOND:
- case INTERVAL_HOUR:
- case INTERVAL_HOUR_MINUTE:
- case INTERVAL_HOUR_SECOND:
- case INTERVAL_MINUTE:
- case INTERVAL_MINUTE_SECOND:
- case INTERVAL_SECOND:
- return MIN_INTERVAL_FRACTIONAL_SECOND_PRECISION;
- default:
- return -1;
- }
- }
-
- /**
- * Returns {@code HOUR} for {@code HOUR TO SECOND} and {@code HOUR}, {@code SECOND} for {@code
- * SECOND}.
- */
- public TimeUnit getStartUnit() {
- switch (this) {
- case INTERVAL_YEAR:
- case INTERVAL_YEAR_MONTH:
- return TimeUnit.YEAR;
- case INTERVAL_MONTH:
- return TimeUnit.MONTH;
- case INTERVAL_DAY:
- case INTERVAL_DAY_HOUR:
- case INTERVAL_DAY_MINUTE:
- case INTERVAL_DAY_SECOND:
- return TimeUnit.DAY;
- case INTERVAL_HOUR:
- case INTERVAL_HOUR_MINUTE:
- case INTERVAL_HOUR_SECOND:
- return TimeUnit.HOUR;
- case INTERVAL_MINUTE:
- case INTERVAL_MINUTE_SECOND:
- return TimeUnit.MINUTE;
- case INTERVAL_SECOND:
- return TimeUnit.SECOND;
- default:
- throw new AssertionError(this);
- }
- }
-
- /** Returns {@code SECOND} for both {@code HOUR TO SECOND} and {@code SECOND}. */
- public TimeUnit getEndUnit() {
- switch (this) {
- case INTERVAL_YEAR:
- return TimeUnit.YEAR;
- case INTERVAL_YEAR_MONTH:
- case INTERVAL_MONTH:
- return TimeUnit.MONTH;
- case INTERVAL_DAY:
- return TimeUnit.DAY;
- case INTERVAL_DAY_HOUR:
- case INTERVAL_HOUR:
- return TimeUnit.HOUR;
- case INTERVAL_DAY_MINUTE:
- case INTERVAL_HOUR_MINUTE:
- case INTERVAL_MINUTE:
- return TimeUnit.MINUTE;
- case INTERVAL_DAY_SECOND:
- case INTERVAL_HOUR_SECOND:
- case INTERVAL_MINUTE_SECOND:
- case INTERVAL_SECOND:
- return TimeUnit.SECOND;
- default:
- throw new AssertionError(this);
- }
- }
-
- public boolean isYearMonth() {
- switch (this) {
- case INTERVAL_YEAR:
- case INTERVAL_YEAR_MONTH:
- case INTERVAL_MONTH:
- return true;
- default:
- return false;
- }
- }
-
- /** Limit. */
- public enum Limit {
- ZERO,
- UNDERFLOW,
- OVERFLOW
- }
-
- private static @Nullable BigDecimal getNumericLimit(
- int radix, int exponent, boolean sign, Limit limit, boolean beyond) {
- switch (limit) {
- case OVERFLOW:
-
- // 2-based schemes run from -2^(N-1) to 2^(N-1)-1 e.g. -128 to +127
- // 10-based schemas run from -(10^N-1) to 10^N-1 e.g. -99 to +99
- final BigDecimal bigRadix = BigDecimal.valueOf(radix);
- if (radix == 2) {
- --exponent;
- }
- BigDecimal decimal = bigRadix.pow(exponent);
- if (sign || (radix != 2)) {
- decimal = decimal.subtract(BigDecimal.ONE);
- }
- if (beyond) {
- decimal = decimal.add(BigDecimal.ONE);
- }
- if (!sign) {
- decimal = decimal.negate();
- }
- return decimal;
- case UNDERFLOW:
- return beyond ? null : (sign ? BigDecimal.ONE : BigDecimal.ONE.negate());
- case ZERO:
- return BigDecimal.ZERO;
- default:
- throw Util.unexpected(limit);
- }
- }
-
- public SqlLiteral createLiteral(Object o, SqlParserPos pos) {
- switch (this) {
- case BOOLEAN:
- return SqlLiteral.createBoolean((Boolean) o, pos);
- case TINYINT:
- case SMALLINT:
- case INTEGER:
- case BIGINT:
- case DECIMAL:
- return SqlLiteral.createExactNumeric(o.toString(), pos);
- case VARCHAR:
- case CHAR:
- return SqlLiteral.createCharString((String) o, pos);
- case VARBINARY:
- case BINARY:
- return SqlLiteral.createBinaryString((byte[]) o, pos);
- case DATE:
- return SqlLiteral.createDate(
- o instanceof Calendar
- ? DateString.fromCalendarFields((Calendar) o)
- : (DateString) o,
- pos);
- case TIME:
- return SqlLiteral.createTime(
- o instanceof Calendar
- ? TimeString.fromCalendarFields((Calendar) o)
- : (TimeString) o,
- 0 /* todo */,
- pos);
- case TIMESTAMP:
- return SqlLiteral.createTimestamp(
- this,
- o instanceof Calendar
- ? TimestampString.fromCalendarFields((Calendar) o)
- : (TimestampString) o,
- 0 /* todo */,
- pos);
- default:
- throw Util.unexpected(this);
- }
- }
-
- /** Returns the name of this type. */
- public String getName() {
- return name();
- }
-
- /**
- * Returns the name of this type, with underscores converted to spaces, for example "TIMESTAMP
- * WITH LOCAL TIME ZONE", "DATE".
- */
- public String getSpaceName() {
- return name().replace('_', ' ');
- }
-
- /**
- * Flags indicating precision/scale combinations.
- *
- * Note: for intervals:
- *
- *
- * - precision = start (leading field) precision
- *
- scale = fractional second precision
- *
- */
- private interface PrecScale {
- int NO_NO = 1;
- int YES_NO = 2;
- int YES_YES = 4;
- }
-}
diff --git a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/validate/FlinkSqlConformance.java b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/validate/FlinkSqlConformance.java
index e00def6f715ab..ccc1ef14f605a 100644
--- a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/validate/FlinkSqlConformance.java
+++ b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/validate/FlinkSqlConformance.java
@@ -42,6 +42,16 @@ public boolean isGroupByAlias() {
return false;
}
+ @Override
+ public SelectAliasLookup isSelectAlias() {
+ return SqlConformanceEnum.DEFAULT.isSelectAlias();
+ }
+
+ @Override
+ public boolean isNonStrictGroupBy() {
+ return false;
+ }
+
@Override
public boolean isGroupByOrdinal() {
return false;
@@ -67,6 +77,11 @@ public boolean isSortByAliasObscures() {
return false;
}
+ @Override
+ public boolean isSupportedDualTable() {
+ return SqlConformanceEnum.DEFAULT.isSupportedDualTable();
+ }
+
@Override
public boolean isFromRequired() {
return false;
@@ -97,11 +112,6 @@ public boolean isMinusAllowed() {
return false;
}
- @Override
- public boolean isRegexReplaceCaptureGroupDollarIndexed() {
- return true;
- }
-
@Override
public boolean isApplyAllowed() {
return false;
@@ -122,6 +132,11 @@ public boolean allowNiladicParentheses() {
return true;
}
+ @Override
+ public boolean allowNiladicConstantWithoutParentheses() {
+ return true;
+ }
+
@Override
public boolean allowExplicitRowValueConstructor() {
return true;
@@ -177,6 +192,16 @@ public boolean allowLenientCoercion() {
return SqlConformanceEnum.DEFAULT.allowLenientCoercion();
}
+ @Override
+ public boolean checkedArithmetic() {
+ return SqlConformanceEnum.DEFAULT.checkedArithmetic();
+ }
+
+ @Override
+ public boolean supportsUnsignedTypes() {
+ return false;
+ }
+
@Override
public boolean isValueAllowed() {
return true;
diff --git a/flink-table/flink-sql-parser/src/test/java/org/apache/flink/sql/parser/FlinkSqlParserImplTest.java b/flink-table/flink-sql-parser/src/test/java/org/apache/flink/sql/parser/FlinkSqlParserImplTest.java
index 1d1e7d639fa00..b5ea58c8ea2d2 100644
--- a/flink-table/flink-sql-parser/src/test/java/org/apache/flink/sql/parser/FlinkSqlParserImplTest.java
+++ b/flink-table/flink-sql-parser/src/test/java/org/apache/flink/sql/parser/FlinkSqlParserImplTest.java
@@ -73,6 +73,9 @@ void testMapQueryConstructor() {}
@Test
void testMultisetQueryConstructor() {}
+ @Test
+ void testMeasure() {}
+
@Test
void testShowCatalogs() {
sql("show catalogs").ok("SHOW CATALOGS");
diff --git a/flink-table/flink-table-calcite-bridge/pom.xml b/flink-table/flink-table-calcite-bridge/pom.xml
index 2de3dced880a1..77ee0a2fd4950 100644
--- a/flink-table/flink-table-calcite-bridge/pom.xml
+++ b/flink-table/flink-table-calcite-bridge/pom.xml
@@ -45,17 +45,18 @@ under the License.
${calcite.version}
@@ -371,6 +377,7 @@ under the License.
commons-io:commons-io
org.apache.commons:commons-math3
org.checkerframework:checker-qual
+ org.jooq:joou-java-6
org.apache.flink:flink-sql-parser
@@ -425,6 +432,12 @@ under the License.
com.ibm.icu
org.apache.flink.table.shaded.com.ibm.icu
+
+
+
+ org.jooq
+ org.apache.flink.table.shaded.org.jooq
+
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/jdbc/SimpleCalciteSchema.java b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/jdbc/SimpleCalciteSchema.java
index 864d50ef5c9b9..d04374dc546b8 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/jdbc/SimpleCalciteSchema.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/jdbc/SimpleCalciteSchema.java
@@ -133,38 +133,8 @@ public CalciteSchema add(String name, Schema schema) {
}
@Override
- protected @Nullable CalciteSchema getImplicitSubSchema(
- String schemaName, boolean caseSensitive) {
- // Check implicit schemas.
- final String schemaName2 =
- caseSensitive
- ? schemaName
- : caseInsensitiveLookup(schema.getSubSchemaNames(), schemaName);
- if (schemaName2 == null) {
- return null;
- }
- final Schema s = schema.getSubSchema(schemaName2);
- if (s == null) {
- return null;
- }
- return new SimpleCalciteSchema(this, s, schemaName2);
- }
-
- @Override
- protected @Nullable TableEntry getImplicitTable(String tableName, boolean caseSensitive) {
- // Check implicit tables.
- final String tableName2 =
- caseSensitive
- ? tableName
- : caseInsensitiveLookup(schema.getTableNames(), tableName);
- if (tableName2 == null) {
- return null;
- }
- final Table table = schema.getTable(tableName2);
- if (table == null) {
- return null;
- }
- return tableEntry(tableName2, table);
+ protected CalciteSchema createSubSchema(Schema schema, String name) {
+ return new SimpleCalciteSchema(this, schema, name);
}
// ----- FLINK MODIFICATION START -----
@@ -201,28 +171,6 @@ public CalciteSchema add(String name, Schema schema) {
return typeEntry(name2, type);
}
- @Override
- protected void addImplicitSubSchemaToBuilder(
- ImmutableSortedMap.Builder builder) {
- ImmutableSortedMap explicitSubSchemas = builder.build();
- for (String schemaName : schema.getSubSchemaNames()) {
- if (explicitSubSchemas.containsKey(schemaName)) {
- // explicit subschema wins.
- continue;
- }
- Schema s = schema.getSubSchema(schemaName);
- if (s != null) {
- CalciteSchema calciteSchema = new SimpleCalciteSchema(this, s, schemaName);
- builder.put(schemaName, calciteSchema);
- }
- }
- }
-
- @Override
- protected void addImplicitTableToBuilder(ImmutableSortedSet.Builder builder) {
- builder.addAll(schema.getTableNames());
- }
-
@Override
protected void addImplicitFunctionsToBuilder(
ImmutableList.Builder builder, String name, boolean caseSensitive) {
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/plan/RelOptUtil.java b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/plan/RelOptUtil.java
deleted file mode 100644
index 05fdcfc7ed589..0000000000000
--- a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/plan/RelOptUtil.java
+++ /dev/null
@@ -1,4567 +0,0 @@
-/*
- * 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.plan;
-
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Iterables;
-import com.google.common.collect.LinkedHashMultimap;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Multimap;
-import org.apache.calcite.adapter.enumerable.EnumerableRules;
-import org.apache.calcite.avatica.AvaticaConnection;
-import org.apache.calcite.config.CalciteSystemProperty;
-import org.apache.calcite.interpreter.Bindables;
-import org.apache.calcite.linq4j.Ord;
-import org.apache.calcite.linq4j.function.Experimental;
-import org.apache.calcite.rel.RelCollations;
-import org.apache.calcite.rel.RelHomogeneousShuttle;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.RelShuttle;
-import org.apache.calcite.rel.RelVisitor;
-import org.apache.calcite.rel.RelWriter;
-import org.apache.calcite.rel.core.Aggregate;
-import org.apache.calcite.rel.core.AggregateCall;
-import org.apache.calcite.rel.core.Calc;
-import org.apache.calcite.rel.core.CorrelationId;
-import org.apache.calcite.rel.core.Filter;
-import org.apache.calcite.rel.core.Join;
-import org.apache.calcite.rel.core.JoinRelType;
-import org.apache.calcite.rel.core.Project;
-import org.apache.calcite.rel.core.RelFactories;
-import org.apache.calcite.rel.core.Sort;
-import org.apache.calcite.rel.core.TableScan;
-import org.apache.calcite.rel.externalize.RelDotWriter;
-import org.apache.calcite.rel.externalize.RelJsonWriter;
-import org.apache.calcite.rel.externalize.RelWriterImpl;
-import org.apache.calcite.rel.externalize.RelXmlWriter;
-import org.apache.calcite.rel.hint.HintStrategyTable;
-import org.apache.calcite.rel.hint.Hintable;
-import org.apache.calcite.rel.hint.RelHint;
-import org.apache.calcite.rel.logical.LogicalAggregate;
-import org.apache.calcite.rel.logical.LogicalCalc;
-import org.apache.calcite.rel.logical.LogicalFilter;
-import org.apache.calcite.rel.logical.LogicalJoin;
-import org.apache.calcite.rel.logical.LogicalProject;
-import org.apache.calcite.rel.metadata.RelMetadataQuery;
-import org.apache.calcite.rel.rules.CoreRules;
-import org.apache.calcite.rel.rules.MultiJoin;
-import org.apache.calcite.rel.stream.StreamRules;
-import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rel.type.RelDataTypeFactory;
-import org.apache.calcite.rel.type.RelDataTypeField;
-import org.apache.calcite.rel.type.RelDataTypeFieldImpl;
-import org.apache.calcite.rex.LogicVisitor;
-import org.apache.calcite.rex.RexBuilder;
-import org.apache.calcite.rex.RexCall;
-import org.apache.calcite.rex.RexCorrelVariable;
-import org.apache.calcite.rex.RexExecutor;
-import org.apache.calcite.rex.RexExecutorImpl;
-import org.apache.calcite.rex.RexFieldAccess;
-import org.apache.calcite.rex.RexInputRef;
-import org.apache.calcite.rex.RexLiteral;
-import org.apache.calcite.rex.RexLocalRef;
-import org.apache.calcite.rex.RexNode;
-import org.apache.calcite.rex.RexOver;
-import org.apache.calcite.rex.RexProgram;
-import org.apache.calcite.rex.RexShuttle;
-import org.apache.calcite.rex.RexSqlStandardConvertletTable;
-import org.apache.calcite.rex.RexSubQuery;
-import org.apache.calcite.rex.RexToSqlNodeConverter;
-import org.apache.calcite.rex.RexToSqlNodeConverterImpl;
-import org.apache.calcite.rex.RexUtil;
-import org.apache.calcite.rex.RexVisitorImpl;
-import org.apache.calcite.runtime.CalciteContextException;
-import org.apache.calcite.runtime.PairList;
-import org.apache.calcite.schema.ModifiableView;
-import org.apache.calcite.sql.SqlExplainFormat;
-import org.apache.calcite.sql.SqlExplainLevel;
-import org.apache.calcite.sql.SqlKind;
-import org.apache.calcite.sql.SqlLiteral;
-import org.apache.calcite.sql.SqlNode;
-import org.apache.calcite.sql.SqlOperator;
-import org.apache.calcite.sql.fun.SqlStdOperatorTable;
-import org.apache.calcite.sql.type.MultisetSqlType;
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.apache.calcite.sql.type.SqlTypeUtil;
-import org.apache.calcite.tools.RelBuilder;
-import org.apache.calcite.tools.RelBuilderFactory;
-import org.apache.calcite.util.ImmutableBitSet;
-import org.apache.calcite.util.Litmus;
-import org.apache.calcite.util.Pair;
-import org.apache.calcite.util.Permutation;
-import org.apache.calcite.util.Util;
-import org.apache.calcite.util.mapping.Mapping;
-import org.apache.calcite.util.mapping.MappingType;
-import org.apache.calcite.util.mapping.Mappings;
-import org.checkerframework.checker.initialization.qual.NotOnlyInitialized;
-import org.checkerframework.checker.initialization.qual.UnknownInitialization;
-import org.checkerframework.checker.nullness.qual.Nullable;
-import org.checkerframework.checker.nullness.qual.PolyNull;
-
-import java.io.PrintWriter;
-import java.io.StringWriter;
-import java.util.AbstractList;
-import java.util.ArrayDeque;
-import java.util.ArrayList;
-import java.util.BitSet;
-import java.util.Collection;
-import java.util.Comparator;
-import java.util.Deque;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.LinkedHashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.NavigableSet;
-import java.util.Set;
-import java.util.TreeSet;
-import java.util.function.Supplier;
-import java.util.stream.Collectors;
-
-import static java.util.Objects.requireNonNull;
-import static org.apache.calcite.rel.type.RelDataTypeImpl.NON_NULLABLE_SUFFIX;
-
-/**
- * RelOptUtil defines static utility methods for use in optimizing {@link RelNode}s.
- *
- * FLINK modifications (backport of CALCITE-6764): Lines 2074 ~ 2106
- */
-public abstract class RelOptUtil {
- // ~ Static fields/initializers ---------------------------------------------
-
- public static final double EPSILON = 1.0e-5;
-
- @SuppressWarnings("Guava")
- @Deprecated // to be removed before 2.0
- public static final com.google.common.base.Predicate FILTER_PREDICATE =
- f -> !f.containsOver();
-
- @SuppressWarnings("Guava")
- @Deprecated // to be removed before 2.0
- public static final com.google.common.base.Predicate PROJECT_PREDICATE =
- RelOptUtil::notContainsWindowedAgg;
-
- @SuppressWarnings("Guava")
- @Deprecated // to be removed before 2.0
- public static final com.google.common.base.Predicate CALC_PREDICATE =
- RelOptUtil::notContainsWindowedAgg;
-
- // ~ Methods ----------------------------------------------------------------
-
- /** Whether this node is a limit without sort specification. */
- public static boolean isPureLimit(RelNode rel) {
- return isLimit(rel) && !isOrder(rel);
- }
-
- /** Whether this node is a sort without limit specification. */
- public static boolean isPureOrder(RelNode rel) {
- return !isLimit(rel) && isOrder(rel);
- }
-
- /** Whether this node contains a limit specification. */
- public static boolean isLimit(RelNode rel) {
- return (rel instanceof Sort) && ((Sort) rel).fetch != null;
- }
-
- /** Whether this node contains a sort specification. */
- public static boolean isOrder(RelNode rel) {
- return (rel instanceof Sort) && !((Sort) rel).getCollation().getFieldCollations().isEmpty();
- }
-
- /** Whether this node contains a offset specification. */
- public static boolean isOffset(RelNode rel) {
- return (rel instanceof Sort) && ((Sort) rel).offset != null;
- }
-
- /** Returns a set of tables used by this expression or its children. */
- public static Set findTables(RelNode rel) {
- return new LinkedHashSet<>(findAllTables(rel));
- }
-
- /** Returns a list of all tables used by this expression or its children. */
- public static List findAllTables(RelNode rel) {
- final Multimap, RelNode> nodes =
- rel.getCluster().getMetadataQuery().getNodeTypes(rel);
- final List usedTables = new ArrayList<>();
- if (nodes == null) {
- return usedTables;
- }
- for (Map.Entry, Collection> e :
- nodes.asMap().entrySet()) {
- if (TableScan.class.isAssignableFrom(e.getKey())) {
- for (RelNode node : e.getValue()) {
- TableScan scan = (TableScan) node;
- usedTables.add(scan.getTable());
- }
- }
- }
- return usedTables;
- }
-
- /** Returns a list of all table qualified names used by this expression or its children. */
- public static List findAllTableQualifiedNames(RelNode rel) {
- return findAllTables(rel).stream()
- .map(table -> table.getQualifiedName().toString())
- .collect(Collectors.toList());
- }
-
- /** Returns a list of variables set by a relational expression or its descendants. */
- public static Set getVariablesSet(RelNode rel) {
- VariableSetVisitor visitor = new VariableSetVisitor();
- go(visitor, rel);
- return visitor.variables;
- }
-
- @Deprecated // to be removed before 2.0
- @SuppressWarnings("MixedMutabilityReturnType")
- public static List getVariablesSetAndUsed(RelNode rel0, RelNode rel1) {
- Set set = getVariablesSet(rel0);
- if (set.size() == 0) {
- return ImmutableList.of();
- }
- Set used = getVariablesUsed(rel1);
- if (used.size() == 0) {
- return ImmutableList.of();
- }
- final List result = new ArrayList<>();
- for (CorrelationId s : set) {
- if (used.contains(s) && !result.contains(s)) {
- result.add(s);
- }
- }
- return result;
- }
-
- /**
- * Returns the set of variables used by a relational expression or its descendants.
- *
- * The set may contain "duplicates" (variables with different ids that, when resolved, will
- * reference the same source relational expression).
- *
- *
The item type is the same as {@link org.apache.calcite.rex.RexCorrelVariable#id}.
- */
- public static Set getVariablesUsed(RelNode rel) {
- CorrelationCollector visitor = new CorrelationCollector();
- rel.accept(visitor);
- return visitor.vuv.variables;
- }
-
- /**
- * Returns the set of variables used by the given list of sub-queries and its descendants.
- *
- * @param subQueries The sub-queries containing correlation variables
- * @return A list of correlation identifiers found within the sub-queries. The type of the
- * [CorrelationId] parameter corresponds to {@link
- * org.apache.calcite.rex.RexCorrelVariable#id}.
- */
- public static Set getVariablesUsed(List subQueries) {
- // Internally this function calls getVariablesUsed on a RelNode to get all the
- // correlated variables in that RelNode
- Set correlationIds = new HashSet<>();
- for (RexSubQuery subQ : subQueries) {
- correlationIds.addAll(getVariablesUsed(subQ.rel));
- }
- return correlationIds;
- }
-
- /** Finds which columns of a correlation variable are used within a relational expression. */
- public static ImmutableBitSet correlationColumns(CorrelationId id, RelNode rel) {
- final CorrelationCollector collector = new CorrelationCollector();
- rel.accept(collector);
- final ImmutableBitSet.Builder builder = ImmutableBitSet.builder();
- for (int field : collector.vuv.variableFields.get(id)) {
- if (field >= 0) {
- builder.set(field);
- }
- }
- return builder.build();
- }
-
- /**
- * Returns true, and calls {@link Litmus#succeed()} if a given relational expression does not
- * contain a given correlation.
- */
- public static boolean notContainsCorrelation(
- RelNode r, CorrelationId correlationId, Litmus litmus) {
- final Set set = getVariablesUsed(r);
- if (!set.contains(correlationId)) {
- return litmus.succeed();
- } else {
- return litmus.fail("contains {}", correlationId);
- }
- }
-
- /** Sets a {@link RelVisitor} going on a given relational expression, and returns the result. */
- public static void go(RelVisitor visitor, RelNode p) {
- try {
- visitor.go(p);
- } catch (Exception e) {
- throw new RuntimeException("while visiting tree", e);
- }
- }
-
- /**
- * Returns a list of the types of the fields in a given struct type. The list is immutable.
- *
- * @param type Struct type
- * @return List of field types
- * @see org.apache.calcite.rel.type.RelDataType#getFieldNames()
- */
- public static List getFieldTypeList(final RelDataType type) {
- return Util.transform(type.getFieldList(), RelDataTypeField::getType);
- }
-
- public static boolean areRowTypesEqual(
- RelDataType rowType1, RelDataType rowType2, boolean compareNames) {
- if (rowType1 == rowType2) {
- return true;
- }
- if (compareNames) {
- // if types are not identity-equal, then either the names or
- // the types must be different
- return false;
- }
- if (rowType2.getFieldCount() != rowType1.getFieldCount()) {
- return false;
- }
- final List f1 = rowType1.getFieldList();
- final List f2 = rowType2.getFieldList();
- for (Pair pair : Pair.zip(f1, f2)) {
- final RelDataType type1 = pair.left.getType();
- final RelDataType type2 = pair.right.getType();
- // If one of the types is ANY comparison should succeed
- if (type1.getSqlTypeName() == SqlTypeName.ANY
- || type2.getSqlTypeName() == SqlTypeName.ANY) {
- continue;
- }
- if (!type1.equals(type2)) {
- return false;
- }
- }
- return true;
- }
-
- /**
- * Verifies that a row type being added to an equivalence class matches the existing type,
- * raising an assertion if this is not the case.
- *
- * @param originalRel canonical rel for equivalence class
- * @param newRel rel being added to equivalence class
- * @param equivalenceClass object representing equivalence class
- */
- public static void verifyTypeEquivalence(
- RelNode originalRel, RelNode newRel, Object equivalenceClass) {
- RelDataType expectedRowType = originalRel.getRowType();
- RelDataType actualRowType = newRel.getRowType();
-
- // Row types must be the same, except for field names.
- if (areRowTypesEqual(expectedRowType, actualRowType, false)) {
- return;
- }
-
- String s =
- "Cannot add expression of different type to set:\n"
- + "set type is "
- + expectedRowType.getFullTypeString()
- + "\nexpression type is "
- + actualRowType.getFullTypeString()
- + "\nset is "
- + equivalenceClass.toString()
- + "\nexpression is "
- + RelOptUtil.toString(newRel)
- + getFullTypeDifferenceString(
- "rowtype of original rel",
- expectedRowType,
- "rowtype of new rel",
- actualRowType);
- throw new AssertionError(s);
- }
-
- /**
- * Copy the {@link org.apache.calcite.rel.hint.RelHint}s from {@code originalRel} to {@code
- * newRel} if both of them are {@link Hintable}.
- *
- * The two relational expressions are assumed as semantically equivalent, that means the
- * hints should be attached to the relational expression that expects to have them.
- *
- *
Try to propagate the hints to the first relational expression that matches, this is needed
- * because many planner rules would generate a sub-tree whose root rel type is different with
- * the original matched rel.
- *
- *
For the worst case, there is no relational expression that can apply these hints, and the
- * whole sub-tree would be visited. We add a protection here: if the visiting depth is over than
- * 3, just returns, because there are rare cases the new created sub-tree has layers bigger than
- * that.
- *
- *
This is a best effort, we do not know exactly how the nodes are transformed in all kinds
- * of planner rules, so for some complex relational expressions, the hints would very probably
- * lost.
- *
- *
This function is experimental and would change without any notes.
- *
- * @param originalRel Original relational expression
- * @param equiv New equivalent relational expression
- * @return A copy of {@code newRel} with attached qualified hints from {@code originalRel}, or
- * {@code newRel} directly if one of them are not {@link Hintable}
- */
- @Experimental
- public static RelNode propagateRelHints(RelNode originalRel, RelNode equiv) {
- if (!(originalRel instanceof Hintable) || ((Hintable) originalRel).getHints().size() == 0) {
- return equiv;
- }
- final RelShuttle shuttle =
- new SubTreeHintPropagateShuttle(
- originalRel.getCluster().getHintStrategies(),
- ((Hintable) originalRel).getHints());
- return equiv.accept(shuttle);
- }
-
- /**
- * Propagates the relational expression hints from root node to leaf node.
- *
- * @param rel The relational expression
- * @param reset Flag saying if to reset the existing hints before the propagation
- * @return New relational expression with hints propagated
- */
- public static RelNode propagateRelHints(RelNode rel, boolean reset) {
- if (reset) {
- rel = rel.accept(new ResetHintsShuttle());
- }
- final RelShuttle shuttle =
- new RelHintPropagateShuttle(rel.getCluster().getHintStrategies());
- return rel.accept(shuttle);
- }
-
- /**
- * Copy the {@link org.apache.calcite.rel.hint.RelHint}s from {@code originalRel} to {@code
- * newRel} if both of them are {@link Hintable}.
- *
- *
The hints would be attached directly(e.g. without any filtering).
- *
- * @param originalRel Original relational expression
- * @param newRel New relational expression
- * @return A copy of {@code newRel} with attached hints from {@code originalRel}, or {@code
- * newRel} directly if one of them are not {@link Hintable}
- */
- public static RelNode copyRelHints(RelNode originalRel, RelNode newRel) {
- return copyRelHints(originalRel, newRel, false);
- }
-
- /**
- * Copy the {@link org.apache.calcite.rel.hint.RelHint}s from {@code originalRel} to {@code
- * newRel} if both of them are {@link Hintable}.
- *
- *
The hints would be filtered by the specified hint strategies if {@code filterHints} is
- * true.
- *
- * @param originalRel Original relational expression
- * @param newRel New relational expression
- * @param filterHints Flag saying if to filter out unqualified hints for {@code newRel}
- * @return A copy of {@code newRel} with attached hints from {@code originalRel}, or {@code
- * newRel} directly if one of them are not {@link Hintable}
- */
- public static RelNode copyRelHints(RelNode originalRel, RelNode newRel, boolean filterHints) {
- if (originalRel == newRel && !filterHints) {
- return originalRel;
- }
-
- if (originalRel instanceof Hintable
- && newRel instanceof Hintable
- && ((Hintable) originalRel).getHints().size() > 0) {
- final List hints = ((Hintable) originalRel).getHints();
- if (filterHints) {
- HintStrategyTable hintStrategies = originalRel.getCluster().getHintStrategies();
- return ((Hintable) newRel).attachHints(hintStrategies.apply(hints, newRel));
- } else {
- // Keep all the hints if filterHints is false for 2 reasons:
- // 1. Keep sync with the hints propagation logic,
- // see RelHintPropagateShuttle for details.
- // 2. We may re-propagate these hints when decorrelating a query.
- return ((Hintable) newRel).attachHints(hints);
- }
- }
- return newRel;
- }
-
- /**
- * Returns a permutation describing where output fields come from. In the returned map, value of
- * {@code map.getTargetOpt(i)} is {@code n} if field {@code i} projects input field {@code n} or
- * applies a cast on {@code n}, -1 if it is another expression.
- */
- public static Mappings.TargetMapping permutationIgnoreCast(
- List nodes, RelDataType inputRowType) {
- final Mappings.TargetMapping mapping =
- Mappings.create(
- MappingType.PARTIAL_FUNCTION, nodes.size(), inputRowType.getFieldCount());
- for (Ord node : Ord.zip(nodes)) {
- if (node.e instanceof RexInputRef) {
- mapping.set(node.i, ((RexInputRef) node.e).getIndex());
- } else if (node.e.isA(SqlKind.CAST)) {
- final RexNode operand = ((RexCall) node.e).getOperands().get(0);
- if (operand instanceof RexInputRef) {
- mapping.set(node.i, ((RexInputRef) operand).getIndex());
- }
- }
- }
- return mapping;
- }
-
- /**
- * Returns a permutation describing where output fields come from. In the returned map, value of
- * {@code map.getTargetOpt(i)} is {@code n} if field {@code i} projects input field {@code n},
- * -1 if it is an expression.
- */
- public static Mappings.TargetMapping permutation(
- List nodes, RelDataType inputRowType) {
- final Mappings.TargetMapping mapping =
- Mappings.create(
- MappingType.PARTIAL_FUNCTION, nodes.size(), inputRowType.getFieldCount());
- for (Ord node : Ord.zip(nodes)) {
- if (node.e instanceof RexInputRef) {
- mapping.set(node.i, ((RexInputRef) node.e).getIndex());
- }
- }
- return mapping;
- }
-
- /**
- * Returns a permutation describing where the Project's fields come from after the Project is
- * pushed down.
- */
- public static Mappings.TargetMapping permutationPushDownProject(
- List nodes, RelDataType inputRowType, int sourceOffset, int targetOffset) {
- final Mappings.TargetMapping mapping =
- Mappings.create(
- MappingType.PARTIAL_FUNCTION,
- inputRowType.getFieldCount() + sourceOffset,
- nodes.size() + targetOffset);
- for (Ord node : Ord.zip(nodes)) {
- if (node.e instanceof RexInputRef) {
- mapping.set(
- ((RexInputRef) node.e).getIndex() + sourceOffset, node.i + targetOffset);
- }
- }
- return mapping;
- }
-
- @Deprecated // to be removed before 2.0
- public static RelNode createExistsPlan(
- RelOptCluster cluster,
- RelNode seekRel,
- @Nullable List conditions,
- @Nullable RexLiteral extraExpr,
- @Nullable String extraName) {
- assert extraExpr == null || extraName != null;
- RelNode ret = seekRel;
-
- if ((conditions != null) && (conditions.size() > 0)) {
- RexNode conditionExp =
- RexUtil.composeConjunction(cluster.getRexBuilder(), conditions, true);
-
- if (conditionExp != null) {
- final RelFactories.FilterFactory factory = RelFactories.DEFAULT_FILTER_FACTORY;
- ret = factory.createFilter(ret, conditionExp, ImmutableSet.of());
- }
- }
-
- if (extraExpr != null) {
- RexBuilder rexBuilder = cluster.getRexBuilder();
-
- assert extraExpr == rexBuilder.makeLiteral(true);
-
- // this should only be called for the exists case
- // first stick an Agg on top of the sub-query
- // agg does not like no agg functions so just pretend it is
- // doing a min(TRUE)
-
- final RelBuilder relBuilder = RelFactories.LOGICAL_BUILDER.create(cluster, null);
- ret =
- relBuilder
- .push(ret)
- .project(extraExpr)
- .aggregate(
- relBuilder.groupKey(),
- relBuilder.min(relBuilder.field(0)).as(extraName))
- .build();
- }
-
- return ret;
- }
-
- @Deprecated // to be removed before 2.0
- public static Exists createExistsPlan(
- RelNode seekRel, SubQueryType subQueryType, Logic logic, boolean notIn) {
- final RelBuilder relBuilder =
- RelFactories.LOGICAL_BUILDER.create(seekRel.getCluster(), null);
- return createExistsPlan(seekRel, subQueryType, logic, notIn, relBuilder);
- }
-
- /**
- * Creates a plan suitable for use in EXISTS or IN statements.
- *
- * @see org.apache.calcite.sql2rel.SqlToRelConverter SqlToRelConverter#convertExists
- * @param seekRel A query rel, for example the resulting rel from 'select * from emp' or 'values
- * (1,2,3)' or '('Foo', 34)'.
- * @param subQueryType Sub-query type
- * @param logic Whether to use 2- or 3-valued boolean logic
- * @param notIn Whether the operator is NOT IN
- * @param relBuilder Builder for relational expressions
- * @return A pair of a relational expression which outer joins a boolean condition column, and a
- * numeric offset. The offset is 2 if column 0 is the number of rows and column 1 is the
- * number of rows with not-null keys; 0 otherwise.
- */
- public static Exists createExistsPlan(
- RelNode seekRel,
- SubQueryType subQueryType,
- Logic logic,
- boolean notIn,
- RelBuilder relBuilder) {
- switch (subQueryType) {
- case SCALAR:
- return new Exists(seekRel, false, true);
- default:
- break;
- }
-
- switch (logic) {
- case TRUE_FALSE_UNKNOWN:
- case UNKNOWN_AS_TRUE:
- if (notIn && !containsNullableFields(seekRel)) {
- logic = Logic.TRUE_FALSE;
- }
- break;
- default:
- break;
- }
- RelNode ret = seekRel;
- final RelOptCluster cluster = seekRel.getCluster();
- final RexBuilder rexBuilder = cluster.getRexBuilder();
- final int keyCount = ret.getRowType().getFieldCount();
- final boolean outerJoin = notIn || logic == RelOptUtil.Logic.TRUE_FALSE_UNKNOWN;
- if (!outerJoin) {
- final LogicalAggregate aggregate =
- LogicalAggregate.create(
- ret,
- ImmutableList.of(),
- ImmutableBitSet.range(keyCount),
- null,
- ImmutableList.of());
- return new Exists(aggregate, false, false);
- }
-
- // for IN/NOT IN, it needs to output the fields
- final List exprs = new ArrayList<>();
- if (subQueryType == SubQueryType.IN) {
- for (int i = 0; i < keyCount; i++) {
- exprs.add(rexBuilder.makeInputRef(ret, i));
- }
- }
-
- final int projectedKeyCount = exprs.size();
- exprs.add(rexBuilder.makeLiteral(true));
-
- ret =
- relBuilder
- .push(ret)
- .project(exprs)
- .aggregate(
- relBuilder.groupKey(ImmutableBitSet.range(projectedKeyCount)),
- relBuilder.min(relBuilder.field(projectedKeyCount)))
- .build();
-
- switch (logic) {
- case TRUE_FALSE_UNKNOWN:
- case UNKNOWN_AS_TRUE:
- return new Exists(ret, true, true);
- default:
- return new Exists(ret, false, true);
- }
- }
-
- @Deprecated // to be removed before 2.0
- public static RelNode createRenameRel(RelDataType outputType, RelNode rel) {
- RelDataType inputType = rel.getRowType();
- List inputFields = inputType.getFieldList();
- int n = inputFields.size();
-
- List outputFields = outputType.getFieldList();
- assert outputFields.size() == n
- : "rename: field count mismatch: in=" + inputType + ", out" + outputType;
-
- final PairList renames = PairList.of();
- final RexBuilder rexBuilder = rel.getCluster().getRexBuilder();
- Pair.forEach(
- inputFields,
- outputFields,
- (inputField, outputField) -> {
- assert inputField.getType().equals(outputField.getType());
- renames.add(
- rexBuilder.makeInputRef(inputField.getType(), inputField.getIndex()),
- outputField.getName());
- });
- final RelBuilder relBuilder = RelFactories.LOGICAL_BUILDER.create(rel.getCluster(), null);
- return relBuilder.push(rel).project(renames.leftList(), renames.rightList(), true).build();
- }
-
- @Deprecated // to be removed before 2.0
- public static RelNode createFilter(RelNode child, RexNode condition) {
- final RelFactories.FilterFactory factory = RelFactories.DEFAULT_FILTER_FACTORY;
- return factory.createFilter(child, condition, ImmutableSet.of());
- }
-
- @Deprecated // to be removed before 2.0
- public static RelNode createFilter(
- RelNode child, RexNode condition, RelFactories.FilterFactory filterFactory) {
- return filterFactory.createFilter(child, condition, ImmutableSet.of());
- }
-
- /**
- * Creates a filter, using the default filter factory, or returns the original relational
- * expression if the condition is trivial.
- */
- public static RelNode createFilter(RelNode child, Iterable extends RexNode> conditions) {
- return createFilter(child, conditions, RelFactories.DEFAULT_FILTER_FACTORY);
- }
-
- /**
- * Creates a filter using the default factory, or returns the original relational expression if
- * the condition is trivial.
- */
- public static RelNode createFilter(
- RelNode child,
- Iterable extends RexNode> conditions,
- RelFactories.FilterFactory filterFactory) {
- final RelOptCluster cluster = child.getCluster();
- final RexNode condition =
- RexUtil.composeConjunction(cluster.getRexBuilder(), conditions, true);
- if (condition == null) {
- return child;
- } else {
- return filterFactory.createFilter(child, condition, ImmutableSet.of());
- }
- }
-
- @Deprecated // to be removed before 2.0
- public static RelNode createNullFilter(RelNode rel, Integer[] fieldOrdinals) {
- RexNode condition = null;
- final RexBuilder rexBuilder = rel.getCluster().getRexBuilder();
- RelDataType rowType = rel.getRowType();
- int n;
- if (fieldOrdinals != null) {
- n = fieldOrdinals.length;
- } else {
- n = rowType.getFieldCount();
- }
- List fields = rowType.getFieldList();
- for (int i = 0; i < n; ++i) {
- int iField;
- if (fieldOrdinals != null) {
- iField = fieldOrdinals[i];
- } else {
- iField = i;
- }
- RelDataType type = fields.get(iField).getType();
- if (!type.isNullable()) {
- continue;
- }
- RexNode newCondition =
- rexBuilder.makeCall(
- SqlStdOperatorTable.IS_NOT_NULL, rexBuilder.makeInputRef(type, iField));
- if (condition == null) {
- condition = newCondition;
- } else {
- condition = rexBuilder.makeCall(SqlStdOperatorTable.AND, condition, newCondition);
- }
- }
- if (condition == null) {
- // no filtering required
- return rel;
- }
-
- final RelFactories.FilterFactory factory = RelFactories.DEFAULT_FILTER_FACTORY;
- return factory.createFilter(rel, condition, ImmutableSet.of());
- }
-
- /**
- * Creates a projection which casts a rel's output to a desired row type.
- *
- * No need to create new projection if {@code rel} is already a project, instead, create a
- * projection with the input of {@code rel} and the new cast expressions.
- *
- *
The desired row type and the row type to be converted must have the same number of fields.
- *
- * @param rel producer of rows to be converted
- * @param castRowType row type after cast
- * @param rename if true, use field names from castRowType; if false, preserve field names from
- * rel
- * @return conversion rel
- */
- public static RelNode createCastRel(
- final RelNode rel, RelDataType castRowType, boolean rename) {
- return createCastRel(rel, castRowType, rename, RelFactories.DEFAULT_PROJECT_FACTORY);
- }
-
- /**
- * Creates a projection which casts a rel's output to a desired row type.
- *
- *
No need to create new projection if {@code rel} is already a project, instead, create a
- * projection with the input of {@code rel} and the new cast expressions.
- *
- *
The desired row type and the row type to be converted must have the same number of fields.
- *
- * @param rel producer of rows to be converted
- * @param castRowType row type after cast
- * @param rename if true, use field names from castRowType; if false, preserve field names from
- * rel
- * @param projectFactory Project Factory
- * @return conversion rel
- */
- public static RelNode createCastRel(
- final RelNode rel,
- RelDataType castRowType,
- boolean rename,
- RelFactories.ProjectFactory projectFactory) {
- assert projectFactory != null;
- RelDataType rowType = rel.getRowType();
- if (areRowTypesEqual(rowType, castRowType, rename)) {
- // nothing to do
- return rel;
- }
- if (rowType.getFieldCount() != castRowType.getFieldCount()) {
- throw new IllegalArgumentException(
- "Field counts are not equal: "
- + "rowType ["
- + rowType
- + "] castRowType ["
- + castRowType
- + "]");
- }
- final RexBuilder rexBuilder = rel.getCluster().getRexBuilder();
- List castExps;
- RelNode input;
- List hints = ImmutableList.of();
- Set correlationVariables;
- if (rel instanceof Project) {
- // No need to create another project node if the rel
- // is already a project.
- final Project project = (Project) rel;
- castExps =
- RexUtil.generateCastExpressions(
- rexBuilder, castRowType, ((Project) rel).getProjects());
- input = rel.getInput(0);
- hints = project.getHints();
- correlationVariables = project.getVariablesSet();
- } else {
- castExps = RexUtil.generateCastExpressions(rexBuilder, castRowType, rowType);
- input = rel;
- correlationVariables = ImmutableSet.of();
- }
- if (rename) {
- // Use names and types from castRowType.
- return projectFactory.createProject(
- input, hints, castExps, castRowType.getFieldNames(), correlationVariables);
- } else {
- // Use names from rowType, types from castRowType.
- return projectFactory.createProject(
- input, hints, castExps, rowType.getFieldNames(), correlationVariables);
- }
- }
-
- /** Gets all fields in an aggregate. */
- public static Set getAllFields(Aggregate aggregate) {
- return getAllFields2(aggregate.getGroupSet(), aggregate.getAggCallList());
- }
-
- /** Gets all fields in an aggregate. */
- public static Set getAllFields2(
- ImmutableBitSet groupSet, List aggCallList) {
- final Set allFields = new TreeSet<>();
- allFields.addAll(groupSet.asList());
- for (AggregateCall aggregateCall : aggCallList) {
- allFields.addAll(aggregateCall.getArgList());
- if (aggregateCall.filterArg >= 0) {
- allFields.add(aggregateCall.filterArg);
- }
- if (aggregateCall.distinctKeys != null) {
- allFields.addAll(aggregateCall.distinctKeys.asList());
- }
- allFields.addAll(RelCollations.ordinals(aggregateCall.collation));
- }
- return allFields;
- }
-
- /**
- * Creates a LogicalAggregate that removes all duplicates from the result of an underlying
- * relational expression.
- *
- * @param rel underlying rel
- * @return rel implementing SingleValueAgg
- */
- public static RelNode createSingleValueAggRel(RelOptCluster cluster, RelNode rel) {
- final int aggCallCnt = rel.getRowType().getFieldCount();
- final List aggCalls = new ArrayList<>();
-
- for (int i = 0; i < aggCallCnt; i++) {
- aggCalls.add(
- AggregateCall.create(
- SqlStdOperatorTable.SINGLE_VALUE,
- false,
- false,
- false,
- ImmutableList.of(),
- ImmutableList.of(i),
- -1,
- null,
- RelCollations.EMPTY,
- 0,
- rel,
- null,
- null));
- }
-
- return LogicalAggregate.create(
- rel, ImmutableList.of(), ImmutableBitSet.of(), null, aggCalls);
- }
-
- // CHECKSTYLE: IGNORE 1
- /**
- * @deprecated Use {@link RelBuilder#distinct()}.
- */
- @Deprecated // to be removed before 2.0
- public static RelNode createDistinctRel(RelNode rel) {
- return LogicalAggregate.create(
- rel,
- ImmutableList.of(),
- ImmutableBitSet.range(rel.getRowType().getFieldCount()),
- null,
- ImmutableList.of());
- }
-
- @Deprecated // to be removed before 2.0
- public static boolean analyzeSimpleEquiJoin(LogicalJoin join, int[] joinFieldOrdinals) {
- RexNode joinExp = join.getCondition();
- if (joinExp.getKind() != SqlKind.EQUALS) {
- return false;
- }
- RexCall binaryExpression = (RexCall) joinExp;
- RexNode leftComparand = binaryExpression.operands.get(0);
- RexNode rightComparand = binaryExpression.operands.get(1);
- if (!(leftComparand instanceof RexInputRef)) {
- return false;
- }
- if (!(rightComparand instanceof RexInputRef)) {
- return false;
- }
-
- final int leftFieldCount = join.getLeft().getRowType().getFieldCount();
- RexInputRef leftFieldAccess = (RexInputRef) leftComparand;
- if (!(leftFieldAccess.getIndex() < leftFieldCount)) {
- // left field must access left side of join
- return false;
- }
-
- RexInputRef rightFieldAccess = (RexInputRef) rightComparand;
- if (!(rightFieldAccess.getIndex() >= leftFieldCount)) {
- // right field must access right side of join
- return false;
- }
-
- joinFieldOrdinals[0] = leftFieldAccess.getIndex();
- joinFieldOrdinals[1] = rightFieldAccess.getIndex() - leftFieldCount;
- return true;
- }
-
- /**
- * Splits out the equi-join components of a join condition, and returns what's left. For
- * example, given the condition
- *
- *
- *
- * L.A = R.X AND L.B = L.C AND (L.D = 5 OR L.E =
- * R.Y)
- *
- *
- *
- * returns
- *
- *
- * - leftKeys = {A}
- *
- rightKeys = {X}
- *
- rest = L.B = L.C AND (L.D = 5 OR L.E = R.Y)
- *
- *
- * @param left left input to join
- * @param right right input to join
- * @param condition join condition
- * @param leftKeys The ordinals of the fields from the left input which are equi-join keys
- * @param rightKeys The ordinals of the fields from the right input which are equi-join keys
- * @param filterNulls List of boolean values for each join key position indicating whether the
- * operator filters out nulls or not. Value is true if the operator is EQUALS and false if
- * the operator is IS NOT DISTINCT FROM (or an expanded version). If filterNulls
- * is null, only join conditions with EQUALS operators are considered equi-join
- * components. Rest (including IS NOT DISTINCT FROM) are returned in remaining join
- * condition.
- * @return remaining join filters that are not equijoins; may return a {@link RexLiteral} true,
- * but never null
- */
- public static RexNode splitJoinCondition(
- RelNode left,
- RelNode right,
- RexNode condition,
- List leftKeys,
- List rightKeys,
- @Nullable List filterNulls) {
- final List nonEquiList = new ArrayList<>();
-
- splitJoinCondition(left, right, condition, leftKeys, rightKeys, filterNulls, nonEquiList);
-
- return RexUtil.composeConjunction(left.getCluster().getRexBuilder(), nonEquiList);
- }
-
- /**
- * As {@link #splitJoinCondition(RelNode, RelNode, RexNode, List, List, List)}, but writes
- * non-equi conditions to a conjunctive list.
- */
- public static void splitJoinCondition(
- RelNode left,
- RelNode right,
- RexNode condition,
- List leftKeys,
- List rightKeys,
- @Nullable List filterNulls,
- List nonEquiList) {
- splitJoinCondition(
- left.getCluster().getRexBuilder(),
- left.getRowType().getFieldCount(),
- condition,
- leftKeys,
- rightKeys,
- filterNulls,
- nonEquiList);
- }
-
- @Deprecated // to be removed before 2.0
- public static boolean isEqui(RelNode left, RelNode right, RexNode condition) {
- final List leftKeys = new ArrayList<>();
- final List rightKeys = new ArrayList<>();
- final List filterNulls = new ArrayList<>();
- final List nonEquiList = new ArrayList<>();
- splitJoinCondition(
- left.getCluster().getRexBuilder(),
- left.getRowType().getFieldCount(),
- condition,
- leftKeys,
- rightKeys,
- filterNulls,
- nonEquiList);
- return nonEquiList.size() == 0;
- }
-
- /**
- * Splits out the equi-join (and optionally, a single non-equi) components of a join condition,
- * and returns what's left. Projection might be required by the caller to provide join keys that
- * are not direct field references.
- *
- * @param sysFieldList list of system fields
- * @param leftRel left join input
- * @param rightRel right join input
- * @param condition join condition
- * @param leftJoinKeys The join keys from the left input which are equi-join keys
- * @param rightJoinKeys The join keys from the right input which are equi-join keys
- * @param filterNulls The join key positions for which null values will not match. null values
- * only match for the "is not distinct from" condition.
- * @param rangeOp if null, only locate equi-joins; otherwise, locate a single non-equi join
- * predicate and return its operator in this list; join keys associated with the non-equi
- * join predicate are at the end of the key lists returned
- * @return What's left, never null
- */
- public static RexNode splitJoinCondition(
- List sysFieldList,
- RelNode leftRel,
- RelNode rightRel,
- RexNode condition,
- List leftJoinKeys,
- List rightJoinKeys,
- @Nullable List filterNulls,
- @Nullable List rangeOp) {
- return splitJoinCondition(
- sysFieldList,
- ImmutableList.of(leftRel, rightRel),
- condition,
- ImmutableList.of(leftJoinKeys, rightJoinKeys),
- filterNulls,
- rangeOp);
- }
-
- /**
- * Splits out the equi-join (and optionally, a single non-equi) components of a join condition,
- * and returns what's left. Projection might be required by the caller to provide join keys that
- * are not direct field references.
- *
- * @param sysFieldList list of system fields
- * @param inputs join inputs
- * @param condition join condition
- * @param joinKeys The join keys from the inputs which are equi-join keys
- * @param filterNulls The join key positions for which null values will not match. null values
- * only match for the "is not distinct from" condition.
- * @param rangeOp if null, only locate equi-joins; otherwise, locate a single non-equi join
- * predicate and return its operator in this list; join keys associated with the non-equi
- * join predicate are at the end of the key lists returned
- * @return What's left, never null
- */
- public static RexNode splitJoinCondition(
- List sysFieldList,
- List inputs,
- RexNode condition,
- List