Skip to content

Commit

Permalink
[FLINK-22484][table] add case with null for map_from_arrays
Browse files Browse the repository at this point in the history
  • Loading branch information
snuyanzin committed Mar 12, 2023
1 parent d994cd1 commit 46d48a4
Show file tree
Hide file tree
Showing 2 changed files with 114 additions and 113 deletions.
Expand Up @@ -23,18 +23,11 @@
import org.apache.flink.types.Row;

import java.time.LocalDate;
import java.util.Collections;
import java.util.Map;
import java.util.stream.Stream;

import static org.apache.flink.table.api.Expressions.$;
import static org.apache.flink.table.api.Expressions.lit;
import static org.apache.flink.table.api.Expressions.call;
import static org.apache.flink.table.api.Expressions.map;
import static org.apache.flink.table.api.Expressions.mapFromArrays;
import static org.apache.flink.table.api.Expressions.row;
import static org.apache.flink.util.CollectionUtil.entry;
import static org.apache.flink.util.CollectionUtil.map;

/** Tests for {@link BuiltInFunctionDefinitions} around arrays. */
class CollectionFunctionsITCase extends BuiltInFunctionTestBase {
Expand Down Expand Up @@ -185,110 +178,6 @@ Stream<TestSetSpec> getTestSetSpecs() {
null
},
DataTypes.ARRAY(
DataTypes.ROW(DataTypes.BOOLEAN(), DataTypes.DATE()))),
TestSetSpec.forFunction(BuiltInFunctionDefinitions.MAP_KEYS)
.onFieldsWithData(
null,
"item",
Collections.singletonMap(1, "value"),
Collections.singletonMap(new Integer[] {1, 2}, "value"))
.andDataTypes(
DataTypes.BOOLEAN().nullable(),
DataTypes.STRING(),
DataTypes.MAP(DataTypes.INT(), DataTypes.STRING()),
DataTypes.MAP(DataTypes.ARRAY(DataTypes.INT()), DataTypes.STRING()))
.testTableApiValidationError(
call("MAP_KEYS", $("f0"), $("f1")),
"Invalid function call:\nMAP_KEYS(BOOLEAN, STRING)")
.testResult(
map(
$("f0").cast(DataTypes.BOOLEAN()),
$("f1").cast(DataTypes.STRING()))
.mapKeys(),
"MAP_KEYS(MAP[CAST(f0 AS BOOLEAN), CAST(f1 AS STRING)])",
new Boolean[] {null},
DataTypes.ARRAY(DataTypes.BOOLEAN()).notNull())
.testResult(
$("f2").mapKeys(),
"MAP_KEYS(f2)",
new Integer[] {1},
DataTypes.ARRAY(DataTypes.INT()))
.testResult(
$("f3").mapKeys(),
"MAP_KEYS(f3)",
new Integer[][] {new Integer[] {1, 2}},
DataTypes.ARRAY(DataTypes.ARRAY(DataTypes.INT()))),
TestSetSpec.forFunction(BuiltInFunctionDefinitions.MAP_VALUES)
.onFieldsWithData(
null,
"item",
Collections.singletonMap(1, "value1"),
Collections.singletonMap(
3, Collections.singletonMap(true, "value2")))
.andDataTypes(
DataTypes.BOOLEAN().nullable(),
DataTypes.STRING(),
DataTypes.MAP(DataTypes.INT(), DataTypes.STRING()),
DataTypes.MAP(
DataTypes.INT(),
DataTypes.MAP(DataTypes.BOOLEAN(), DataTypes.STRING())))
.testTableApiValidationError(
call("MAP_VALUES", $("f0"), $("f1")),
"Invalid function call:\nMAP_VALUES(BOOLEAN, STRING)")
.testResult(
map(
$("f1").cast(DataTypes.STRING()),
$("f0").cast(DataTypes.BOOLEAN()))
.mapValues(),
"MAP_VALUES(MAP[CAST(f1 AS STRING), CAST(f0 AS BOOLEAN)])",
new Boolean[] {null},
DataTypes.ARRAY(DataTypes.BOOLEAN()).notNull())
.testResult(
$("f2").mapValues(),
"MAP_VALUES(f2)",
new String[] {"value1"},
DataTypes.ARRAY(DataTypes.STRING()))
.testResult(
$("f3").mapValues(),
"MAP_VALUES(f3)",
new Map[] {Collections.singletonMap(true, "value2")},
DataTypes.ARRAY(
DataTypes.MAP(DataTypes.BOOLEAN(), DataTypes.STRING()))),
TestSetSpec.forFunction(BuiltInFunctionDefinitions.MAP_FROM_ARRAYS, "Invalid input")
.onFieldsWithData(null, null, new Integer[] {1}, new Integer[] {1, 2})
.andDataTypes(
DataTypes.BOOLEAN().nullable(),
DataTypes.INT().nullable(),
DataTypes.ARRAY(DataTypes.INT()),
DataTypes.ARRAY(DataTypes.INT()))
.testTableApiRuntimeError(
mapFromArrays($("f2"), $("f3")),
"Invalid function MAP_FROM_ARRAYS call:\n"
+ "The length of the keys array 1 is not equal to the length of the values array 2")
.testSqlRuntimeError(
"MAP_FROM_ARRAYS(array[1, 2, 3], array[1, 2])",
"Invalid function MAP_FROM_ARRAYS call:\n"
+ "The length of the keys array 3 is not equal to the length of the values array 2"),
TestSetSpec.forFunction(BuiltInFunctionDefinitions.MAP_FROM_ARRAYS)
.onFieldsWithData(
new Integer[] {1, 2},
new String[] {"one", "two"},
new Integer[][] {new Integer[] {1, 2}, new Integer[] {3, 4}})
.andDataTypes(
DataTypes.ARRAY(DataTypes.INT()),
DataTypes.ARRAY(DataTypes.STRING()),
DataTypes.ARRAY(DataTypes.ARRAY(DataTypes.INT())))
.testResult(
mapFromArrays($("f0"), $("f1")),
"MAP_FROM_ARRAYS(f0, f1)",
map(entry(1, "one"), entry(2, "two")),
DataTypes.MAP(DataTypes.INT(), DataTypes.STRING()))
.testTableApiResult(
mapFromArrays($("f1"), $("f2")),
map(
entry("one", new Integer[] {1, 2}),
entry("two", new Integer[] {3, 4})),
DataTypes.MAP(
DataTypes.STRING(), DataTypes.ARRAY(DataTypes.INT()))));
DataTypes.ROW(DataTypes.BOOLEAN(), DataTypes.DATE()))));
}
}
Expand Up @@ -28,6 +28,7 @@
import java.time.LocalTime;
import java.time.Period;
import java.util.Collections;
import java.util.Map;
import java.util.stream.Stream;

