diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/game/StatefulTeamScore2.java b/examples/java/src/main/java/org/apache/beam/examples/complete/game/StatefulTeamScore2.java
new file mode 100644
index 0000000000000..faf8996dedc77
--- /dev/null
+++ b/examples/java/src/main/java/org/apache/beam/examples/complete/game/StatefulTeamScore2.java
@@ -0,0 +1,223 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.examples.complete.game;
+
+import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.MoreObjects.firstNonNull;
+
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.beam.examples.common.ExampleUtils;
+import org.apache.beam.examples.complete.game.utils.GameConstants;
+import org.apache.beam.examples.complete.game.utils.WriteToBigQuery.FieldInfo;
+import org.apache.beam.examples.complete.game.utils.WriteWindowedToBigQuery;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.PipelineResult;
+import org.apache.beam.sdk.coders.VarIntCoder;
+import org.apache.beam.sdk.extensions.gcp.options.GcpOptions;
+import org.apache.beam.sdk.io.gcp.pubsub.PubsubIO;
+import org.apache.beam.sdk.options.Default;
+import org.apache.beam.sdk.options.Description;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.state.StateSpec;
+import org.apache.beam.sdk.state.StateSpecs;
+import org.apache.beam.sdk.state.ReadModifyWriteState;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.MapElements;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.TypeDescriptor;
+import org.apache.beam.sdk.values.TypeDescriptors;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
+import org.joda.time.Instant;
+
+/**
+ * This class is part of a series of pipelines that tell a story in a gaming domain. Concepts
+ * include: stateful processing.
+ *
+ *
This pipeline processes an unbounded stream of 'game events'. It uses stateful processing to
+ * aggregate team scores per team and outputs team name and it's total score every time the team
+ * passes a new multiple of a threshold score. For example, multiples of the threshold could be the
+ * corresponding scores required to pass each level of the game. By default, this threshold is set
+ * to 5000.
+ *
+ *
Stateful processing allows us to write pipelines that output based on a runtime state (when a
+ * team reaches a certain score, in every 100 game events etc) without time triggers. See
+ * https://beam.apache.org/blog/2017/02/13/stateful-processing.html for more information on using
+ * stateful processing.
+ *
+ *
Run {@code injector.Injector} to generate pubsub data for this pipeline. The Injector
+ * documentation provides more detail on how to do this.
+ *
+ *
To execute this pipeline, specify the pipeline configuration like this:
+ *
+ *
The BigQuery dataset you specify must already exist. The PubSub topic you specify should be
+ * the same topic to which the Injector is publishing.
+ */
+public class StatefulTeamScore2 extends LeaderBoard {
+
+ /** Options supported by {@link StatefulTeamScore}. */
+ public interface Options extends LeaderBoard.Options {
+
+ @Description("Numeric value, multiple of which is used as threshold for outputting team score.")
+ @Default.Integer(5000)
+ Integer getThresholdScore();
+
+ void setThresholdScore(Integer value);
+ }
+
+ /**
+ * Create a map of information that describes how to write pipeline output to BigQuery. This map
+ * is used to write team score sums.
+ */
+ private static Map>> configureCompleteWindowedTableWrite() {
+
+ Map>> tableConfigure =
+ new HashMap<>();
+ tableConfigure.put(
+ "team", new WriteWindowedToBigQuery.FieldInfo<>("STRING", (c, w) -> c.element().getKey()));
+ tableConfigure.put(
+ "total_score",
+ new WriteWindowedToBigQuery.FieldInfo<>("INTEGER", (c, w) -> c.element().getValue()));
+ tableConfigure.put(
+ "processing_time",
+ new WriteWindowedToBigQuery.FieldInfo<>(
+ "STRING", (c, w) -> GameConstants.DATE_TIME_FORMATTER.print(Instant.now())));
+ return tableConfigure;
+ }
+
+ public static void main(String[] args) throws Exception {
+
+ Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class);
+ // Enforce that this pipeline is always run in streaming mode.
+ options.setStreaming(true);
+ ExampleUtils exampleUtils = new ExampleUtils(options);
+ Pipeline pipeline = Pipeline.create(options);
+
+ pipeline
+ // Read game events from Pub/Sub using custom timestamps, which are extracted from the
+ // pubsub data elements, and parse the data.
+ .apply(
+ PubsubIO.readStrings()
+ .withTimestampAttribute(GameConstants.TIMESTAMP_ATTRIBUTE)
+ .fromTopic(options.getTopic()))
+ .apply("ParseGameEvent", ParDo.of(new ParseEventFn()))
+ // Create mapping. UpdateTeamScore uses team name as key.
+ .apply(
+ "MapTeamAsKey",
+ MapElements.into(
+ TypeDescriptors.kvs(
+ TypeDescriptors.strings(), TypeDescriptor.of(GameActionInfo.class)))
+ .via((GameActionInfo gInfo) -> KV.of(gInfo.team, gInfo)))
+ // Outputs a team's score every time it passes a new multiple of the threshold.
+ .apply("UpdateTeamScore", ParDo.of(new UpdateTeamScoreFn(options.getThresholdScore())))
+ // Write the results to BigQuery.
+ .apply(
+ "WriteTeamLeaders",
+ new WriteWindowedToBigQuery<>(
+ options.as(GcpOptions.class).getProject(),
+ options.getDataset(),
+ options.getLeaderBoardTableName() + "_team_leader",
+ configureCompleteWindowedTableWrite()));
+
+ // Run the pipeline and wait for the pipeline to finish; capture cancellation requests from the
+ // command line.
+ PipelineResult result = pipeline.run();
+ exampleUtils.waitToFinish(result);
+ }
+
+ /**
+ * Tracks each team's score separately in a single state cell and outputs the score every time it
+ * passes a new multiple of a threshold.
+ *
+ *
We use stateful {@link DoFn} because:
+ *
+ *
+ *
State is key-partitioned. Therefore, the score is calculated per team.
+ *
Stateful {@link DoFn} can determine when to output based on the state. This only allows
+ * outputting when a team's score passes a given threshold.
+ *
+ */
+ @VisibleForTesting
+ public static class UpdateTeamScoreFn
+ extends DoFn, KV> {
+
+ private static final String TOTAL_SCORE = "totalScore";
+ private final int thresholdScore;
+
+ public UpdateTeamScoreFn(int thresholdScore) {
+ this.thresholdScore = thresholdScore;
+ }
+
+ /**
+ * Describes the state for storing team score. Let's break down this statement.
+ *
+ *
{@link StateSpec} configures the state cell, which is provided by a runner during pipeline
+ * execution.
+ *
+ *
{@link org.apache.beam.sdk.transforms.DoFn.StateId} annotation assigns an identifier to
+ * the state, which is used to refer the state in {@link
+ * org.apache.beam.sdk.transforms.DoFn.ProcessElement}.
+ *
+ *
A {@link ReadModifyWriteState} stores single value per key and per window. Because our pipeline is
+ * globally windowed in this example, this {@link ReadModifyWriteState} is just key partitioned, with one
+ * score per team. Any other class that extends {@link org.apache.beam.sdk.state.State} can be
+ * used.
+ *
+ *
In order to store the value, the state must be encoded. Therefore, we provide a coder, in
+ * this case the {@link VarIntCoder}. If the coder is not provided as in {@code
+ * StateSpecs.value()}, Beam's coder inference will try to provide a coder automatically.
+ */
+ @StateId(TOTAL_SCORE)
+ private final StateSpec> totalScoreSpec =
+ StateSpecs.readModifyWrite(VarIntCoder.of());
+
+ /**
+ * To use a state cell, annotate a parameter with {@link
+ * org.apache.beam.sdk.transforms.DoFn.StateId} that matches the state declaration. The type of
+ * the parameter should match the {@link StateSpec} type.
+ */
+ @ProcessElement
+ public void processElement(
+ ProcessContext c, @StateId(TOTAL_SCORE) ReadModifyWriteState totalScore) {
+ String teamName = c.element().getKey();
+ GameActionInfo gInfo = c.element().getValue();
+
+ // ReadModifyWriteState cells do not contain a default value. If the state is possibly not written, make
+ // sure to check for null on read.
+ int oldTotalScore = firstNonNull(totalScore.read(), 0);
+ totalScore.write(oldTotalScore + gInfo.score);
+
+ // Since there are no negative scores, the easiest way to check whether a team just passed a
+ // new multiple of the threshold score is to compare the quotients of dividing total scores by
+ // threshold before and after this aggregation. For example, if the total score was 1999,
+ // the new total is 2002, and the threshold is 1000, 1999 / 1000 = 1, 2002 / 1000 = 2.
+ // Therefore, this team passed the threshold.
+ if (oldTotalScore / this.thresholdScore < totalScore.read() / this.thresholdScore) {
+ c.output(KV.of(teamName, totalScore.read()));
+ }
+ }
+ }
+}
diff --git a/examples/java/src/test/java/org/apache/beam/examples/complete/game/StatefulTeamScoreTest2.java b/examples/java/src/test/java/org/apache/beam/examples/complete/game/StatefulTeamScoreTest2.java
new file mode 100644
index 0000000000000..078a4bddd8add
--- /dev/null
+++ b/examples/java/src/test/java/org/apache/beam/examples/complete/game/StatefulTeamScoreTest2.java
@@ -0,0 +1,183 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.examples.complete.game;
+
+import org.apache.beam.examples.complete.game.StatefulTeamScore2.UpdateTeamScoreFn;
+import org.apache.beam.examples.complete.game.UserScore.GameActionInfo;
+import org.apache.beam.sdk.coders.AvroCoder;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.testing.TestStream;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.windowing.FixedWindows;
+import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
+import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.TimestampedValue;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/** Tests for {@link StatefulTeamScore2}. */
+@RunWith(JUnit4.class)
+public class StatefulTeamScoreTest2 {
+
+ private Instant baseTime = new Instant(0);
+
+ @Rule public TestPipeline p = TestPipeline.create();
+
+ /** Some example users, on two separate teams. */
+ private enum TestUser {
+ RED_ONE("scarlet", "red"),
+ RED_TWO("burgundy", "red"),
+ BLUE_ONE("navy", "blue"),
+ BLUE_TWO("sky", "blue");
+
+ private final String userName;
+ private final String teamName;
+
+ TestUser(String userName, String teamName) {
+ this.userName = userName;
+ this.teamName = teamName;
+ }
+
+ public String getUser() {
+ return userName;
+ }
+
+ public String getTeam() {
+ return teamName;
+ }
+ }
+
+ /**
+ * Tests that {@link UpdateTeamScoreFn} {@link org.apache.beam.sdk.transforms.DoFn} outputs
+ * correctly for one team.
+ */
+ @Test
+ public void testScoreUpdatesOneTeam() {
+
+ TestStream> createEvents =
+ TestStream.create(KvCoder.of(StringUtf8Coder.of(), AvroCoder.of(GameActionInfo.class)))
+ .advanceWatermarkTo(baseTime)
+ .addElements(
+ event(TestUser.RED_TWO, 99, Duration.standardSeconds(10)),
+ event(TestUser.RED_ONE, 1, Duration.standardSeconds(20)),
+ event(TestUser.RED_ONE, 0, Duration.standardSeconds(30)),
+ event(TestUser.RED_TWO, 100, Duration.standardSeconds(40)),
+ event(TestUser.RED_TWO, 201, Duration.standardSeconds(50)))
+ .advanceWatermarkToInfinity();
+
+ PCollection> teamScores =
+ p.apply(createEvents).apply(ParDo.of(new UpdateTeamScoreFn(100)));
+
+ String redTeam = TestUser.RED_ONE.getTeam();
+
+ PAssert.that(teamScores)
+ .inWindow(GlobalWindow.INSTANCE)
+ .containsInAnyOrder(KV.of(redTeam, 100), KV.of(redTeam, 200), KV.of(redTeam, 401));
+
+ p.run().waitUntilFinish();
+ }
+
+ /**
+ * Tests that {@link UpdateTeamScoreFn} {@link org.apache.beam.sdk.transforms.DoFn} outputs
+ * correctly for multiple teams.
+ */
+ @Test
+ public void testScoreUpdatesPerTeam() {
+
+ TestStream> createEvents =
+ TestStream.create(KvCoder.of(StringUtf8Coder.of(), AvroCoder.of(GameActionInfo.class)))
+ .advanceWatermarkTo(baseTime)
+ .addElements(
+ event(TestUser.RED_ONE, 50, Duration.standardSeconds(10)),
+ event(TestUser.RED_TWO, 50, Duration.standardSeconds(20)),
+ event(TestUser.BLUE_ONE, 70, Duration.standardSeconds(30)),
+ event(TestUser.BLUE_TWO, 80, Duration.standardSeconds(40)),
+ event(TestUser.BLUE_TWO, 50, Duration.standardSeconds(50)))
+ .advanceWatermarkToInfinity();
+
+ PCollection> teamScores =
+ p.apply(createEvents).apply(ParDo.of(new UpdateTeamScoreFn(100)));
+
+ String redTeam = TestUser.RED_ONE.getTeam();
+ String blueTeam = TestUser.BLUE_ONE.getTeam();
+
+ PAssert.that(teamScores)
+ .inWindow(GlobalWindow.INSTANCE)
+ .containsInAnyOrder(KV.of(redTeam, 100), KV.of(blueTeam, 150), KV.of(blueTeam, 200));
+
+ p.run().waitUntilFinish();
+ }
+
+ /**
+ * Tests that {@link UpdateTeamScoreFn} {@link org.apache.beam.sdk.transforms.DoFn} outputs
+ * correctly per window and per key.
+ */
+ @Test
+ public void testScoreUpdatesPerWindow() {
+
+ TestStream> createEvents =
+ TestStream.create(KvCoder.of(StringUtf8Coder.of(), AvroCoder.of(GameActionInfo.class)))
+ .advanceWatermarkTo(baseTime)
+ .addElements(
+ event(TestUser.RED_ONE, 50, Duration.standardMinutes(1)),
+ event(TestUser.RED_TWO, 50, Duration.standardMinutes(2)),
+ event(TestUser.RED_ONE, 50, Duration.standardMinutes(3)),
+ event(TestUser.RED_ONE, 60, Duration.standardMinutes(6)),
+ event(TestUser.RED_TWO, 60, Duration.standardMinutes(7)))
+ .advanceWatermarkToInfinity();
+
+ Duration teamWindowDuration = Duration.standardMinutes(5);
+
+ PCollection> teamScores =
+ p.apply(createEvents)
+ .apply(Window.>into(FixedWindows.of(teamWindowDuration)))
+ .apply(ParDo.of(new UpdateTeamScoreFn(100)));
+
+ String redTeam = TestUser.RED_ONE.getTeam();
+ String blueTeam = TestUser.BLUE_ONE.getTeam();
+
+ IntervalWindow window1 = new IntervalWindow(baseTime, teamWindowDuration);
+ IntervalWindow window2 = new IntervalWindow(window1.end(), teamWindowDuration);
+
+ PAssert.that(teamScores).inWindow(window1).containsInAnyOrder(KV.of(redTeam, 100));
+
+ PAssert.that(teamScores).inWindow(window2).containsInAnyOrder(KV.of(redTeam, 120));
+
+ p.run().waitUntilFinish();
+ }
+
+ private TimestampedValue> event(
+ TestUser user, int score, Duration baseTimeOffset) {
+ return TimestampedValue.of(
+ KV.of(
+ user.getTeam(),
+ new GameActionInfo(
+ user.getUser(), user.getTeam(), score, baseTime.plus(baseTimeOffset).getMillis())),
+ baseTime.plus(baseTimeOffset));
+ }
+}
diff --git a/model/pipeline/src/main/proto/beam_runner_api.proto b/model/pipeline/src/main/proto/beam_runner_api.proto
index 204c408ba1a4c..fa5adec3dc419 100644
--- a/model/pipeline/src/main/proto/beam_runner_api.proto
+++ b/model/pipeline/src/main/proto/beam_runner_api.proto
@@ -411,7 +411,7 @@ message Parameter {
message StateSpec {
oneof spec {
- ValueStateSpec value_spec = 1;
+ ReadModifyWriteStateSpec read_modify_write_spec = 1;
BagStateSpec bag_spec = 2;
CombiningStateSpec combining_spec = 3;
MapStateSpec map_spec = 4;
@@ -419,7 +419,7 @@ message StateSpec {
}
}
-message ValueStateSpec {
+message ReadModifyWriteStateSpec {
string coder_id = 1;
}
diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ParDoTranslation.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ParDoTranslation.java
index 067a651e0b509..6b0648f71e1dc 100644
--- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ParDoTranslation.java
+++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ParDoTranslation.java
@@ -422,14 +422,26 @@ public static RunnerApi.StateSpec translateStateSpec(
return stateSpec.match(
new StateSpec.Cases() {
@Override
- public RunnerApi.StateSpec dispatchValue(Coder> valueCoder) {
+ public RunnerApi.StateSpec dispatchReadModifyWrite(Coder> valueCoder) {
return builder
- .setValueSpec(
- RunnerApi.ValueStateSpec.newBuilder()
+ .setReadModifyWriteSpec(
+ RunnerApi.ReadModifyWriteStateSpec.newBuilder()
.setCoderId(registerCoderOrThrow(components, valueCoder)))
.build();
}
+ @Override
+ public RunnerApi.StateSpec dispatchValue(Coder> valueCoder) {
+ /* We are keeping this method for backward compatibility but we are using
+ ReadModifyWriteState under the hood.
+ */
+ return builder
+ .setReadModifyWriteSpec(
+ RunnerApi.ReadModifyWriteStateSpec.newBuilder()
+ .setCoderId(registerCoderOrThrow(components, valueCoder)))
+ .build();
+ }
+
@Override
public RunnerApi.StateSpec dispatchBag(Coder> elementCoder) {
return builder
@@ -475,8 +487,8 @@ public RunnerApi.StateSpec dispatchSet(Coder> elementCoder) {
static StateSpec> fromProto(RunnerApi.StateSpec stateSpec, RehydratedComponents components)
throws IOException {
switch (stateSpec.getSpecCase()) {
- case VALUE_SPEC:
- return StateSpecs.value(components.getCoder(stateSpec.getValueSpec().getCoderId()));
+ case READ_MODIFY_WRITE_SPEC:
+ return StateSpecs.readModifyWrite(components.getCoder(stateSpec.getReadModifyWriteSpec().getCoderId()));
case BAG_SPEC:
return StateSpecs.bag(components.getCoder(stateSpec.getBagSpec().getElementCoderId()));
case COMBINING_SPEC:
diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ParDoTranslationTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ParDoTranslationTest.java
index 6ea3bde27b2ca..5614b94c1d282 100644
--- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ParDoTranslationTest.java
+++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ParDoTranslationTest.java
@@ -210,7 +210,7 @@ public static class TestStateAndTimerTranslation {
@Parameters(name = "{index}: {0}")
public static Iterable> stateSpecs() {
return ImmutableList.of(
- StateSpecs.value(VarIntCoder.of()),
+ StateSpecs.readModifyWrite(VarIntCoder.of()),
StateSpecs.bag(VarIntCoder.of()),
StateSpecs.set(VarIntCoder.of()),
StateSpecs.map(StringUtf8Coder.of(), VarIntCoder.of()));
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/InMemoryStateInternals.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/InMemoryStateInternals.java
index 9628cff4b63dc..a0c499695bda4 100644
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/InMemoryStateInternals.java
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/InMemoryStateInternals.java
@@ -41,6 +41,7 @@
import org.apache.beam.sdk.state.State;
import org.apache.beam.sdk.state.StateContext;
import org.apache.beam.sdk.state.ValueState;
+import org.apache.beam.sdk.state.ReadModifyWriteState;
import org.apache.beam.sdk.state.WatermarkHoldState;
import org.apache.beam.sdk.transforms.Combine.CombineFn;
import org.apache.beam.sdk.transforms.CombineWithContext.CombineFnWithContext;
@@ -124,6 +125,11 @@ public ValueState bindValue(StateTag> address, Coder cod
return new InMemoryValue<>(coder);
}
+ @Override
+ public ReadModifyWriteState bindReadModifyWrite(StateTag> address, Coder coder) {
+ return new InMemoryReadModifyWrite<>(coder);
+ }
+
@Override
public BagState bindBag(final StateTag> address, Coder elemCoder) {
return new InMemoryBag<>(elemCoder);
@@ -166,9 +172,62 @@ CombiningState bindCombiningValueWithContext(
}
}
- /** An {@link InMemoryState} implementation of {@link ValueState}. */
+
+ /** An {@link InMemoryState} implementation of {@link ReadModifyWriteState}. */
+ public static final class InMemoryReadModifyWrite
+ implements ReadModifyWriteState, InMemoryState> {
+ private final Coder coder;
+
+ private boolean isCleared = true;
+ private @Nullable T value = null;
+
+ public InMemoryReadModifyWrite(Coder coder) {
+ this.coder = coder;
+ }
+
+ @Override
+ public void clear() {
+ // Even though we're clearing we can't remove this from the in-memory state map, since
+ // other users may already have a handle on this Value.
+ value = null;
+ isCleared = true;
+ }
+
+ @Override
+ public InMemoryReadModifyWrite readLater() {
+ return this;
+ }
+
+ @Override
+ public T read() {
+ return value;
+ }
+
+ @Override
+ public void write(T input) {
+ isCleared = false;
+ this.value = input;
+ }
+
+ @Override
+ public InMemoryReadModifyWrite copy() {
+ InMemoryReadModifyWrite that = new InMemoryReadModifyWrite<>(coder);
+ if (!this.isCleared) {
+ that.isCleared = this.isCleared;
+ that.value = uncheckedClone(coder, this.value);
+ }
+ return that;
+ }
+
+ @Override
+ public boolean isCleared() {
+ return isCleared;
+ }
+ }
+
+ /** An {@link InMemoryState} implementation of {@link ReadModifyWriteState}. */
public static final class InMemoryValue
- implements ValueState, InMemoryState> {
+ implements ValueState, InMemoryState> {
private final Coder coder;
private boolean isCleared = true;
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/MergingActiveWindowSet.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/MergingActiveWindowSet.java
index 9d2d8478b8a81..e338f6f7408cd 100644
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/MergingActiveWindowSet.java
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/MergingActiveWindowSet.java
@@ -32,7 +32,7 @@
import javax.annotation.Nullable;
import org.apache.beam.sdk.coders.MapCoder;
import org.apache.beam.sdk.coders.SetCoder;
-import org.apache.beam.sdk.state.ValueState;
+import org.apache.beam.sdk.state.ReadModifyWriteState;
import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
import org.apache.beam.sdk.transforms.windowing.WindowFn;
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
@@ -60,19 +60,19 @@ public class MergingActiveWindowSet implements ActiveWi
private final Map> originalActiveWindowToStateAddressWindows;
/** Handle representing our state in the backend. */
- private final ValueState