[
https://issues.apache.org/jira/browse/FLINK-7939?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16223723#comment-16223723
]
ASF GitHub Bot commented on FLINK-7939:
---------------------------------------
GitHub user fhueske opened a pull request:
https://github.com/apache/flink/pull/4917
[FLINK-7939] [table] Fix Table conversion of DataStream of AtomicType.
## What is the purpose of the change
Fix the conversion of a `DataStream` of an `AtomicType` to a `Table` with
time attributes.
## Brief change log
* Fix index computation for table fields from `AtomicType`
* Fix field type computation for tables from `AtomicType`
## Verifying this change
Added tests to
* `StreamTableEnvironmentTest`
* `TimeAttributesITCase`
## Does this pull request potentially affect one of the following parts:
- Dependencies (does it add or upgrade a dependency): **no**
- The public API, i.e., is any changed class annotated with
`@Public(Evolving)`: **no**
- The serializers: **no**
- The runtime per-record code paths (performance sensitive): **no**
- Anything that affects deployment or recovery: JobManager (and its
components), Checkpointing, Yarn/Mesos, ZooKeeper: **no**
## Documentation
Fix doesn't need documentation.
You can merge this pull request into a Git repository by running:
$ git pull https://github.com/fhueske/flink tableAtomicStream
Alternatively you can review and apply these changes as the patch at:
https://github.com/apache/flink/pull/4917.patch
To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:
This closes #4917
----
commit 437eeba6c0d78143a53654517bb4ace47b9bf53a
Author: Fabian Hueske <[email protected]>
Date: 2017-10-28T20:13:23Z
[FLINK-7939] [table] Fix Table conversion of DataStream of AtomicType.
----
> DataStream of atomic type cannot be converted to Table with time attributes
> ---------------------------------------------------------------------------
>
> Key: FLINK-7939
> URL: https://issues.apache.org/jira/browse/FLINK-7939
> Project: Flink
> Issue Type: Bug
> Components: Table API & SQL
> Affects Versions: 1.4.0, 1.3.3
> Reporter: Fabian Hueske
> Assignee: Fabian Hueske
> Fix For: 1.4.0, 1.3.3
>
>
> A DataStream of an atomic type, such as {{DataStream<String>}} or
> {{DataStream<Long>}} cannot be converted into a {{Table}} with a time
> attribute.
> {code}
> DataStream<String> stream = ...
> Table table = tEnv.fromDataStream(stream, "string, rowtime.rowtime")
> {code}
> yields
> {code}
> Exception in thread "main" org.apache.flink.table.api.TableException: Field
> reference expression requested.
> at
> org.apache.flink.table.api.TableEnvironment$$anonfun$1.apply(TableEnvironment.scala:630)
> at
> org.apache.flink.table.api.TableEnvironment$$anonfun$1.apply(TableEnvironment.scala:624)
> at
> scala.collection.TraversableLike$$anonfun$flatMap$1.apply(TraversableLike.scala:241)
> at
> scala.collection.TraversableLike$$anonfun$flatMap$1.apply(TraversableLike.scala:241)
> at
> scala.collection.IndexedSeqOptimized$class.foreach(IndexedSeqOptimized.scala:33)
> at scala.collection.mutable.ArrayOps$ofRef.foreach(ArrayOps.scala:186)
> at
> scala.collection.TraversableLike$class.flatMap(TraversableLike.scala:241)
> at scala.collection.mutable.ArrayOps$ofRef.flatMap(ArrayOps.scala:186)
> at
> org.apache.flink.table.api.TableEnvironment.getFieldInfo(TableEnvironment.scala:624)
> at
> org.apache.flink.table.api.StreamTableEnvironment.registerDataStreamInternal(StreamTableEnvironment.scala:398)
> at
> org.apache.flink.table.api.scala.StreamTableEnvironment.fromDataStream(StreamTableEnvironment.scala:85)
> {code}
> As a workaround the atomic type can be wrapped in {{Tuple1}}, i.e., convert a
> {{DataStream<String>}} into a {{DataStream<Tuple1<String>>}}.
--
This message was sent by Atlassian JIRA
(v6.4.14#64029)