[ 
https://issues.apache.org/jira/browse/FLINK-10167?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16587442#comment-16587442
 ] 

Andrew Roberts commented on FLINK-10167:
----------------------------------------

Can you clarify what you mean by "values"? Tuples are also values - do windows 
only work when the key is concretely expressed as part of a tuple in the stream?

 

Can you point me to any of the reasoning behind this choice? This toy job was 
just the simplest reproduction of the issue I could produce. In my actual job, 
I switched to "ingest time" and re-implemented `EventTimeSessionWindows` to 
expose its type parameters, and I am seeing exceptions in this shape:
{code:java}
java.io.IOException: Exception while applying ReduceFunction in reducing state
at 
org.apache.flink.runtime.state.heap.HeapReducingState.add(HeapReducingState.java:82)
at 
org.apache.flink.streaming.runtime.operators.windowing.WindowOperator.processElement(WindowOperator.java:409)
at 
org.apache.flink.streaming.runtime.io.StreamInputProcessor.processInput(StreamInputProcessor.java:206)
at 
org.apache.flink.streaming.runtime.tasks.OneInputStreamTask.run(OneInputStreamTask.java:69)
at 
org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:263)
at org.apache.flink.runtime.taskmanager.Task.run(Task.java:702)
at java.lang.Thread.run(Thread.java:748)
Caused by: java.lang.IllegalArgumentException: Key group index out of range of 
key group range [99, 101).
at 
org.apache.flink.runtime.state.heap.NestedMapsStateTable.setMapForKeyGroup(NestedMapsStateTable.java:104)
at 
org.apache.flink.runtime.state.heap.NestedMapsStateTable.transform(NestedMapsStateTable.java:288)
at 
org.apache.flink.runtime.state.heap.HeapReducingState.add(HeapReducingState.java:80)
... 6 more{code}
I'm keying my stream by a string `val` provided by the object passing through 
the stream, but I'm not creating a tuple. Is this issue because I'm not passing 
the key along in the data stream?

> SessionWindows not compatible with typed DataStreams in scala
> -------------------------------------------------------------
>
>                 Key: FLINK-10167
>                 URL: https://issues.apache.org/jira/browse/FLINK-10167
>             Project: Flink
>          Issue Type: Bug
>            Reporter: Andrew Roberts
>            Priority: Major
>
> I'm trying to construct a trivial job that uses session windows, and it looks 
> like the data type parameter is hardcoded to `Object`/`AnyRef`. Due to the 
> invariance of java classes in scala, this means that we can't use the 
> provided SessionWindow helper classes in scala on typed streams.
>  
> Example job:
> {code:java}
> import org.apache.flink.api.scala._
> import org.apache.flink.streaming.api.scala.{DataStream, 
> StreamExecutionEnvironment}
> import 
> org.apache.flink.streaming.api.windowing.assigners.ProcessingTimeSessionWindows
> import org.apache.flink.streaming.api.windowing.time.Time
> import org.apache.flink.streaming.api.windowing.windows.{TimeWindow, Window}
> import org.apache.flink.util.Collector
> object TestJob {
>   val jobName = "TestJob"
>   def main(args: Array[String]): Unit = {
>     val env = StreamExecutionEnvironment.getExecutionEnvironment
>     env.fromCollection(Range(0, 100).toList)
>       .keyBy(_ / 10)
>       .window(ProcessingTimeSessionWindows.withGap(Time.minutes(1)))
>       .reduce(
>         (a: Int, b: Int) => a + b,
>         (key: Int, window: Window, items: Iterable[Int], out: 
> Collector[String]) => s"${key}: ${items}"
>       )
>       .map(println(_))
>     env.execute(jobName)
>   }
> }{code}
>  
> Compile error:
> {code:java}
> [error]  found   : 
> org.apache.flink.streaming.api.windowing.assigners.ProcessingTimeSessionWindows
> [error]  required: 
> org.apache.flink.streaming.api.windowing.assigners.WindowAssigner[_ >: Int, ?]
> [error] Note: Object <: Any (and 
> org.apache.flink.streaming.api.windowing.assigners.ProcessingTimeSessionWindows
>  <: 
> org.apache.flink.streaming.api.windowing.assigners.MergingWindowAssigner[Object,org.apache.flink.streaming.api.windowing.windows.TimeWindow]),
>  but Java-defined class WindowAssigner is invariant in type T.
> [error] You may wish to investigate a wildcard type such as `_ <: Any`. (SLS 
> 3.2.10)
> [error]       
> .window(ProcessingTimeSessionWindows.withGap(Time.minutes(1))){code}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to