- Scala API Extensions
- Accept partial functions
- DataSet API
- DataStream API
- Accept partial functions
Scala API Extensions
In order to keep a fair amount of consistency between the Scala and Java APIs, someof the features that allow a high-level of expressiveness in Scala have been leftout from the standard APIs for both batch and streaming.
If you want to enjoy the full Scala experience you can choose to opt-in toextensions that enhance the Scala API via implicit conversions.
To use all the available extensions, you can just add a simple import
for theDataSet API
import org.apache.flink.api.scala.extensions._
or the DataStream API
import org.apache.flink.streaming.api.scala.extensions._
Alternatively, you can import individual extensions a-là-carte to only use thoseyou prefer.
Accept partial functions
Normally, both the DataSet and DataStream APIs don’t accept anonymous patternmatching functions to deconstruct tuples, case classes or collections, like thefollowing:
val data: DataSet[(Int, String, Double)] = // [...]
data.map {
case (id, name, temperature) => // [...]
// The previous line causes the following compilation error:
// "The argument types of an anonymous function must be fully known. (SLS 8.5)"
}
This extension introduces new methods in both the DataSet and DataStream Scala APIthat have a one-to-one correspondence in the extended API. These delegating methodsdo support anonymous pattern matching functions.
DataSet API
Method | Original | Example |
---|---|---|
mapWith | map (DataSet) |
|
mapPartitionWith | mapPartition (DataSet) |
|
flatMapWith | flatMap (DataSet) |
|
filterWith | filter (DataSet) |
|
reduceWith | reduce (DataSet, GroupedDataSet) |
|
reduceGroupWith | reduceGroup (GroupedDataSet) |
|
groupingBy | groupBy (DataSet) |
|
sortGroupWith | sortGroup (GroupedDataSet) |
|
combineGroupWith | combineGroup (GroupedDataSet) |
|
projecting | apply (JoinDataSet, CrossDataSet) |
|
projecting | apply (CoGroupDataSet) |
|
DataStream API
Method | Original | Example |
---|---|---|
mapWith | map (DataStream) |
|
flatMapWith | flatMap (DataStream) |
|
filterWith | filter (DataStream) |
|
keyingBy | keyBy (DataStream) |
|
mapWith | map (ConnectedDataStream) |
|
flatMapWith | flatMap (ConnectedDataStream) |
|
keyingBy | keyBy (ConnectedDataStream) |
|
reduceWith | reduce (KeyedStream, WindowedStream) |
|
foldWith | fold (KeyedStream, WindowedStream) |
|
applyWith | apply (WindowedStream) |
|
projecting | apply (JoinedStream) |
|
For more information on the semantics of each method, please refer to theDataSet and DataStream API documentation.
To use this extension exclusively, you can add the following import
:
import org.apache.flink.api.scala.extensions.acceptPartialFunctions
for the DataSet extensions and
import org.apache.flink.streaming.api.scala.extensions.acceptPartialFunctions
The following snippet shows a minimal example of how to use these extensionmethods together (with the DataSet API):
object Main {
import org.apache.flink.api.scala.extensions._
case class Point(x: Double, y: Double)
def main(args: Array[String]): Unit = {
val env = ExecutionEnvironment.getExecutionEnvironment
val ds = env.fromElements(Point(1, 2), Point(3, 4), Point(5, 6))
ds.filterWith {
case Point(x, _) => x > 1
}.reduceWith {
case (Point(x1, y1), (Point(x2, y2))) => Point(x1 + y1, x2 + y2)
}.mapWith {
case Point(x, y) => (x, y)
}.flatMapWith {
case (x, y) => Seq("x" -> x, "y" -> y)
}.groupingBy {
case (id, value) => id
}
}
}