twalthr commented on a change in pull request #9354: [FLINK-13568][sql-parser]
DDL create table doesn't allow STRING data …
URL: https://github.com/apache/flink/pull/9354#discussion_r318542971
##########
File path:
flink-table/flink-table-planner/src/test/java/org/apache/flink/table/sqlexec/SqlToOperationConverterTest.java
##########
@@ -163,6 +169,229 @@ public void testSqlInsertWithStaticPartition() {
assertEquals(expectedStaticPartitions,
sinkModifyOperation.getStaticPartitions());
}
+ @Test(expected = AssertionError.class) // TODO: tweak the tests when
FLINK-13604 is fixed.
+ public void testCreateTableWithFullDataTypes() {
+ final String sql = "create table t1(\n" +
+ " f0 CHAR,\n" +
+ " f1 CHAR NOT NULL,\n" +
+ " f2 CHAR NULL,\n" +
+ " f3 CHAR(33),\n" +
+ " f4 VARCHAR,\n" +
+ " f5 VARCHAR(33),\n" +
+ " f6 STRING,\n" +
+ " f7 BOOLEAN,\n" +
+ // " f13 DECIMAL,\n" +
+ // " f14 DEC,\n" +
+ // " f15 NUMERIC,\n" +
+ // " f16 DECIMAL(10),\n" +
+ // " f17 DEC(10),\n" +
+ // " f18 NUMERIC(10),\n" +
+ // " f19 DECIMAL(10, 3),\n" +
+ // " f20 DEC(10, 3),\n" +
+ // " f21 NUMERIC(10, 3),\n" +
+ " f22 TINYINT,\n" +
+ " f23 SMALLINT,\n" +
+ " f24 INTEGER,\n" +
+ " f25 INT,\n" +
+ " f26 BIGINT,\n" +
+ " f27 FLOAT,\n" +
+ " f28 DOUBLE,\n" +
+ " f29 DOUBLE PRECISION,\n" +
+ " f30 DATE,\n" +
+ " f31 TIME,\n" +
+ " f32 TIME WITHOUT TIME ZONE,\n" +
+ " f33 TIME(3),\n" +
+ " f34 TIME(3) WITHOUT TIME ZONE,\n" +
+ " f35 TIMESTAMP,\n" +
+ " f36 TIMESTAMP WITHOUT TIME ZONE,\n" +
+ " f37 TIMESTAMP(3),\n" +
+ " f38 TIMESTAMP(3) WITHOUT TIME ZONE,\n" +
+ " f42 ARRAY<INT NOT NULL>,\n" +
+ " f43 INT ARRAY,\n" +
+ " f44 INT NOT NULL ARRAY,\n" +
+ " f45 INT ARRAY NOT NULL,\n" +
+ " f46 MULTISET<INT NOT NULL>,\n" +
+ " f47 INT MULTISET,\n" +
+ " f48 INT NOT NULL MULTISET,\n" +
+ " f49 INT MULTISET NOT NULL,\n" +
+ " f50 MAP<BIGINT, BOOLEAN>,\n" +
+ " f51 ROW<f0 INT NOT NULL, f1 BOOLEAN>,\n" +
+ " f52 ROW(f0 INT NOT NULL, f1 BOOLEAN),\n" +
+ " f53 ROW<`f0` INT>,\n" +
+ " f54 ROW(`f0` INT),\n" +
+ " f55 ROW<>,\n" +
+ " f56 ROW(),\n" +
+ " f57 ROW<f0 INT NOT NULL 'This is a comment.', f1
BOOLEAN 'This as well.'>)";
+ final FlinkPlannerImpl planner =
getPlannerBySqlDialect(SqlDialect.DEFAULT);
+ SqlNode node = planner.parse(sql);
+ assert node instanceof SqlCreateTable;
+ Operation operation = SqlToOperationConverter.convert(planner,
node);
+ assert operation instanceof CreateTableOperation;
+ TableSchema schema = ((CreateTableOperation)
operation).getCatalogTable().getSchema();
+ assertArrayEquals(new DataType[] {
+ // expect to be CHAR(1)
+ DataTypes.STRING(),
+ // expect to be CHAR(1) NOT NULL
+ DataTypes.STRING(),
+ // expect to be CHAR(1)
+ DataTypes.STRING(),
+ // expect to be CHAR(33)
+ DataTypes.STRING(),
+ DataTypes.STRING(),
+ // expect to be VARCHAR(33)
+ DataTypes.STRING(),
+ DataTypes.STRING(),
+ DataTypes.BOOLEAN(),
+ // DataTypes.DECIMAL(10, 0),
+ // DataTypes.DECIMAL(10, 0),
+ // DataTypes.DECIMAL(10, 0),
+ // DataTypes.DECIMAL(10, 0),
+ // DataTypes.DECIMAL(10, 0),
+ // DataTypes.DECIMAL(10, 0),
+ // DataTypes.DECIMAL(10, 3),
+ // DataTypes.DECIMAL(10, 3),
+ // DataTypes.DECIMAL(10, 3),
+ DataTypes.TINYINT(),
+ DataTypes.SMALLINT(),
+ DataTypes.INT(),
+ DataTypes.INT(),
+ DataTypes.BIGINT(),
+ DataTypes.FLOAT(),
+ DataTypes.DOUBLE(),
+ DataTypes.DOUBLE(),
+ DataTypes.DATE(),
+ DataTypes.TIME(),
+ DataTypes.TIME(),
+ // expect to be TIME(3), the precision is lost.
+ DataTypes.TIME(),
+ // expect to be TIME(3), the precision is lost.
+ DataTypes.TIME(),
+ DataTypes.TIMESTAMP(3),
+ DataTypes.TIMESTAMP(3),
+ DataTypes.TIMESTAMP(3),
+ DataTypes.TIMESTAMP(3),
+ DataTypes.ARRAY(DataTypes.INT().notNull()),
+ DataTypes.ARRAY(DataTypes.INT()),
+ DataTypes.ARRAY(DataTypes.INT().notNull()),
+ DataTypes.ARRAY(DataTypes.INT()).notNull(),
+ DataTypes.MULTISET(DataTypes.INT().notNull()),
+ DataTypes.MULTISET(DataTypes.INT()),
+ DataTypes.MULTISET(DataTypes.INT().notNull()),
+ DataTypes.MULTISET(DataTypes.INT()).notNull(),
+ DataTypes.MAP(DataTypes.BIGINT(), DataTypes.BOOLEAN()),
+ // expect to be ROW<`f0` INT NOT NULL, `f1` BOOLEAN>,
nullable attr is lost.
+ DataTypes.ROW(DataTypes.FIELD("f0", DataTypes.INT()),
+ DataTypes.FIELD("f1", DataTypes.BOOLEAN())),
+ // expect to be ROW<`f0` INT NOT NULL, `f1` BOOLEAN>,
nullable attr is lost.
+ DataTypes.ROW(DataTypes.FIELD("f0", DataTypes.INT()),
+ DataTypes.FIELD("f1", DataTypes.BOOLEAN())),
+ DataTypes.ROW(DataTypes.FIELD("f0", DataTypes.INT())),
+ DataTypes.ROW(DataTypes.FIELD("f0", DataTypes.INT())),
+ DataTypes.ROW(),
+ DataTypes.ROW(),
+ // should be ROW<`f0` INT NOT NULL '...', `f1` BOOLEAN
'...'>, the nullable attr
+ // and comments are both lost
+ DataTypes.ROW(DataTypes.FIELD("f0", DataTypes.INT()),
+ DataTypes.FIELD("f1", DataTypes.BOOLEAN()))
+ }, schema.getFieldDataTypes());
+ }
+
+ @Test
+ public void testCreateTableWithUnSupportedDataType1() {
+ expectedEx.expect(TableException.class);
+ expectedEx.expectMessage("Type is not supported:
TIMESTAMP_WITH_LOCAL_TIME_ZONE");
+ final String sql = "create table t1(\n" +
+ " f41 ARRAY<TIMESTAMP(3) WITH LOCAL TIME ZONE>)";
+ final FlinkPlannerImpl planner =
getPlannerBySqlDialect(SqlDialect.DEFAULT);
+ SqlNode node = planner.parse(sql);
+ assert node instanceof SqlCreateTable;
+ SqlToOperationConverter.convert(planner, node);
+ }
+
+ @Test
+ public void testCreateTableWithUnSupportedDataType2() {
+ expectedEx.expect(TableException.class);
+ expectedEx.expectMessage("Type is not supported:
TIMESTAMP_WITH_LOCAL_TIME_ZONE");
+ final String sql = "create table t1(\n" +
+ " f40 TIMESTAMP(3) WITH LOCAL TIME ZONE)";
+ final FlinkPlannerImpl planner =
getPlannerBySqlDialect(SqlDialect.DEFAULT);
+ SqlNode node = planner.parse(sql);
+ assert node instanceof SqlCreateTable;
+ SqlToOperationConverter.convert(planner, node);
+ }
+
+ @Test
+ public void testCreateTableWithUnSupportedDataType3() {
+ expectedEx.expect(TableException.class);
+ expectedEx.expectMessage("Type is not supported:
TIMESTAMP_WITH_LOCAL_TIME_ZONE");
+ final String sql = "create table t1(\n" +
+ " f39 TIMESTAMP WITH LOCAL TIME ZONE)";
+ final FlinkPlannerImpl planner =
getPlannerBySqlDialect(SqlDialect.DEFAULT);
+ SqlNode node = planner.parse(sql);
+ assert node instanceof SqlCreateTable;
+ SqlToOperationConverter.convert(planner, node);
+ }
+
+ @Test
+ public void testCreateTableWithUnSupportedDataType4() {
+ expectedEx.expect(TableException.class);
+ expectedEx.expectMessage("Type is not supported: VARBINARY");
Review comment:
if the exception is generated with `LogicalType#asSummaryString` `BYTES`
should be the representation
----------------------------------------------------------------
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.
For queries about this service, please contact Infrastructure at:
[email protected]
With regards,
Apache Git Services