Details
-
Bug
-
Status: Closed
-
Critical
-
Resolution: Fixed
-
None
Description
The following test
@Test public void test() throws Exception { hiveShell.execute("create table src (x int,y string)"); hiveShell.insertInto("default", "src").addRow(1, "a").commit(); hiveShell.execute("create table dest (x int) partitioned by (p1 int,p2 string)"); TableEnvironment tableEnvironment = getTableEnvWithHiveCatalog(); tableEnvironment.executeSql("insert into dest partition (p1=1) select * from src") .getJobClient().get().getJobExecutionResult(Thread.currentThread().getContextClassLoader()).get(); }
Fails with
org.apache.flink.table.api.ValidationException: Field types of query result and registered TableSink test-catalog.default.dest do not match. Query schema: [x: INT, y: VARCHAR(2147483647), EXPR$2: INT NOT NULL] Sink schema: [x: INT, p1: INT, p2: VARCHAR(2147483647)] at org.apache.flink.table.planner.sinks.TableSinkUtils$.validateSchemaAndApplyImplicitCast(TableSinkUtils.scala:95) at org.apache.flink.table.planner.delegation.PlannerBase$$anonfun$2.apply(PlannerBase.scala:199) at org.apache.flink.table.planner.delegation.PlannerBase$$anonfun$2.apply(PlannerBase.scala:191) at scala.Option.map(Option.scala:146) at org.apache.flink.table.planner.delegation.PlannerBase.translateToRel(PlannerBase.scala:191) at org.apache.flink.table.planner.delegation.PlannerBase$$anonfun$1.apply(PlannerBase.scala:150) at org.apache.flink.table.planner.delegation.PlannerBase$$anonfun$1.apply(PlannerBase.scala:150) at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234) at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234) at scala.collection.Iterator$class.foreach(Iterator.scala:891) at scala.collection.AbstractIterator.foreach(Iterator.scala:1334) at scala.collection.IterableLike$class.foreach(IterableLike.scala:72) at scala.collection.AbstractIterable.foreach(Iterable.scala:54) at scala.collection.TraversableLike$class.map(TraversableLike.scala:234) at scala.collection.AbstractTraversable.map(Traversable.scala:104) at org.apache.flink.table.planner.delegation.PlannerBase.translate(PlannerBase.scala:150) at org.apache.flink.table.api.internal.TableEnvironmentImpl.translate(TableEnvironmentImpl.java:1202) at org.apache.flink.table.api.internal.TableEnvironmentImpl.executeInternal(TableEnvironmentImpl.java:687) at org.apache.flink.table.api.internal.TableEnvironmentImpl.executeOperation(TableEnvironmentImpl.java:773) at org.apache.flink.table.api.internal.TableEnvironmentImpl.executeSql(TableEnvironmentImpl.java:677) ......
However, the same DML passes if I change "select *" to "select x,y" in the query.
Attachments
Issue Links
- links to