twalthr commented on code in PR #28170:
URL: https://github.com/apache/flink/pull/28170#discussion_r3412859478
##########
flink-table/flink-sql-parser/src/main/codegen/templates/Parser.jj:
##########
@@ -1957,13 +1959,19 @@ SqlInsert WhenNotMatchedClause(SqlNode table) :
*/
void AddSelectItem(List<SqlNode> list) :
{
- final SqlNode e;
+ SqlNode e;
final SqlIdentifier id;
}
{
e = SelectExpression()
(
- [ <AS> ]
+ [ <AS>
+ [ <MEASURE> {
Review Comment:
Are measures disabled in Flink?
##########
flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/serde/RexNodeJsonSerdeTest.java:
##########
@@ -720,7 +720,7 @@ private static Stream<RexNode> testRexNodeSerde() {
FACTORY.createSqlType(SqlTypeName.INTEGER)),
rexBuilder.makeExactLiteral(
new BigDecimal(Long.MAX_VALUE),
FACTORY.createSqlType(SqlTypeName.BIGINT)),
- rexBuilder.makeExactLiteral(
+ rexBuilder.makeApproxLiteral(
Review Comment:
Why this change? Shouldn't we pass a double in this case? Not BigDecimal?
##########
flink-table/flink-table-planner/src/main/java/org/apache/calcite/rex/RexBuilder.java:
##########
@@ -1023,6 +1264,21 @@ protected RexLiteral makeLiteral(
}
o = ((TimestampWithTimeZoneString) o).round(p);
break;
+ /*
+ case DECIMAL:
Review Comment:
I guess this is a Flink modification? If yes, it should be marked.
##########
flink-table/flink-table-planner/src/main/java/org/apache/calcite/runtime/SqlFunctions.java:
##########
@@ -122,17 +128,27 @@
import java.util.regex.Pattern;
import java.util.regex.PatternSyntaxException;
+import static java.lang.Byte.parseByte;
+import static java.lang.Double.parseDouble;
+import static java.lang.Float.parseFloat;
+import static java.lang.Integer.parseInt;
+import static java.lang.Long.parseLong;
+import static java.lang.Short.parseShort;
import static java.nio.charset.StandardCharsets.UTF_8;
import static java.util.Objects.requireNonNull;
+import static java.util.stream.Collectors.joining;
import static
org.apache.calcite.config.CalciteSystemProperty.FUNCTION_LEVEL_CACHE_MAX_SIZE;
import static org.apache.calcite.linq4j.Nullness.castNonNull;
import static org.apache.calcite.util.Static.RESOURCE;
/**
- * Flink modifications:
+ * Helper methods to implement SQL functions in generated code.
*
- * <p>so far there are no Flink modifications. Could be removed once upgraded
to the version with
- * fixed CALCITE-6393.
Review Comment:
What happened to CALCITE-6393. It seems this is still not fixed?
##########
flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/metadata/RelMdPredicates.java:
##########
@@ -1,1051 +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.rel.metadata;
-
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Iterables;
-import org.apache.calcite.linq4j.Ord;
-import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelOptPredicateList;
-import org.apache.calcite.plan.RelOptUtil;
-import org.apache.calcite.plan.RexImplicationChecker;
-import org.apache.calcite.plan.Strong;
-import org.apache.calcite.plan.volcano.RelSubset;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.core.Aggregate;
-import org.apache.calcite.rel.core.AggregateCall;
-import org.apache.calcite.rel.core.Correlate;
-import org.apache.calcite.rel.core.Exchange;
-import org.apache.calcite.rel.core.Filter;
-import org.apache.calcite.rel.core.Intersect;
-import org.apache.calcite.rel.core.Join;
-import org.apache.calcite.rel.core.JoinRelType;
-import org.apache.calcite.rel.core.Minus;
-import org.apache.calcite.rel.core.Project;
-import org.apache.calcite.rel.core.Sample;
-import org.apache.calcite.rel.core.Sort;
-import org.apache.calcite.rel.core.TableModify;
-import org.apache.calcite.rel.core.TableScan;
-import org.apache.calcite.rel.core.Union;
-import org.apache.calcite.rel.core.Values;
-import org.apache.calcite.rex.RexBuilder;
-import org.apache.calcite.rex.RexCall;
-import org.apache.calcite.rex.RexExecutor;
-import org.apache.calcite.rex.RexInputRef;
-import org.apache.calcite.rex.RexLiteral;
-import org.apache.calcite.rex.RexNode;
-import org.apache.calcite.rex.RexPermuteInputsShuttle;
-import org.apache.calcite.rex.RexSimplify;
-import org.apache.calcite.rex.RexUnknownAs;
-import org.apache.calcite.rex.RexUtil;
-import org.apache.calcite.rex.RexVisitorImpl;
-import org.apache.calcite.sql.SqlKind;
-import org.apache.calcite.sql.fun.SqlInternalOperators;
-import org.apache.calcite.sql.fun.SqlStdOperatorTable;
-import org.apache.calcite.util.BitSets;
-import org.apache.calcite.util.Bug;
-import org.apache.calcite.util.ImmutableBitSet;
-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.nullness.qual.Nullable;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.BitSet;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.NoSuchElementException;
-import java.util.Set;
-import java.util.SortedMap;
-import java.util.TreeMap;
-import java.util.stream.Collectors;
-
-import static java.util.Objects.requireNonNull;
-
-/**
- * Utility to infer Predicates that are applicable above a RelNode.
- *
- * <p>This is currently used by {@link
- * org.apache.calcite.rel.rules.JoinPushTransitivePredicatesRule} to infer
<em>Predicates</em> that
- * can be inferred from one side of a Join to the other.
- *
- * <p>The PullUp Strategy is sound but not complete. Here are some of the
limitations:
- *
- * <ol>
- * <li>For Aggregations we only PullUp predicates that only contain Grouping
Keys. This can be
- * extended to infer predicates on Aggregation expressions from
expressions on the aggregated
- * columns. For e.g.
- * <pre>
- * select a, max(b) from R1 where b > 7
- * → max(b) > 7 or max(b) is null
- * </pre>
- * <li>For Projections we only look at columns that are projected without
any function applied.
- * So:
- * <pre>
- * select a from R1 where a > 7
- * → "a > 7" is pulled up from the Projection.
- * select a + 1 from R1 where a + 1 > 7
- * → "a + 1 gt; 7" is not pulled up
- * </pre>
- * <li>There are several restrictions on Joins:
- * <ul>
- * <li>We only pullUp inferred predicates for now. Pulling up existing
predicates causes an
- * explosion of duplicates. The existing predicates are pushed
back down as new
- * predicates. Once we have rules to eliminate duplicate Filter
conditions, we should
- * pullUp all predicates.
- * <li>For Left Outer: we infer new predicates from the left and set
them as applicable on
- * the Right side. No predicates are pulledUp.
- * <li>Right Outer Joins are handled in an analogous manner.
- * <li>For Full Outer Joins no predicates are pulledUp or inferred.
- * </ul>
- * </ol>
- *
- * The class contains CALCITE-6599 fix from 1.39.0, should be dropped after
upgrade to that version
Review Comment:
Is this correct? CALCITE-6599 was fixed in 1.39
##########
flink-table/flink-sql-parser/src/main/codegen/templates/Parser.jj:
##########
@@ -2844,6 +2877,27 @@ SqlNode WindowRange() :
)
}
+/** Parses an exclusion clause for WINDOW FRAME. */
+SqlLiteral WindowExclusion() :
+{
+}
+{
+ (
Review Comment:
open a followup subtask
##########
flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/agg/OverAggregateTest.xml:
##########
@@ -52,22 +52,22 @@ FROM MyTable
</Resource>
<Resource name="ast">
<![CDATA[
-LogicalProject(EXPR$0=[CASE(>(COUNT($0) OVER (PARTITION BY $1 ORDER BY $0
NULLS FIRST), 0), $SUM0($0) OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST),
null:INTEGER)], EXPR$1=[MAX($0) OVER (PARTITION BY $1 ORDER BY $2 NULLS
FIRST)], EXPR$2=[CAST(/(CASE(>(COUNT($0) OVER (PARTITION BY $2 ORDER BY $0
NULLS FIRST), 0), $SUM0($0) OVER (PARTITION BY $2 ORDER BY $0 NULLS FIRST),
null:INTEGER), COUNT($0) OVER (PARTITION BY $2 ORDER BY $0 NULLS
FIRST))):INTEGER], EXPR$3=[RANK() OVER (PARTITION BY $1 ORDER BY $0 NULLS
FIRST)], EXPR$4=[MIN($0) OVER (PARTITION BY $2 ORDER BY $0 NULLS FIRST)])
+LogicalProject(EXPR$0=[CASE(>(COUNT($0) OVER (PARTITION BY $1 ORDER BY $0
NULLS FIRST), 0), SUM($0) OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST),
null:INTEGER)], EXPR$1=[MAX($0) OVER (PARTITION BY $1 ORDER BY $2 NULLS
FIRST)], EXPR$2=[CAST(/(CASE(>(COUNT($0) OVER (PARTITION BY $2 ORDER BY $0
NULLS FIRST), 0), SUM($0) OVER (PARTITION BY $2 ORDER BY $0 NULLS FIRST),
null:INTEGER), COUNT($0) OVER (PARTITION BY $2 ORDER BY $0 NULLS
FIRST))):INTEGER], EXPR$3=[RANK() OVER (PARTITION BY $1 ORDER BY $0 NULLS
FIRST)], EXPR$4=[MIN($0) OVER (PARTITION BY $2 ORDER BY $0 NULLS FIRST)])
Review Comment:
Do we know what happened here?
##########
flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/subquery/SubQuerySemiJoinTest.scala:
##########
@@ -1711,7 +1679,7 @@ class SubQuerySemiJoinTest extends SubQueryTestBase {
// TODO some bugs in SubQueryRemoveRule
// the result RelNode (LogicalJoin(condition=[=($1, $8)],
joinType=[left]))
// after SubQueryRemoveRule is unexpected
- assertThatExceptionOfType(classOf[AssertionError])
+ assertThatExceptionOfType(classOf[NullPointerException])
Review Comment:
same comment as above
##########
flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql/fun/SqlStdOperatorTable.java:
##########
@@ -0,0 +1,2553 @@
+/*
+ * 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.fun;
+
+import com.google.common.base.Suppliers;
+import com.google.common.collect.ImmutableList;
+import org.apache.calcite.avatica.util.TimeUnit;
+import org.apache.calcite.sql.SqlAggFunction;
+import org.apache.calcite.sql.SqlAsOperator;
+import org.apache.calcite.sql.SqlBasicCall;
+import org.apache.calcite.sql.SqlBasicFunction;
+import org.apache.calcite.sql.SqlBinaryOperator;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlDescriptorOperator;
+import org.apache.calcite.sql.SqlFilterOperator;
+import org.apache.calcite.sql.SqlFunction;
+import org.apache.calcite.sql.SqlFunctionCategory;
+import org.apache.calcite.sql.SqlGroupedWindowFunction;
+import org.apache.calcite.sql.SqlHopTableFunction;
+import org.apache.calcite.sql.SqlInternalOperator;
+import org.apache.calcite.sql.SqlJsonConstructorNullClause;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlLateralOperator;
+import org.apache.calcite.sql.SqlLiteral;
+import org.apache.calcite.sql.SqlMatchFunction;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlNullTreatmentOperator;
+import org.apache.calcite.sql.SqlNumericLiteral;
+import org.apache.calcite.sql.SqlOperandCountRange;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.SqlOverOperator;
+import org.apache.calcite.sql.SqlPostfixOperator;
+import org.apache.calcite.sql.SqlPrefixOperator;
+import org.apache.calcite.sql.SqlProcedureCallOperator;
+import org.apache.calcite.sql.SqlRankFunction;
+import org.apache.calcite.sql.SqlSampleSpec;
+import org.apache.calcite.sql.SqlSessionTableFunction;
+import org.apache.calcite.sql.SqlSetOperator;
+import org.apache.calcite.sql.SqlSetSemanticsTableOperator;
+import org.apache.calcite.sql.SqlSpecialOperator;
+import org.apache.calcite.sql.SqlSyntax;
+import org.apache.calcite.sql.SqlTumbleTableFunction;
+import org.apache.calcite.sql.SqlUnnestOperator;
+import org.apache.calcite.sql.SqlUtil;
+import org.apache.calcite.sql.SqlValuesOperator;
+import org.apache.calcite.sql.SqlWindow;
+import org.apache.calcite.sql.SqlWithinDistinctOperator;
+import org.apache.calcite.sql.SqlWithinGroupOperator;
+import org.apache.calcite.sql.SqlWriter;
+import org.apache.calcite.sql.type.InferTypes;
+import org.apache.calcite.sql.type.OperandTypes;
+import org.apache.calcite.sql.type.ReturnTypes;
+import org.apache.calcite.sql.type.SqlOperandCountRanges;
+import org.apache.calcite.sql.type.SqlReturnTypeInference;
+import org.apache.calcite.sql.type.SqlTypeFamily;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.sql.util.ReflectiveSqlOperatorTable;
+import org.apache.calcite.sql.validate.SqlConformance;
+import org.apache.calcite.sql.validate.SqlConformanceEnum;
+import org.apache.calcite.sql.validate.SqlModality;
+import org.apache.calcite.sql2rel.AuxiliaryConverter;
+import org.apache.calcite.util.Litmus;
+import org.apache.calcite.util.Optionality;
+import org.apache.calcite.util.Pair;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+import java.util.List;
+import java.util.function.BiConsumer;
+import java.util.function.Consumer;
+import java.util.function.Supplier;
+
+import static java.util.Objects.requireNonNull;
+import static org.apache.calcite.linq4j.Nullness.castNonNull;
+
+/**
+ * Implementation of {@link org.apache.calcite.sql.SqlOperatorTable}
containing the standard
+ * operators and functions.
+ *
+ * <p>Lines 765 ~ 767, 785 ~ 787, 796 ~ 798, 807 ~ 809, 818 ~ 820, 829 ~ 831,
840 ~ 842, Flink
+ * changes the return type of the {@code IS [NOT] JSON ...} predicates from
{@link
+ * ReturnTypes#BOOLEAN_NULLABLE} to {@link ReturnTypes#BOOLEAN} so that they
always return a
+ * non-nullable {@code BOOLEAN}.
Review Comment:
please link a ticket for this
##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/calcite/FlinkTypeSystem.java:
##########
@@ -100,6 +100,16 @@ public int getMaxPrecision(SqlTypeName typeName) {
return super.getMaxPrecision(typeName);
}
+ @Override
+ public int getMinScale(SqlTypeName typeName) {
Review Comment:
but do we? Better block this for now?
##########
flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/subquery/SubQueryAntiJoinTest.scala:
##########
@@ -737,7 +737,7 @@ class SubQueryAntiJoinTest extends SubQueryTestBase {
// TODO some bugs in SubQueryRemoveRule
// the result RelNode (LogicalJoin(condition=[=($1, $11)],
joinType=[left]))
// after SubQueryRemoveRule is unexpected
- assertThatExceptionOfType(classOf[AssertionError])
+ assertThatExceptionOfType(classOf[NullPointerException])
Review Comment:
What is this test about? Checking against a NullPointerException doesn't
seem right.
##########
flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql/SqlWindow.java:
##########
@@ -0,0 +1,1061 @@
+/*
+ * 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.flink.table.api.ValidationException;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.calcite.linq4j.Ord;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexWindowBound;
+import org.apache.calcite.rex.RexWindowBounds;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.type.ReturnTypes;
+import org.apache.calcite.sql.type.SqlTypeFamily;
+import org.apache.calcite.sql.util.SqlBasicVisitor;
+import org.apache.calcite.sql.util.SqlVisitor;
+import org.apache.calcite.sql.validate.SqlValidator;
+import org.apache.calcite.sql.validate.SqlValidatorScope;
+import org.apache.calcite.sql.validate.SqlValidatorUtil;
+import org.apache.calcite.util.ControlFlowException;
+import org.apache.calcite.util.ImmutableNullableList;
+import org.apache.calcite.util.Litmus;
+import org.apache.calcite.util.Util;
+import org.checkerframework.checker.nullness.qual.EnsuresNonNullIf;
+import org.checkerframework.checker.nullness.qual.Nullable;
+import org.checkerframework.dataflow.qual.Pure;
+
+import java.math.BigDecimal;
+import java.util.List;
+
+import static java.util.Objects.requireNonNull;
+import static org.apache.calcite.linq4j.Nullness.castNonNull;
+import static org.apache.calcite.util.Static.RESOURCE;
+
+/**
+ * Copied from Calcite.
+ *
+ * <p>Currently disabling EXCLUDE.
Review Comment:
Instead of copying the whole Calcite class, can't we just check for the
exclude in the Logical converter rule?
##########
flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql/type/BasicSqlType.java:
##########
@@ -0,0 +1,322 @@
+/*
+ * 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 org.apache.calcite.rel.type.RelDataTypeSystem;
+import org.apache.calcite.sql.SqlCollation;
+import org.apache.calcite.util.SerializableCharset;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+import java.nio.charset.Charset;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static java.util.Objects.requireNonNull;
+
+/**
+ * BasicSqlType represents a standard atomic SQL type (excluding interval
types).
+ *
+ * <p>Instances of this class are immutable.
+ */
+public class BasicSqlType extends AbstractSqlType {
Review Comment:
Can you add this explanation to the JavaDoc in code and link CALCITE-7531
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]