[
https://issues.apache.org/jira/browse/BEAM-9562?focusedWorklogId=411556&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-411556
]
ASF GitHub Bot logged work on BEAM-9562:
----------------------------------------
Author: ASF GitHub Bot
Created on: 28/Mar/20 00:22
Start Date: 28/Mar/20 00:22
Worklog Time Spent: 10m
Work Description: lukecwik commented on pull request #11199:
[BEAM-9562][WIP] Update Timer encoding with respect of dynamic timers
URL: https://github.com/apache/beam/pull/11199#discussion_r399589315
##########
File path:
runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/Timer.java
##########
@@ -73,59 +107,81 @@
*/
public static class Coder<T> extends StructuredCoder<Timer<T>> {
- public static <T> Coder of(org.apache.beam.sdk.coders.Coder<T>
payloadCoder) {
- return new Coder(payloadCoder);
+ public static <T> Coder<T> of(
+ org.apache.beam.sdk.coders.Coder<T> keyCoder,
+ org.apache.beam.sdk.coders.Coder<? extends BoundedWindow> windowCoder)
{
+ return new Coder<>(keyCoder, windowCoder);
}
- private final org.apache.beam.sdk.coders.Coder<T> payloadCoder;
+ private final org.apache.beam.sdk.coders.Coder<T> keyCoder;
+ private final org.apache.beam.sdk.coders.Coder<Collection<? extends
BoundedWindow>>
+ windowsCoder;
- private Coder(org.apache.beam.sdk.coders.Coder<T> payloadCoder) {
- this.payloadCoder = payloadCoder;
+ private Coder(
+ org.apache.beam.sdk.coders.Coder<T> keyCoder,
+ org.apache.beam.sdk.coders.Coder<? extends BoundedWindow> windowCoder)
{
+ this.keyCoder = keyCoder;
+ this.windowsCoder = (org.apache.beam.sdk.coders.Coder)
CollectionCoder.of(windowCoder);
+ ;
}
@Override
public void encode(Timer<T> timer, OutputStream outStream) throws
CoderException, IOException {
- InstantCoder.of().encode(timer.getTimestamp(), outStream);
- payloadCoder.encode(timer.getPayload(), outStream);
+ keyCoder.encode(timer.getUserKey(), outStream);
+ StringUtf8Coder.of().encode(timer.getDynamicTimerTag(), outStream);
+ BooleanCoder.of().encode(timer.getClearBit(), outStream);
+ if (!timer.getClearBit()) {
+ InstantCoder.of().encode(timer.getFireTimestamp(), outStream);
+ InstantCoder.of().encode(timer.getHoldTimestamp(), outStream);
+ windowsCoder.encode(timer.getWindows(), outStream);
+ PaneInfoCoder.INSTANCE.encode(timer.getPane(), outStream);
+ }
}
@Override
public Timer<T> decode(InputStream inStream) throws CoderException,
IOException {
- Instant instant = InstantCoder.of().decode(inStream);
- T value = payloadCoder.decode(inStream);
- return Timer.of(instant, value);
+ T userKey = keyCoder.decode(inStream);
+ String dynamicTimerTag = StringUtf8Coder.of().decode(inStream);
+ Boolean clearBit = BooleanCoder.of().decode(inStream);
+ if (clearBit) {
+ return Timer.of(userKey, dynamicTimerTag, clearBit);
+ }
+ Instant fireTimestamp = InstantCoder.of().decode(inStream);
+ Instant holeTimestamp = InstantCoder.of().decode(inStream);
Review comment:
```suggestion
Instant holdTimestamp = InstantCoder.of().decode(inStream);
```
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
[email protected]
Issue Time Tracking
-------------------
Worklog Id: (was: 411556)
> Remove timer from PCollection and treat timers as Elements
> -----------------------------------------------------------
>
> Key: BEAM-9562
> URL: https://issues.apache.org/jira/browse/BEAM-9562
> Project: Beam
> Issue Type: New Feature
> Components: sdk-py-harness
> Reporter: Boyuan Zhang
> Assignee: Boyuan Zhang
> Priority: Major
> Time Spent: 6h 10m
> Remaining Estimate: 0h
>
--
This message was sent by Atlassian Jira
(v8.3.4#803005)