[
https://issues.apache.org/jira/browse/FLINK-7337?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16124599#comment-16124599
]
ASF GitHub Bot commented on FLINK-7337:
---------------------------------------
Github user wuchong commented on a diff in the pull request:
https://github.com/apache/flink/pull/4488#discussion_r132817997
--- Diff:
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/CodeGenerator.scala
---
@@ -1247,27 +1258,32 @@ abstract class CodeGenerator(
}
}
- private[flink] def generateRecordTimestamp(isEventTime: Boolean):
GeneratedExpression = {
+ private[flink] def generateRowtimeAccess(): GeneratedExpression = {
val resultTerm = newName("result")
- val resultTypeTerm =
primitiveTypeTermForTypeInfo(SqlTimeTypeInfo.TIMESTAMP)
+ val nullTerm = newName("isNull")
- val resultCode = if (isEventTime) {
+ val accessCode =
s"""
- |$resultTypeTerm $resultTerm;
- |if ($contextTerm.timestamp() == null) {
+ |Long $resultTerm = $contextTerm.timestamp();
+ |if ($resultTerm == null) {
| throw new RuntimeException("Rowtime timestamp is null. Please
make sure that a proper " +
| "TimestampAssigner is defined and the stream environment uses
the EventTime time " +
| "characteristic.");
|}
- |else {
- | $resultTerm = $contextTerm.timestamp();
- |}
- |""".stripMargin
- } else {
+ |boolean $nullTerm = false;
+ """.stripMargin
+
+ GeneratedExpression(resultTerm, nullTerm, accessCode,
TimeIndicatorTypeInfo.ROWTIME_INDICATOR)
+ }
+
+ private[flink] def generateProctimeTimestamp(): GeneratedExpression = {
+ val resultTerm = newName("result")
+ val resultTypeTerm =
primitiveTypeTermForTypeInfo(SqlTimeTypeInfo.TIMESTAMP)
+
+ val resultCode =
s"""
|$resultTypeTerm $resultTerm =
$contextTerm.timerService().currentProcessingTime();
--- End diff --
Why not hardcode the `$resultTypeTerm` as `long` ? The
`currentProcessingTime()` always returns `long` primitive type.
> Refactor handling of time indicator attributes
> ----------------------------------------------
>
> Key: FLINK-7337
> URL: https://issues.apache.org/jira/browse/FLINK-7337
> Project: Flink
> Issue Type: Improvement
> Components: Table API & SQL
> Affects Versions: 1.4.0
> Reporter: Fabian Hueske
> Assignee: Fabian Hueske
>
> After a [discussion on the dev mailing
> list|https://lists.apache.org/thread.html/735d55f9022df8ff73566a9f1553e14be94f8443986ad46559b35869@%3Cdev.flink.apache.org%3E]
> I propose the following changes to the current handling of time indicator
> attributes:
> * Remove the separation of logical and physical row type.
> ** Hold the event-time timestamp as regular Long field in Row
> ** Represent the processing-time indicator type as a null-valued field in Row
> (1 bit overhead)
> * Remove materialization of event-time timestamps because timestamp is
> already accessible in Row.
> * Add {{ProcessFunction}} to set timestamp into the timestamp field of a
> {{StreamRecord}}.
--
This message was sent by Atlassian JIRA
(v6.4.14#64029)