-
Notifications
You must be signed in to change notification settings - Fork 93
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
Improvements: Add missing Flink TypeInformation for proper serializat…
…ion (#7058)
- Loading branch information
Showing
22 changed files
with
381 additions
and
109 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
32 changes: 27 additions & 5 deletions
32
...ain/scala/pl/touk/nussknacker/engine/flink/util/timestamp/TimestampAssignmentHelper.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -1,22 +1,44 @@ | ||
package pl.touk.nussknacker.engine.flink.util.timestamp | ||
|
||
import org.apache.flink.api.common.typeinfo.TypeInformation | ||
import org.apache.flink.api.common.typeinfo.{TypeInformation, Types} | ||
import org.apache.flink.streaming.api.datastream.DataStream | ||
import org.apache.flink.streaming.api.functions._ | ||
import org.apache.flink.streaming.runtime.operators.windowing.TimestampedValue | ||
import org.apache.flink.util.Collector | ||
import pl.touk.nussknacker.engine.flink.api.timestampwatermark.TimestampWatermarkHandler | ||
|
||
import java.util | ||
|
||
class TimestampAssignmentHelper[T: TypeInformation](timestampAssigner: TimestampWatermarkHandler[TimestampedValue[T]]) { | ||
|
||
def assignWatermarks(stream: DataStream[T]): DataStream[T] = { | ||
val valueTypeInfo: TypeInformation[T] = implicitly[TypeInformation[T]] | ||
|
||
val timestampAssignmentHelperType: TypeInformation[TimestampedValue[T]] = Types.POJO( | ||
classOf[TimestampedValue[T]], | ||
util.Map.of( | ||
"timestamp", | ||
Types.LONG, | ||
"hasTimestamp", // TODO: Should we mark this field? | ||
Types.BOOLEAN, | ||
"value", | ||
valueTypeInfo | ||
) | ||
) | ||
|
||
val timestampedStream = stream | ||
.process((value: T, ctx: ProcessFunction[T, TimestampedValue[T]]#Context, out: Collector[TimestampedValue[T]]) => | ||
out.collect(new TimestampedValue(value, ctx.timestamp())) | ||
.process( | ||
(value: T, ctx: ProcessFunction[T, TimestampedValue[T]]#Context, out: Collector[TimestampedValue[T]]) => | ||
out.collect(new TimestampedValue(value, ctx.timestamp())), | ||
timestampAssignmentHelperType | ||
) | ||
|
||
val withTimestampAssigner = timestampAssigner.assignTimestampAndWatermarks(timestampedStream) | ||
withTimestampAssigner.map((tv: TimestampedValue[T]) => tv.getValue) | ||
timestampAssigner | ||
.assignTimestampAndWatermarks(timestampedStream) | ||
.map( | ||
(tv: TimestampedValue[T]) => tv.getValue, | ||
valueTypeInfo | ||
) | ||
} | ||
|
||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.