[
https://issues.apache.org/jira/browse/FLINK-10167?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16588629#comment-16588629
]
Aljoscha Krettek commented on FLINK-10167:
------------------------------------------
I meant values as the Scala language uses values, i.e. subclasses of
{{AnyVal}}, which are roughly those types that the Java language calls
primitives (for reference:
[https://www.scala-lang.org/api/current/scala/AnyVal.html]). A Scala {{Tuple}}
is not a value so it works. The issue is that Java {{Object}} is roughly equal
to Scala {{AnyRef}}, so only subclasses of {{AnyRef}} work here.
Regarding your second question, there was no specific reasoning behind this,
{{Object}} was used here because it allows using streams of any Java type that
is not a primitive, likewise, it works for Scala types that are not values. I
think if we implemented it now and took Scala into consideration more carefully
we would have done it differently.
Regarding your exception, I think this could be caused by the key not being
deterministic. If you post your code I could have a look.
> 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)