[ 
https://issues.apache.org/jira/browse/BEAM-10028?focusedWorklogId=435622&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-435622
 ]

ASF GitHub Bot logged work on BEAM-10028:
-----------------------------------------

                Author: ASF GitHub Bot
            Created on: 20/May/20 19:08
            Start Date: 20/May/20 19:08
    Worklog Time Spent: 10m 
      Work Description: lukecwik commented on a change in pull request #11746:
URL: https://github.com/apache/beam/pull/11746#discussion_r428246181



##########
File path: 
runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ModelCoders.java
##########
@@ -59,6 +60,14 @@ private ModelCoders() {}
 
   public static final String ROW_CODER_URN = getUrn(StandardCoders.Enum.ROW);
 
+  public static final String STATE_BACKED_ITERABLE_CODER_URN =
+      "beam:coder:state_backed_iterable:v1";

Review comment:
       We use a string constant here so that it can be used in switch/case 
statements. We make sure that the string constant matches the enum in the proto 
using the static block below.

##########
File path: 
sdks/java/harness/src/test/java/org/apache/beam/fn/harness/state/StateBackedIterableTest.java
##########
@@ -0,0 +1,184 @@
+/*
+ * 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.fn.harness.state;
+
+import static org.junit.Assert.assertEquals;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Random;
+import org.apache.beam.model.fnexecution.v1.BeamFnApi.StateKey;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString;
+import 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.FluentIterable;
+import 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
+import 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Lists;
+import org.junit.Test;
+import org.junit.experimental.runners.Enclosed;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+import org.junit.runners.Parameterized;
+
+/** Tests for {@link StateBackedIterable}. */
+@RunWith(Enclosed.class)
+public class StateBackedIterableTest {
+
+  @RunWith(Parameterized.class)
+  public static class IterationTest {
+    @Parameterized.Parameters
+    public static Iterable<Object[]> data() {
+      return ImmutableList.<Object[]>builder()
+          .add(new Object[] {Collections.emptyList(), "emptySuffix", 
ImmutableList.of()})
+          .add(new Object[] {ImmutableList.of("A", "B"), "emptySuffix", 
ImmutableList.of("A", "B")})
+          .add(
+              new Object[] {
+                Collections.emptyList(),
+                "nonEmptySuffix",
+                ImmutableList.of("C", "D", "E", "F", "G", "H", "I", "J", "K")
+              })
+          .add(
+              new Object[] {
+                ImmutableList.of("A", "B"),
+                "nonEmptySuffix",
+                ImmutableList.of("A", "B", "C", "D", "E", "F", "G", "H", "I", 
"J", "K")
+              })
+          .build();
+    }
+
+    @Parameterized.Parameter(0)
+    public List<String> prefix;
+
+    @Parameterized.Parameter(1)
+    public String suffixKey;
+
+    @Parameterized.Parameter(2)
+    public List<String> expected;
+
+    @Test
+    public void testIteration() throws Exception {
+      FakeBeamFnStateClient fakeBeamFnStateClient =
+          new FakeBeamFnStateClient(
+              ImmutableMap.of(
+                  key("nonEmptySuffix"), encode("C", "D", "E", "F", "G", "H", 
"I", "J", "K"),
+                  key("emptySuffix"), encode()));
+
+      StateBackedIterable<String> iterable =
+          new StateBackedIterable<>(
+              fakeBeamFnStateClient,
+              "instruction",
+              encode(suffixKey),
+              StringUtf8Coder.of(),
+              prefix);
+
+      assertEquals(expected, Lists.newArrayList(iterable));
+      assertEquals(expected, Lists.newArrayList(iterable));
+      assertEquals(expected, Lists.newArrayList(iterable));
+    }
+
+    @Test
+    public void testUsingMultipleInterleavedIterators() throws Exception {
+      FakeBeamFnStateClient fakeBeamFnStateClient =
+          new FakeBeamFnStateClient(
+              ImmutableMap.of(
+                  key("nonEmptySuffix"), encode("C", "D", "E", "F", "G", "H", 
"I", "J", "K"),
+                  key("emptySuffix"), encode()));
+
+      StateBackedIterable<String> iterable =
+          new StateBackedIterable<>(
+              fakeBeamFnStateClient,
+              "instruction",
+              encode(suffixKey),
+              StringUtf8Coder.of(),
+              prefix);
+
+      List<Iterator<String>> iterators = new ArrayList<>();
+      List<List<String>> results = new ArrayList<>();
+      for (int i = 0; i < 3; ++i) {
+        iterators.add(iterable.iterator());
+        results.add(new ArrayList<>());
+      }
+
+      Random random = new Random(42L);
+      while (!iterators.isEmpty()) {

Review comment:
       This is covering reiteration since the same iterable is producing 
multiple iterators that are read in random order.
   
   `testIteration` also covers re-iteration, renamed test for clarity

##########
File path: 
sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/StateBackedIterable.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.fn.harness.state;
+
+import static 
org.apache.beam.runners.core.construction.ModelCoders.STATE_BACKED_ITERABLE_CODER_URN;
+
+import com.google.auto.service.AutoService;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.function.Supplier;
+import org.apache.beam.model.fnexecution.v1.BeamFnApi.StateRequest;
+import 
org.apache.beam.runners.core.construction.CoderTranslation.TranslationContext;
+import org.apache.beam.runners.core.construction.CoderTranslator;
+import org.apache.beam.runners.core.construction.CoderTranslatorRegistrar;
+import org.apache.beam.sdk.coders.IterableLikeCoder;
+import org.apache.beam.sdk.fn.stream.DataStreams;
+import org.apache.beam.sdk.util.BufferedElementCountingOutputStream;
+import org.apache.beam.sdk.util.VarInt;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString;
+import 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
+import 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
+import 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables;
+import 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterators;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.io.ByteStreams;
+
+/**
+ * A {@link BeamFnStateClient state} backed iterable which allows for fetching 
elements over the
+ * portability state API. See <a
+ * href="https://s.apache.org/beam-fn-state-api-and-bundle-processing";>remote 
references</a> for
+ * additional details.
+ *
+ * <p>One must supply a {@link StateBackedIterableTranslationContext} when 
using {@link
+ * CoderTranslator#fromComponents} to be able to create a {@link 
StateBackedIterable.Coder}.
+ */
+public class StateBackedIterable<T> implements Iterable<T> {
+
+  private final BeamFnStateClient beamFnStateClient;
+  private final org.apache.beam.sdk.coders.Coder<T> elemCoder;
+  @VisibleForTesting final StateRequest request;
+  @VisibleForTesting final List<T> prefix;
+
+  public StateBackedIterable(
+      BeamFnStateClient beamFnStateClient,
+      String instructionId,
+      ByteString runnerKey,
+      org.apache.beam.sdk.coders.Coder<T> elemCoder,
+      List<T> prefix) {
+    this.beamFnStateClient = beamFnStateClient;
+    this.elemCoder = elemCoder;
+
+    StateRequest.Builder requestBuilder = StateRequest.newBuilder();
+    requestBuilder
+        .setInstructionId(instructionId)
+        .getStateKeyBuilder()
+        .getRunnerBuilder()
+        .setKey(runnerKey);
+    this.request = requestBuilder.build();
+    this.prefix = prefix;
+  }
+
+  @Override
+  public Iterator<T> iterator() {
+    return Iterators.concat(
+        prefix.iterator(),
+        new DataStreams.DataStreamDecoder(
+            elemCoder,
+            
DataStreams.inbound(StateFetchingIterators.forFirstChunk(beamFnStateClient, 
request))));
+  }
+
+  /**
+   * Decodes an {@link Iterable} that might be backed by state. If the 
terminator at the end of the
+   * value stream is {@code -1} then we return a {@link StateBackedIterable} 
otherwise we return an
+   * {@link Iterable}.
+   */
+  public static class Coder<T> extends IterableLikeCoder<T, Iterable<T>> {
+
+    private final BeamFnStateClient beamFnStateClient;
+    private final Supplier<String> instructionId;
+
+    public Coder(
+        BeamFnStateClient beamFnStateClient,
+        Supplier<String> instructionId,
+        org.apache.beam.sdk.coders.Coder<T> elemCoder) {
+      super(elemCoder, "StateBackedIterable");
+      this.beamFnStateClient = beamFnStateClient;
+      this.instructionId = instructionId;
+    }
+
+    @Override
+    protected Iterable<T> decodeToIterable(List<T> decodedElements) {
+      return decodedElements;
+    }
+
+    @Override
+    protected Iterable<T> decodeToIterable(
+        List<T> decodedElements, long terminatorValue, InputStream in) throws 
IOException {
+      if (terminatorValue == -1L) {
+        long tokenLength = VarInt.decodeLong(in);
+        ByteString token = ByteString.readFrom(ByteStreams.limit(in, 
tokenLength));
+        return new StateBackedIterable<>(
+            beamFnStateClient, instructionId.get(), token, getElemCoder(), 
decodedElements);
+      } else {
+        throw new IllegalStateException(
+            String.format(
+                "StateBackedIterable expected terminator of 0 or -1 but 
received %s.",
+                terminatorValue));
+      }
+    }
+
+    @Override
+    public void encode(Iterable<T> iterable, OutputStream outStream) throws 
IOException {

Review comment:
       See 
https://github.com/apache/beam/blob/master/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/BeamFnStateClient.java
 and 
https://github.com/apache/beam/blob/master/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/BeamFnStateGrpcClientCache.java

##########
File path: 
sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/StateBackedIterable.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.fn.harness.state;
+
+import static 
org.apache.beam.runners.core.construction.ModelCoders.STATE_BACKED_ITERABLE_CODER_URN;
+
+import com.google.auto.service.AutoService;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.function.Supplier;
+import org.apache.beam.model.fnexecution.v1.BeamFnApi.StateRequest;
+import 
org.apache.beam.runners.core.construction.CoderTranslation.TranslationContext;
+import org.apache.beam.runners.core.construction.CoderTranslator;
+import org.apache.beam.runners.core.construction.CoderTranslatorRegistrar;
+import org.apache.beam.sdk.coders.IterableLikeCoder;
+import org.apache.beam.sdk.fn.stream.DataStreams;
+import org.apache.beam.sdk.util.BufferedElementCountingOutputStream;
+import org.apache.beam.sdk.util.VarInt;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString;
+import 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
+import 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
+import 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables;
+import 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterators;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.io.ByteStreams;
+
+/**
+ * A {@link BeamFnStateClient state} backed iterable which allows for fetching 
elements over the
+ * portability state API. See <a
+ * href="https://s.apache.org/beam-fn-state-api-and-bundle-processing";>remote 
references</a> for
+ * additional details.
+ *
+ * <p>One must supply a {@link StateBackedIterableTranslationContext} when 
using {@link
+ * CoderTranslator#fromComponents} to be able to create a {@link 
StateBackedIterable.Coder}.
+ */
+public class StateBackedIterable<T> implements Iterable<T> {
+
+  private final BeamFnStateClient beamFnStateClient;
+  private final org.apache.beam.sdk.coders.Coder<T> elemCoder;
+  @VisibleForTesting final StateRequest request;
+  @VisibleForTesting final List<T> prefix;
+
+  public StateBackedIterable(
+      BeamFnStateClient beamFnStateClient,
+      String instructionId,
+      ByteString runnerKey,
+      org.apache.beam.sdk.coders.Coder<T> elemCoder,
+      List<T> prefix) {
+    this.beamFnStateClient = beamFnStateClient;
+    this.elemCoder = elemCoder;
+
+    StateRequest.Builder requestBuilder = StateRequest.newBuilder();
+    requestBuilder
+        .setInstructionId(instructionId)
+        .getStateKeyBuilder()
+        .getRunnerBuilder()
+        .setKey(runnerKey);
+    this.request = requestBuilder.build();
+    this.prefix = prefix;
+  }
+
+  @Override
+  public Iterator<T> iterator() {
+    return Iterators.concat(
+        prefix.iterator(),
+        new DataStreams.DataStreamDecoder(

Review comment:
       Renamed to `readAllStartingFrom`




----------------------------------------------------------------
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:
us...@infra.apache.org


Issue Time Tracking
-------------------

    Worklog Id:     (was: 435622)
    Time Spent: 40m  (was: 0.5h)

> [Java SDK] Support state backed iterables within the SDK harness
> ----------------------------------------------------------------
>
>                 Key: BEAM-10028
>                 URL: https://issues.apache.org/jira/browse/BEAM-10028
>             Project: Beam
>          Issue Type: Improvement
>          Components: sdk-java-harness
>            Reporter: Luke Cwik
>            Assignee: Luke Cwik
>            Priority: P2
>          Time Spent: 40m
>  Remaining Estimate: 0h
>




--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to