[
https://issues.apache.org/jira/browse/FLINK-6250?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15984779#comment-15984779
]
ASF GitHub Bot commented on FLINK-6250:
---------------------------------------
Github user fhueske commented on a diff in the pull request:
https://github.com/apache/flink/pull/3771#discussion_r113442648
--- Diff:
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/CodeGenerator.scala
---
@@ -296,6 +297,41 @@ class CodeGenerator(
fields.mkString(", ")
}
+def genInitialize(existDistinct : Boolean): String = {
+
+ val sig: String =
+ j"""
+ | org.apache.flink.api.common.state.MapState[] distStateList =
+ | new org.apache.flink.api.common.state.MapState[
${distinctAggsFlags.size} ];
+ |
+ | public void initialize(
+ | org.apache.flink.api.common.functions.RuntimeContext ctx
+ | )""".stripMargin
+ if(existDistinct){
+ val initDist: String = {
+ for(i <- distinctAggsFlags.indices) yield
+ if( distinctAggsFlags(i)) {
+ j"""
+ |
+ |
org.apache.flink.api.common.state.MapStateDescriptor<Object, Long> distDesc$i =
--- End diff --
We should use the right type of the field instead of `Object`. With
`Object`, we will use the GenericTypeSerializer instead of Flink's
type-specific serializers.
> Distinct procTime with Rows boundaries
> --------------------------------------
>
> Key: FLINK-6250
> URL: https://issues.apache.org/jira/browse/FLINK-6250
> Project: Flink
> Issue Type: Sub-task
> Components: Table API & SQL
> Reporter: radu
> Assignee: Stefano Bortoli
>
> Support proctime with rows boundaries
> Q1.1. `SELECT SUM( DISTINCT b) OVER (ORDER BY procTime() ROWS BETWEEN 2
> PRECEDING AND CURRENT ROW) FROM stream1`
> Q1.1. `SELECT COUNT(b), SUM( DISTINCT b) OVER (ORDER BY procTime() ROWS
> BETWEEN 2 PRECEDING AND CURRENT ROW) FROM stream1`
--
This message was sent by Atlassian JIRA
(v6.3.15#6346)