Skip to content

Commit 8bb4c36

Browse files
committed
1.35
1 parent f5203ca commit 8bb4c36

File tree

4 files changed

+97
-16
lines changed

4 files changed

+97
-16
lines changed

flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql/fun/SqlCastFunction.java

Lines changed: 3 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -37,6 +37,7 @@
3737
import org.apache.calcite.sql.SqlSyntax;
3838
import org.apache.calcite.sql.SqlUtil;
3939
import org.apache.calcite.sql.SqlWriter;
40+
import org.apache.calcite.sql.type.FlinkCalciteTableMappingRule;
4041
import org.apache.calcite.sql.type.InferTypes;
4142
import org.apache.calcite.sql.type.SqlOperandCountRanges;
4243
import org.apache.calcite.sql.type.SqlReturnTypeInference;
@@ -267,7 +268,8 @@ private boolean canCastFrom(RelDataType toType, RelDataType fromType) {
267268
FlinkTypeFactory.toLogicalType(fromType),
268269
FlinkTypeFactory.toLogicalType(toType));
269270
default:
270-
return SqlTypeUtil.canCastFrom(toType, fromType, true);
271+
return SqlTypeUtil.canCastFrom(
272+
toType, fromType, FlinkCalciteTableMappingRule.instance());
271273
}
272274
}
273275

Original file line numberDiff line numberDiff line change
@@ -0,0 +1,79 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one
3+
* or more contributor license agreements. See the NOTICE file
4+
* distributed with this work for additional information
5+
* regarding copyright ownership. The ASF licenses this file
6+
* to you under the Apache License, Version 2.0 (the
7+
* "License"); you may not use this file except in compliance
8+
* with the License. You may obtain a copy of the License at
9+
*
10+
* http://www.apache.org/licenses/LICENSE-2.0
11+
*
12+
* Unless required by applicable law or agreed to in writing, software
13+
* distributed under the License is distributed on an "AS IS" BASIS,
14+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15+
* See the License for the specific language governing permissions and
16+
* limitations under the License.
17+
*/
18+
19+
package org.apache.calcite.sql.type;
20+
21+
import com.google.common.collect.ImmutableMap;
22+
import com.google.common.collect.ImmutableSet;
23+
import org.checkerframework.checker.nullness.qual.Nullable;
24+
25+
import java.util.HashSet;
26+
import java.util.Map;
27+
import java.util.Objects;
28+
import java.util.Set;
29+
30+
public class FlinkCalciteTableMappingRule implements SqlTypeMappingRule {
31+
private static final FlinkCalciteTableMappingRule INSTANCE;
32+
33+
private final Map<SqlTypeName, ImmutableSet<SqlTypeName>> map;
34+
35+
private FlinkCalciteTableMappingRule(Map<SqlTypeName, ImmutableSet<SqlTypeName>> map) {
36+
this.map = ImmutableMap.copyOf(map);
37+
}
38+
39+
public static FlinkCalciteTableMappingRule instance() {
40+
return Objects.requireNonNull(FLINK_THREAD_PROVIDERS.get(), "flinkThreadProviders");
41+
}
42+
43+
public static FlinkCalciteTableMappingRule instance(
44+
Map<SqlTypeName, ImmutableSet<SqlTypeName>> map) {
45+
return new FlinkCalciteTableMappingRule(map);
46+
}
47+
48+
public Map<SqlTypeName, ImmutableSet<SqlTypeName>> getTypeMapping() {
49+
return this.map;
50+
}
51+
52+
static {
53+
SqlTypeMappingRules.Builder coerceRules = SqlTypeMappingRules.builder();
54+
coerceRules.addAll(SqlTypeCoercionRule.lenientInstance().getTypeMapping());
55+
Map<SqlTypeName, ImmutableSet<SqlTypeName>> map =
56+
SqlTypeCoercionRule.lenientInstance().getTypeMapping();
57+
Set<SqlTypeName> rule = new HashSet<>();
58+
rule.add(SqlTypeName.TINYINT);
59+
rule.add(SqlTypeName.SMALLINT);
60+
rule.add(SqlTypeName.INTEGER);
61+
rule.add(SqlTypeName.BIGINT);
62+
rule.add(SqlTypeName.DECIMAL);
63+
rule.add(SqlTypeName.FLOAT);
64+
rule.add(SqlTypeName.REAL);
65+
rule.add(SqlTypeName.DOUBLE);
66+
rule.add(SqlTypeName.CHAR);
67+
rule.add(SqlTypeName.VARCHAR);
68+
rule.add(SqlTypeName.BOOLEAN);
69+
rule.add(SqlTypeName.TIMESTAMP);
70+
rule.add(SqlTypeName.TIMESTAMP_WITH_LOCAL_TIME_ZONE);
71+
coerceRules.add(SqlTypeName.FLOAT, rule);
72+
coerceRules.add(SqlTypeName.DOUBLE, rule);
73+
coerceRules.add(SqlTypeName.DECIMAL, rule);
74+
INSTANCE = new FlinkCalciteTableMappingRule(coerceRules.map);
75+
}
76+
77+
public static final ThreadLocal<@Nullable FlinkCalciteTableMappingRule> FLINK_THREAD_PROVIDERS =
78+
ThreadLocal.withInitial(() -> INSTANCE);
79+
}

flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/CastFunctionITCase.java

Lines changed: 14 additions & 14 deletions
Original file line numberDiff line numberDiff line change
@@ -294,8 +294,8 @@ private static List<TestSetSpec> allTypesBasic() {
294294
.failRuntime(STRING(), "Apache", NumberFormatException.class)
295295
.fromCase(STRING(), "1.234", new BigDecimal("1.234"))
296296
.fromCase(STRING(), "1.2", new BigDecimal("1.200"))
297-
// .fromCase(BOOLEAN(), true, new BigDecimal("1.000"))
298-
// .fromCase(BOOLEAN(), false, new BigDecimal("0.000"))
297+
.fromCase(BOOLEAN(), true, new BigDecimal("1.000"))
298+
.fromCase(BOOLEAN(), false, new BigDecimal("0.000"))
299299
// Not supported - no fix
300300
.failValidation(BINARY(2), DEFAULT_BINARY)
301301
.failValidation(VARBINARY(5), DEFAULT_VARBINARY)
@@ -330,8 +330,8 @@ private static List<TestSetSpec> allTypesBasic() {
330330
.fromCase(STRING(), "1.234", (byte) 1)
331331
.fromCase(STRING(), "123", (byte) 123)
332332
.failRuntime(STRING(), "-130", NumberFormatException.class)
333-
// .fromCase(BOOLEAN(), true, (byte) 1)
334-
// .fromCase(BOOLEAN(), false, (byte) 0)
333+
.fromCase(BOOLEAN(), true, (byte) 1)
334+
.fromCase(BOOLEAN(), false, (byte) 0)
335335
// Not supported - no fix
336336
.failValidation(BINARY(2), DEFAULT_BINARY)
337337
.failValidation(VARBINARY(5), DEFAULT_VARBINARY)
@@ -377,8 +377,8 @@ private static List<TestSetSpec> allTypesBasic() {
377377
.fromCase(STRING(), "1.234", (short) 1)
378378
.fromCase(STRING(), "123", (short) 123)
379379
.failRuntime(STRING(), "-32769", NumberFormatException.class)
380-
// .fromCase(BOOLEAN(), true, (short) 1)
381-
// .fromCase(BOOLEAN(), false, (short) 0)
380+
.fromCase(BOOLEAN(), true, (short) 1)
381+
.fromCase(BOOLEAN(), false, (short) 0)
382382
// Not supported - no fix
383383
.failValidation(BINARY(2), DEFAULT_BINARY)
384384
.failValidation(VARBINARY(5), DEFAULT_VARBINARY)
@@ -435,8 +435,8 @@ private static List<TestSetSpec> allTypesBasic() {
435435
.fromCase(STRING(), "1.234", 1)
436436
.fromCase(STRING(), "123", 123)
437437
.failRuntime(STRING(), "-3276913443134", NumberFormatException.class)
438-
// .fromCase(BOOLEAN(), true, 1)
439-
// .fromCase(BOOLEAN(), false, 0)
438+
.fromCase(BOOLEAN(), true, 1)
439+
.fromCase(BOOLEAN(), false, 0)
440440
// Not supported - no fix
441441
.failValidation(BINARY(2), DEFAULT_BINARY)
442442
.failValidation(VARBINARY(5), DEFAULT_VARBINARY)
@@ -495,8 +495,8 @@ private static List<TestSetSpec> allTypesBasic() {
495495
.fromCase(STRING(), "1.234", 1L)
496496
.fromCase(STRING(), "123", 123L)
497497
.fromCase(STRING(), "-3276913443134", -3276913443134L)
498-
// .fromCase(BOOLEAN(), true, 1L)
499-
// .fromCase(BOOLEAN(), false, 0L)
498+
.fromCase(BOOLEAN(), true, 1L)
499+
.fromCase(BOOLEAN(), false, 0L)
500500
// Not supported - no fix
501501
.failValidation(BINARY(2), DEFAULT_BINARY)
502502
.failValidation(VARBINARY(5), DEFAULT_VARBINARY)
@@ -552,8 +552,8 @@ private static List<TestSetSpec> allTypesBasic() {
552552
.fromCase(STRING(), "1.234", 1.234f)
553553
.fromCase(STRING(), "123", 123.0f)
554554
.fromCase(STRING(), "-3276913443134", -3.27691403E12f)
555-
// .fromCase(BOOLEAN(), true, 1.0f)
556-
// .fromCase(BOOLEAN(), false, 0.0f)
555+
.fromCase(BOOLEAN(), true, 1.0f)
556+
.fromCase(BOOLEAN(), false, 0.0f)
557557
// Not supported - no fix
558558
.failValidation(BINARY(2), DEFAULT_BINARY)
559559
.failValidation(VARBINARY(5), DEFAULT_VARBINARY)
@@ -614,8 +614,8 @@ private static List<TestSetSpec> allTypesBasic() {
614614
.fromCase(STRING(), "1.234", 1.234d)
615615
.fromCase(STRING(), "123", 123.0d)
616616
.fromCase(STRING(), "-3276913443134", -3.276913443134E12)
617-
// .fromCase(BOOLEAN(), true, 1.0d)
618-
// .fromCase(BOOLEAN(), false, 0.0d)
617+
.fromCase(BOOLEAN(), true, 1.0d)
618+
.fromCase(BOOLEAN(), false, 0.0d)
619619
// Not supported - no fix
620620
.failValidation(BINARY(2), DEFAULT_BINARY)
621621
.failValidation(VARBINARY(5), DEFAULT_VARBINARY)

tools/maven/suppressions.xml

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -56,7 +56,7 @@ under the License.
5656

5757
<!-- Have to use guava directly -->
5858
<suppress
59-
files="OverConvertRule.java|InConverter.java|SymbolUtil.java|RexNodeJsonDeserializer.java|RexNodeJsonSerializer.java|RexNodeJsonSerdeTest.java|FlinkAggregateProjectMergeRule.java|BatchPhysicalConstantTableFunctionScanRule.java|StreamPhysicalConstantTableFunctionScanRule.java|AggregateReduceGroupingRule.java"
59+
files="OverConvertRule.java|InConverter.java|SymbolUtil.java|RexNodeJsonDeserializer.java|RexNodeJsonSerializer.java|RexNodeJsonSerdeTest.java|FlinkAggregateProjectMergeRule.java|BatchPhysicalConstantTableFunctionScanRule.java|StreamPhysicalConstantTableFunctionScanRule.java|AggregateReduceGroupingRule.java|FlinkCalciteTableMappingRule.java"
6060
checks="IllegalImport"/>
6161
<!-- Classes copied from AWS -->
6262
<suppress

0 commit comments

Comments
 (0)