import static org.apache.flink.table.api.DataTypes.BIGINT;
Expand All @@ -44,7 +45,9 @@
import static org.apache.flink.table.api.DataTypes.TIME;
import static org.apache.flink.table.api.DataTypes.TIMESTAMP;
import static org.apache.flink.table.api.Expressions.$;
import static org.apache.flink.table.api.Expressions.call;
import static org.apache.flink.table.api.Expressions.map;
import static org.apache.flink.table.api.Expressions.mapFromArrays;
import static org.apache.flink.util.CollectionUtil.entry;

/** Test {@link BuiltInFunctionDefinitions#MAP} and its return type. */
Expand Down Expand Up @@ -205,6 +208,115 @@ Stream<TestSetSpec> getTestSetSpecs() {
DataTypes.MAP(
STRING().notNull(),
INTERVAL(MONTH()).nullable())
.notNull())));
.notNull())),
TestSetSpec.forFunction(BuiltInFunctionDefinitions.MAP_KEYS)
.onFieldsWithData(
null,
"item",
Collections.singletonMap(1, "value"),
Collections.singletonMap(new Integer[] {1, 2}, "value"))
.andDataTypes(
DataTypes.BOOLEAN().nullable(),
DataTypes.STRING(),
DataTypes.MAP(DataTypes.INT(), DataTypes.STRING()),
DataTypes.MAP(DataTypes.ARRAY(DataTypes.INT()), DataTypes.STRING()))
.testTableApiValidationError(
call("MAP_KEYS", $("f0"), $("f1")),
"Invalid function call:\nMAP_KEYS(BOOLEAN, STRING)")
.testResult(
map(
$("f0").cast(DataTypes.BOOLEAN()),
$("f1").cast(DataTypes.STRING()))
.mapKeys(),
"MAP_KEYS(MAP[CAST(f0 AS BOOLEAN), CAST(f1 AS STRING)])",
new Boolean[] {null},
DataTypes.ARRAY(DataTypes.BOOLEAN()).notNull())
.testResult(
$("f2").mapKeys(),
"MAP_KEYS(f2)",
new Integer[] {1},
DataTypes.ARRAY(DataTypes.INT()))
.testResult(
$("f3").mapKeys(),
"MAP_KEYS(f3)",
new Integer[][] {new Integer[] {1, 2}},
DataTypes.ARRAY(DataTypes.ARRAY(DataTypes.INT()))),
TestSetSpec.forFunction(BuiltInFunctionDefinitions.MAP_VALUES)
.onFieldsWithData(
null,
"item",
Collections.singletonMap(1, "value1"),
Collections.singletonMap(
3, Collections.singletonMap(true, "value2")))
.andDataTypes(
DataTypes.BOOLEAN().nullable(),
DataTypes.STRING(),
DataTypes.MAP(DataTypes.INT(), DataTypes.STRING()),
DataTypes.MAP(
DataTypes.INT(),
DataTypes.MAP(DataTypes.BOOLEAN(), DataTypes.STRING())))
.testTableApiValidationError(
call("MAP_VALUES", $("f0"), $("f1")),
"Invalid function call:\nMAP_VALUES(BOOLEAN, STRING)")
.testResult(
map(
$("f1").cast(DataTypes.STRING()),
$("f0").cast(DataTypes.BOOLEAN()))
.mapValues(),
"MAP_VALUES(MAP[CAST(f1 AS STRING), CAST(f0 AS BOOLEAN)])",
new Boolean[] {null},
DataTypes.ARRAY(DataTypes.BOOLEAN()).notNull())
.testResult(
$("f2").mapValues(),
"MAP_VALUES(f2)",
new String[] {"value1"},
DataTypes.ARRAY(DataTypes.STRING()))
.testResult(
$("f3").mapValues(),
"MAP_VALUES(f3)",
new Map[] {Collections.singletonMap(true, "value2")},
DataTypes.ARRAY(
DataTypes.MAP(DataTypes.BOOLEAN(), DataTypes.STRING()))),
TestSetSpec.forFunction(BuiltInFunctionDefinitions.MAP_FROM_ARRAYS, "Invalid input")
.onFieldsWithData(null, null, new Integer[] {1}, new Integer[] {1, 2})
.andDataTypes(
DataTypes.ARRAY(DataTypes.BOOLEAN()),
DataTypes.ARRAY(DataTypes.STRING()),
DataTypes.ARRAY(DataTypes.INT()),
DataTypes.ARRAY(DataTypes.INT()))
.testTableApiRuntimeError(
mapFromArrays($("f2"), $("f3")),
"Invalid function MAP_FROM_ARRAYS call:\n"
+ "The length of the keys array 1 is not equal to the length of the values array 2")
.testSqlRuntimeError(
"MAP_FROM_ARRAYS(array[1, 2, 3], array[1, 2])",
"Invalid function MAP_FROM_ARRAYS call:\n"
+ "The length of the keys array 3 is not equal to the length of the values array 2")
.testResult(
mapFromArrays($("f0"), $("f1")),
"MAP_FROM_ARRAYS(f0, f1)",
null,
DataTypes.MAP(DataTypes.BOOLEAN(), DataTypes.STRING())),
TestSetSpec.forFunction(BuiltInFunctionDefinitions.MAP_FROM_ARRAYS)
.onFieldsWithData(
new Integer[] {1, 2},
new String[] {"one", "two"},
new Integer[][] {new Integer[] {1, 2}, new Integer[] {3, 4}})
.andDataTypes(
DataTypes.ARRAY(DataTypes.INT()),
DataTypes.ARRAY(DataTypes.STRING()),
DataTypes.ARRAY(DataTypes.ARRAY(DataTypes.INT())))
.testResult(
mapFromArrays($("f0"), $("f1")),
"MAP_FROM_ARRAYS(f0, f1)",
CollectionUtil.map(entry(1, "one"), entry(2, "two")),
DataTypes.MAP(DataTypes.INT(), DataTypes.STRING()))
.testTableApiResult(
mapFromArrays($("f1"), $("f2")),
CollectionUtil.map(
entry("one", new Integer[] {1, 2}),
entry("two", new Integer[] {3, 4})),
DataTypes.MAP(
DataTypes.STRING(), DataTypes.ARRAY(DataTypes.INT()))));
}
}

0 comments on commit 46d48a4

Please sign in to comment.