You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by fp...@apache.org on 2021/11/24 11:40:03 UTC
[flink] branch master updated: [FLINK-24802][table] Improve cast ROW to STRING
This is an automated email from the ASF dual-hosted git repository.
fpaul pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git
The following commit(s) were added to refs/heads/master by this push:
new 44ed268 [FLINK-24802][table] Improve cast ROW to STRING
44ed268 is described below
commit 44ed268426ac96d27402d57243234387d6e05c48
Author: Shen Zhu <sh...@instacart.com>
AuthorDate: Mon Nov 15 20:29:44 2021 -0800
[FLINK-24802][table] Improve cast ROW to STRING
---
.../functions/AdvancedFunctionsExampleITCase.java | 18 +++++++++---------
.../functions/casting/RowToStringCastRule.java | 2 +-
.../planner/functions/casting/CastRulesTest.java | 9 +++++++--
.../planner/expressions/CompositeAccessTest.scala | 8 ++++----
.../table/planner/expressions/RowTypeTest.scala | 22 +++++++++++-----------
.../planner/expressions/SqlExpressionTest.scala | 6 +++---
6 files changed, 35 insertions(+), 30 deletions(-)
diff --git a/flink-examples/flink-examples-table/src/test/java/org/apache/flink/table/examples/java/functions/AdvancedFunctionsExampleITCase.java b/flink-examples/flink-examples-table/src/test/java/org/apache/flink/table/examples/java/functions/AdvancedFunctionsExampleITCase.java
index 4bc98c2..3306911 100644
--- a/flink-examples/flink-examples-table/src/test/java/org/apache/flink/table/examples/java/functions/AdvancedFunctionsExampleITCase.java
+++ b/flink-examples/flink-examples-table/src/test/java/org/apache/flink/table/examples/java/functions/AdvancedFunctionsExampleITCase.java
@@ -41,41 +41,41 @@ public class AdvancedFunctionsExampleITCase extends ExampleOutputTestBase {
assertThat(
consoleOutput,
containsString(
- "| Guillermo Smith | (5,2020-12-05) |"));
+ "| Guillermo Smith | (5, 2020-12-05) |"));
assertThat(
consoleOutput,
containsString(
- "| John Turner | (12,2020-10-02) |"));
+ "| John Turner | (12, 2020-10-02) |"));
assertThat(
consoleOutput,
containsString(
- "| Brandy Sanders | (1,2020-10-14) |"));
+ "| Brandy Sanders | (1, 2020-10-14) |"));
assertThat(
consoleOutput,
containsString(
- "| Valeria Mendoza | (10,2020-06-02) |"));
+ "| Valeria Mendoza | (10, 2020-06-02) |"));
assertThat(
consoleOutput,
containsString(
- "| Ellen Ortega | (100,2020-06-18) |"));
+ "| Ellen Ortega | (100, 2020-06-18) |"));
assertThat(
consoleOutput,
containsString(
- "| Leann Holloway | (9,2020-05-26) |"));
+ "| Leann Holloway | (9, 2020-05-26) |"));
}
private void testExecuteInternalRowMergerFunction(String consoleOutput) {
assertThat(
consoleOutput,
containsString(
- "| Guillermo Smith | (1992-12-12,New Jersey,816-... |"));
+ "| Guillermo Smith | (1992-12-12, New Jersey, 81... |"));
assertThat(
consoleOutput,
containsString(
- "| Valeria Mendoza | (1970-03-28,Los Angeles,928... |"));
+ "| Valeria Mendoza | (1970-03-28, Los Angeles, 9... |"));
assertThat(
consoleOutput,
containsString(
- "| Leann Holloway | (1989-05-21,Eugene,614-889-... |"));
+ "| Leann Holloway | (1989-05-21, Eugene, 614-88... |"));
}
}
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/RowToStringCastRule.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/RowToStringCastRule.java
index 1264e9d..baa5119 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/RowToStringCastRule.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/RowToStringCastRule.java
@@ -125,7 +125,7 @@ class RowToStringCastRule extends AbstractNullAwareCodeGeneratorCastRule<ArrayDa
// Write the comma
if (fieldIndex != 0) {
- writer.stmt(methodCall(builderTerm, "append", strLiteral(",")));
+ writer.stmt(methodCall(builderTerm, "append", strLiteral(", ")));
}
writer
diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/casting/CastRulesTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/casting/CastRulesTest.java
index 127ece8..55a50de 100644
--- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/casting/CastRulesTest.java
+++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/casting/CastRulesTest.java
@@ -600,11 +600,16 @@ class CastRulesTest {
.fromCase(
ROW(FIELD("f0", INT()), FIELD("f1", STRING())),
GenericRowData.of(123, StringData.fromString("abc")),
- StringData.fromString("(123,abc)"))
+ StringData.fromString("(123, abc)"))
+ .fromCase(
+ ROW(FIELD("f0", STRING()), FIELD("f1", STRING())),
+ GenericRowData.of(
+ StringData.fromString("abc"), StringData.fromString("def")),
+ StringData.fromString("(abc, def)"))
.fromCase(
ROW(FIELD("f0", INT().nullable()), FIELD("f1", STRING())),
GenericRowData.of(null, StringData.fromString("abc")),
- StringData.fromString("(null,abc)"))
+ StringData.fromString("(null, abc)"))
.fromCase(ROW(), GenericRowData.of(), StringData.fromString("()"))
.fromCase(
RAW(LocalDateTime.class, new LocalDateTimeSerializer()),
diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/expressions/CompositeAccessTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/expressions/CompositeAccessTest.scala
index 4a2e7f0..8341b10 100644
--- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/expressions/CompositeAccessTest.scala
+++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/expressions/CompositeAccessTest.scala
@@ -92,15 +92,15 @@ class CompositeAccessTest extends CompositeTypeTestBase {
'f1.get("objectField"),
"f1.get('objectField')",
"testTable.f1.objectField",
- "(25,Timo,false)")
- testSqlApi("f1.objectField", "(25,Timo,false)")
+ "(25, Timo, false)")
+ testSqlApi("f1.objectField", "(25, Timo, false)")
testAllApis(
'f0,
"f0",
"testTable.f0",
- "(42,Bob,true)")
- testSqlApi("f0", "(42,Bob,true)")
+ "(42, Bob, true)")
+ testSqlApi("f0", "(42, Bob, true)")
// flattening (test base only returns first column)
testAllApis(
diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/expressions/RowTypeTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/expressions/RowTypeTest.scala
index fb579d8..3d8e212 100644
--- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/expressions/RowTypeTest.scala
+++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/expressions/RowTypeTest.scala
@@ -34,7 +34,7 @@ class RowTypeTest extends RowTypeTestBase {
row(1, "foo", true),
"row(1, 'foo', true)",
"ROW(1, 'foo', true)",
- "(1,foo,true)")
+ "(1, foo, true)")
// special literal
testTableApi(
@@ -46,25 +46,25 @@ class RowTypeTest extends RowTypeTestBase {
array(1, 2, 3),
map("foo", "bar"),
row(1, true)),
- "(1985-04-11,14:15:16,1985-04-11 14:15:16,0.1,[1, 2, 3],{foo=bar},(1,true))")
+ "(1985-04-11, 14:15:16, 1985-04-11 14:15:16, 0.1, [1, 2, 3], {foo=bar}, (1, true))")
testSqlApi(
"ROW(DATE '1985-04-11', TIME '14:15:16', TIMESTAMP '1985-04-11 14:15:16', " +
"CAST(0.1 AS DECIMAL(2, 1)), ARRAY[1, 2, 3], MAP['foo', 'bar'], row(1, true))",
- "(1985-04-11,14:15:16,1985-04-11 14:15:16,0.1,[1, 2, 3],{foo=bar},(1,true))")
+ "(1985-04-11, 14:15:16, 1985-04-11 14:15:16, 0.1, [1, 2, 3], {foo=bar}, (1, true))")
testSqlApi(
"ROW(DATE '1985-04-11', TIME '14:15:16', TIMESTAMP '1985-04-11 14:15:16.123456', " +
"CAST(0.1 AS DECIMAL(2, 1)), ARRAY[1, 2, 3], MAP['foo', 'bar'], row(1, true))",
- "(1985-04-11,14:15:16,1985-04-11 14:15:16.123456,0.1,[1, 2, 3],{foo=bar},(1,true))")
+ "(1985-04-11, 14:15:16, 1985-04-11 14:15:16.123456, 0.1, [1, 2, 3], {foo=bar}, (1, true))")
testAllApis(
row(1 + 1, 2 * 3, nullOf(DataTypes.STRING())),
"row(1 + 1, 2 * 3, Null(STRING))",
"ROW(1 + 1, 2 * 3, NULLIF(1, 1))",
- "(2,6,null)"
+ "(2, 6, null)"
)
- testSqlApi("(1, 'foo', true)", "(1,foo,true)")
+ testSqlApi("(1, 'foo', true)", "(1, foo, true)")
}
@Test
@@ -73,35 +73,35 @@ class RowTypeTest extends RowTypeTestBase {
row('f0, 'f1),
"row(f0, f1)",
"(f0, f1)",
- "(null,1)"
+ "(null, 1)"
)
testAllApis(
'f2,
"f2",
"f2",
- "(2,foo,true)"
+ "(2, foo, true)"
)
testAllApis(
row('f2, 'f5),
"row(f2, f5)",
"(f2, f5)",
- "((2,foo,true),(foo,null))"
+ "((2, foo, true), (foo, null))"
)
testAllApis(
'f4,
"f4",
"f4",
- "(1984-03-12,0.00000000,[1, 2, 3])"
+ "(1984-03-12, 0.00000000, [1, 2, 3])"
)
testAllApis(
row('f1, "foo", true),
"row(f1, 'foo', true)",
"(f1, 'foo',true)",
- "(1,foo,true)"
+ "(1, foo, true)"
)
}
diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/expressions/SqlExpressionTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/expressions/SqlExpressionTest.scala
index d1cf13b..ab7f015 100644
--- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/expressions/SqlExpressionTest.scala
+++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/expressions/SqlExpressionTest.scala
@@ -366,9 +366,9 @@ class SqlExpressionTest extends ExpressionTestBase {
@Test
def testValueConstructorFunctions(): Unit = {
- testSqlApi("ROW('hello world', 12)", "(hello world,12)")
- testSqlApi("('hello world', 12)", "(hello world,12)")
- testSqlApi("('foo', ('bar', 12))", "(foo,(bar,12))")
+ testSqlApi("ROW('hello world', 12)", "(hello world, 12)")
+ testSqlApi("('hello world', 12)", "(hello world, 12)")
+ testSqlApi("('foo', ('bar', 12))", "(foo, (bar, 12))")
testSqlApi("ARRAY[TRUE, FALSE][2]", "false")
testSqlApi("ARRAY[TRUE, TRUE]", "[true, true]")
testSqlApi("MAP['k1', 'v1', 'k2', 'v2']['k2']", "v2")