[
https://issues.apache.org/jira/browse/BEAM-5930?focusedWorklogId=161490&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-161490
]
ASF GitHub Bot logged work on BEAM-5930:
----------------------------------------
Author: ASF GitHub Bot
Created on: 01/Nov/18 09:07
Start Date: 01/Nov/18 09:07
Worklog Time Spent: 10m
Work Description: mxm closed pull request #6902: [BEAM-5930] Fix Java SDK
harness to be able to access state on timer callbacks.
URL: https://github.com/apache/beam/pull/6902
This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:
As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):
diff --git
a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnApiDoFnRunner.java
b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnApiDoFnRunner.java
index 30129aa258d..60c5ee168a6 100644
---
a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnApiDoFnRunner.java
+++
b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnApiDoFnRunner.java
@@ -22,6 +22,7 @@
import static com.google.common.base.Preconditions.checkState;
import com.google.auto.service.AutoService;
+import com.google.common.base.MoreObjects;
import com.google.common.collect.ImmutableMap;
import java.io.IOException;
import java.util.Collection;
@@ -209,7 +210,7 @@ public void startBundle() {
context.beamFnStateClient,
context.keyCoder,
(Coder<BoundedWindow>) context.windowCoder,
- () -> currentElement,
+ () -> MoreObjects.firstNonNull(currentElement, currentTimer),
() -> currentWindow);
doFnInvoker.invokeStartBundle(startBundleContext);
diff --git
a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/FnApiDoFnRunnerTest.java
b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/FnApiDoFnRunnerTest.java
index a45e56c852a..4c52d05881a 100644
---
a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/FnApiDoFnRunnerTest.java
+++
b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/FnApiDoFnRunnerTest.java
@@ -35,7 +35,6 @@
import java.io.IOException;
import java.io.Serializable;
import java.util.ArrayList;
-import java.util.Collections;
import java.util.List;
import java.util.ServiceLoader;
import org.apache.beam.fn.harness.state.FakeBeamFnStateClient;
@@ -516,6 +515,9 @@ public void testSideInputIsAccessibleForDownstreamCallers()
throws Exception {
}
private static class TestTimerfulDoFn extends DoFn<KV<String, String>,
String> {
+ @StateId("bag")
+ private final StateSpec<BagState<String>> bagStateSpec =
StateSpecs.bag(StringUtf8Coder.of());
+
@TimerId("event")
private final TimerSpec eventTimerSpec =
TimerSpecs.timer(TimeDomain.EVENT_TIME);
@@ -525,9 +527,11 @@ public void
testSideInputIsAccessibleForDownstreamCallers() throws Exception {
@ProcessElement
public void processElement(
ProcessContext context,
+ @StateId("bag") BagState<String> bagState,
@TimerId("event") Timer eventTimeTimer,
@TimerId("processing") Timer processingTimeTimer) {
- context.output("main" + context.element().getKey());
+ context.output("main" + context.element().getKey() +
Iterables.toString(bagState.read()));
+ bagState.add(context.element().getValue());
eventTimeTimer.set(context.timestamp().plus(1L));
processingTimeTimer.offset(Duration.millis(2L));
processingTimeTimer.setRelative();
@@ -536,9 +540,11 @@ public void processElement(
@OnTimer("event")
public void eventTimer(
OnTimerContext context,
+ @StateId("bag") BagState<String> bagState,
@TimerId("event") Timer eventTimeTimer,
@TimerId("processing") Timer processingTimeTimer) {
- context.output("event");
+ context.output("event" + Iterables.toString(bagState.read()));
+ bagState.add("event");
eventTimeTimer.set(context.timestamp().plus(11L));
processingTimeTimer.offset(Duration.millis(12L));
processingTimeTimer.setRelative();
@@ -547,9 +553,11 @@ public void eventTimer(
@OnTimer("processing")
public void processingTimer(
OnTimerContext context,
+ @StateId("bag") BagState<String> bagState,
@TimerId("event") Timer eventTimeTimer,
@TimerId("processing") Timer processingTimeTimer) {
- context.output("processing");
+ context.output("processing" + Iterables.toString(bagState.read()));
+ bagState.add("processing");
eventTimeTimer.set(context.timestamp().plus(21L));
processingTimeTimer.offset(Duration.millis(22L));
processingTimeTimer.setRelative();
@@ -592,7 +600,12 @@ public void testTimers() throws Exception {
.putOutputs("processing", processingTimerOutputPCollectionId)
.build();
- FakeBeamFnStateClient fakeClient = new
FakeBeamFnStateClient(Collections.emptyMap());
+ FakeBeamFnStateClient fakeClient =
+ new FakeBeamFnStateClient(
+ ImmutableMap.of(
+ bagUserStateKey("bag", "X"), encode("X0"),
+ bagUserStateKey("bag", "A"), encode("A0"),
+ bagUserStateKey("bag", "C"), encode("C0")));
List<WindowedValue<String>> mainOutputValues = new ArrayList<>();
List<WindowedValue<KV<String, Timer>>> eventTimerOutputValues = new
ArrayList<>();
@@ -669,22 +682,22 @@ public void testTimers() throws Exception {
eventTimerInput.accept(timerInGlobalWindow("A", new Instant(1400L), new
Instant(2400L)));
eventTimerInput.accept(timerInGlobalWindow("B", new Instant(1500L), new
Instant(2500L)));
eventTimerInput.accept(timerInGlobalWindow("A", new Instant(1600L), new
Instant(2600L)));
- processingTimerInput.accept(timerInGlobalWindow("C", new Instant(1700L),
new Instant(2700L)));
- processingTimerInput.accept(timerInGlobalWindow("D", new Instant(1800L),
new Instant(2800L)));
- processingTimerInput.accept(timerInGlobalWindow("C", new Instant(1900L),
new Instant(2900L)));
+ processingTimerInput.accept(timerInGlobalWindow("X", new Instant(1700L),
new Instant(2700L)));
+ processingTimerInput.accept(timerInGlobalWindow("C", new Instant(1800L),
new Instant(2800L)));
+ processingTimerInput.accept(timerInGlobalWindow("B", new Instant(1900L),
new Instant(2900L)));
assertThat(
mainOutputValues,
contains(
- timestampedValueInGlobalWindow("mainX", new Instant(1000L)),
- timestampedValueInGlobalWindow("mainY", new Instant(1100L)),
- timestampedValueInGlobalWindow("mainX", new Instant(1200L)),
- timestampedValueInGlobalWindow("mainY", new Instant(1300L)),
- timestampedValueInGlobalWindow("event", new Instant(1400L)),
- timestampedValueInGlobalWindow("event", new Instant(1500L)),
- timestampedValueInGlobalWindow("event", new Instant(1600L)),
- timestampedValueInGlobalWindow("processing", new Instant(1700L)),
- timestampedValueInGlobalWindow("processing", new Instant(1800L)),
- timestampedValueInGlobalWindow("processing", new Instant(1900L))));
+ timestampedValueInGlobalWindow("mainX[X0]", new Instant(1000L)),
+ timestampedValueInGlobalWindow("mainY[]", new Instant(1100L)),
+ timestampedValueInGlobalWindow("mainX[X0, X1]", new
Instant(1200L)),
+ timestampedValueInGlobalWindow("mainY[Y1]", new Instant(1300L)),
+ timestampedValueInGlobalWindow("event[A0]", new Instant(1400L)),
+ timestampedValueInGlobalWindow("event[]", new Instant(1500L)),
+ timestampedValueInGlobalWindow("event[A0, event]", new
Instant(1600L)),
+ timestampedValueInGlobalWindow("processing[X0, X1, X2]", new
Instant(1700L)),
+ timestampedValueInGlobalWindow("processing[C0]", new
Instant(1800L)),
+ timestampedValueInGlobalWindow("processing[event]", new
Instant(1900L))));
assertThat(
eventTimerOutputValues,
contains(
@@ -695,9 +708,9 @@ public void testTimers() throws Exception {
timerInGlobalWindow("A", new Instant(1400L), new Instant(1411L)),
timerInGlobalWindow("B", new Instant(1500L), new Instant(1511L)),
timerInGlobalWindow("A", new Instant(1600L), new Instant(1611L)),
- timerInGlobalWindow("C", new Instant(1700L), new Instant(1721L)),
- timerInGlobalWindow("D", new Instant(1800L), new Instant(1821L)),
- timerInGlobalWindow("C", new Instant(1900L), new Instant(1921L))));
+ timerInGlobalWindow("X", new Instant(1700L), new Instant(1721L)),
+ timerInGlobalWindow("C", new Instant(1800L), new Instant(1821L)),
+ timerInGlobalWindow("B", new Instant(1900L), new Instant(1921L))));
assertThat(
processingTimerOutputValues,
contains(
@@ -708,15 +721,23 @@ public void testTimers() throws Exception {
timerInGlobalWindow("A", new Instant(1400L), new Instant(10012L)),
timerInGlobalWindow("B", new Instant(1500L), new Instant(10012L)),
timerInGlobalWindow("A", new Instant(1600L), new Instant(10012L)),
- timerInGlobalWindow("C", new Instant(1700L), new Instant(10022L)),
- timerInGlobalWindow("D", new Instant(1800L), new Instant(10022L)),
- timerInGlobalWindow("C", new Instant(1900L), new
Instant(10022L))));
+ timerInGlobalWindow("X", new Instant(1700L), new Instant(10022L)),
+ timerInGlobalWindow("C", new Instant(1800L), new Instant(10022L)),
+ timerInGlobalWindow("B", new Instant(1900L), new
Instant(10022L))));
mainOutputValues.clear();
Iterables.getOnlyElement(finishFunctions).run();
assertThat(mainOutputValues, empty());
- assertEquals(ImmutableMap.of(), fakeClient.getData());
+ assertEquals(
+ ImmutableMap.<StateKey, ByteString>builder()
+ .put(bagUserStateKey("bag", "X"), encode("X0", "X1", "X2",
"processing"))
+ .put(bagUserStateKey("bag", "Y"), encode("Y1", "Y2"))
+ .put(bagUserStateKey("bag", "A"), encode("A0", "event", "event"))
+ .put(bagUserStateKey("bag", "B"), encode("event", "processing"))
+ .put(bagUserStateKey("bag", "C"), encode("C0", "processing"))
+ .build(),
+ fakeClient.getData());
mainOutputValues.clear();
}
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on 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: 161490)
Time Spent: 0.5h (was: 20m)
> Java SDK harness fails to access state during timer callbacks
> -------------------------------------------------------------
>
> Key: BEAM-5930
> URL: https://issues.apache.org/jira/browse/BEAM-5930
> Project: Beam
> Issue Type: Bug
> Components: sdk-java-harness
> Reporter: Luke Cwik
> Assignee: Luke Cwik
> Priority: Major
> Labels: portability
> Time Spent: 0.5h
> Remaining Estimate: 0h
>
> The FnApiStateAccessor is given a callback to get the current element and it
> is not handling the case where the current element is a timer.
> callback:
> https://github.com/apache/beam/blob/29c443162a2fe4c89d26336b30aa6e3a3bfbade8/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnApiDoFnRunner.java#L212
> where the current "element" gets set:
> https://github.com/apache/beam/blob/29c443162a2fe4c89d26336b30aa6e3a3bfbade8/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnApiDoFnRunner.java#L220
> where the current "timer" gets set:
> https://github.com/apache/beam/blob/29c443162a2fe4c89d26336b30aa6e3a3bfbade8/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnApiDoFnRunner.java#L237
> The easiest fix would be to have the callback return the first non null from
> currentElement/currentTimer but longer term I think we'll want a different
> solution. Alternatively, we could collapse currentElement and currentTimer to
> be currentElementOrTimer which would solve the accessor issue.
> {code:java}
> Caused by: java.util.concurrent.ExecutionException:
> java.lang.RuntimeException: Error received from SDK harness for instruction
> 72: java.util.concurrent.ExecutionException: java.lang.NullPointerException
> at
> java.util.concurrent.CompletableFuture.reportGet(CompletableFuture.java:357)
> at
> java.util.concurrent.CompletableFuture.get(CompletableFuture.java:1895)
> at
> org.apache.beam.sdk.fn.data.CompletableFutureInboundDataClient.awaitCompletion(CompletableFutureInboundDataClient.java:49)
> at
> org.apache.beam.sdk.fn.data.BeamFnDataInboundObserver.awaitCompletion(BeamFnDataInboundObserver.java:90)
> at
> org.apache.beam.fn.harness.BeamFnDataReadRunner.blockTillReadFinishes(BeamFnDataReadRunner.java:185)
> at
> org.apache.beam.fn.harness.control.ProcessBundleHandler.processBundle(ProcessBundleHandler.java:292)
> at
> org.apache.beam.fn.harness.control.BeamFnControlClient.delegateOnInstructionRequestType(BeamFnControlClient.java:161)
> at
> org.apache.beam.fn.harness.control.BeamFnControlClient.lambda$processInstructionRequests$0(BeamFnControlClient.java:145)
> at
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
> at
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
> at java.lang.Thread.run(Thread.java:748)
> Caused by: java.lang.NullPointerException
> at
> org.apache.beam.model.fnexecution.v1.BeamFnApi$StateKey$BagUserState$Builder.setKey(BeamFnApi.java:49694)
> at
> org.apache.beam.fn.harness.state.FnApiStateAccessor.createBagUserStateKey(FnApiStateAccessor.java:451)
> at
> org.apache.beam.fn.harness.state.FnApiStateAccessor.bindBag(FnApiStateAccessor.java:244)
> at
> org.apache.beam.sdk.state.StateSpecs$BagStateSpec.bind(StateSpecs.java:487)
> at
> org.apache.beam.sdk.state.StateSpecs$BagStateSpec.bind(StateSpecs.java:477)
> at
> org.apache.beam.fn.harness.FnApiDoFnRunner$OnTimerContext.state(FnApiDoFnRunner.java:671)
> at StateTest$5$OnTimerInvoker$expiry$ZXhwaXJ5.invokeOnTimer(Unknown
> Source)
> at
> org.apache.beam.sdk.transforms.reflect.ByteBuddyDoFnInvokerFactory$DoFnInvokerBase.invokeOnTimer(ByteBuddyDoFnInvokerFactory.java:187)
> at
> org.apache.beam.fn.harness.FnApiDoFnRunner.processTimer(FnApiDoFnRunner.java:244)
> at
> org.apache.beam.fn.harness.DoFnPTransformRunnerFactory.lambda$createRunnerForPTransform$0(DoFnPTransformRunnerFactory.java:134)
> at
> org.apache.beam.sdk.fn.data.BeamFnDataInboundObserver.accept(BeamFnDataInboundObserver.java:81)
> at
> org.apache.beam.sdk.fn.data.BeamFnDataInboundObserver.accept(BeamFnDataInboundObserver.java:32)
> at
> org.apache.beam.sdk.fn.data.BeamFnDataGrpcMultiplexer$InboundObserver.onNext(BeamFnDataGrpcMultiplexer.java:139)
> at
> org.apache.beam.sdk.fn.data.BeamFnDataGrpcMultiplexer$InboundObserver.onNext(BeamFnDataGrpcMultiplexer.java:125)
> at
> org.apache.beam.sdk.fn.stream.ForwardingClientResponseObserver.onNext(ForwardingClientResponseObserver.java:50)
> at
> org.apache.beam.vendor.grpc.v1.io.grpc.stub.ClientCalls$StreamObserverToCallListenerAdapter.onMessage(ClientCalls.java:407)
> at
> org.apache.beam.vendor.grpc.v1.io.grpc.ForwardingClientCallListener.onMessage(ForwardingClientCallListener.java:33)
> at
> org.apache.beam.vendor.grpc.v1.io.grpc.ForwardingClientCallListener.onMessage(ForwardingClientCallListener.java:33)
> at
> org.apache.beam.vendor.grpc.v1.io.grpc.internal.ClientCallImpl$ClientStreamListenerImpl$1MessagesAvailable.runInContext(ClientCallImpl.java:519)
> at
> org.apache.beam.vendor.grpc.v1.io.grpc.internal.ContextRunnable.run(ContextRunnable.java:37)
> at
> org.apache.beam.vendor.grpc.v1.io.grpc.internal.SerializingExecutor.run(SerializingExecutor.java:123)
> {code}
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)