beam-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "ASF GitHub Bot (JIRA)" <j...@apache.org>
Subject [jira] [Work logged] (BEAM-4286) Pooled artifact source
Date Wed, 16 May 2018 21:31:00 GMT

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

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

                Author: ASF GitHub Bot
            Created on: 16/May/18 21:30
            Start Date: 16/May/18 21:30
    Worklog Time Spent: 10m 
      Work Description: bsidhom closed pull request #5359: [BEAM-4286] Implement pooled artifact
source
URL: https://github.com/apache/beam/pull/5359
 
 
   

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/runners/flink/src/main/java/org/apache/beam/runners/flink/ArtifactSourcePool.java
b/runners/flink/src/main/java/org/apache/beam/runners/flink/ArtifactSourcePool.java
index 3881a0b365a..0fe01e3dce2 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/ArtifactSourcePool.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/ArtifactSourcePool.java
@@ -17,8 +17,14 @@
  */
 package org.apache.beam.runners.flink;
 
+import static com.google.common.base.Preconditions.checkState;
+
+import com.google.common.collect.Maps;
 import io.grpc.stub.StreamObserver;
 import java.io.IOException;
+import java.util.Map;
+import java.util.concurrent.Phaser;
+import javax.annotation.concurrent.GuardedBy;
 import javax.annotation.concurrent.ThreadSafe;
 import org.apache.beam.model.jobmanagement.v1.ArtifactApi.ArtifactChunk;
 import org.apache.beam.model.jobmanagement.v1.ArtifactApi.Manifest;
@@ -39,23 +45,106 @@
 @ThreadSafe
 public class ArtifactSourcePool implements ArtifactSource {
 
+  public static ArtifactSourcePool create() {
+    return new ArtifactSourcePool();
+  }
+
+  private final Object lock = new Object();
+
+  @GuardedBy("lock")
+  private final Map<ArtifactSource, Phaser> artifactSources = Maps.newLinkedHashMap();
+
   private ArtifactSourcePool() {}
 
   /**
    * Adds a new cache to the pool. When the returned {@link AutoCloseable} is closed, the
given
-   * cache will be removed from the pool.
+   * cache will be removed from the pool. The call to {@link AutoCloseable#close()} will
block until
+   * the artifact source is no longer being used.
    */
   public AutoCloseable addToPool(ArtifactSource artifactSource) {
-    throw new UnsupportedOperationException();
+    synchronized (lock) {
+      checkState(!artifactSources.containsKey(artifactSource));
+      // For this new artifact source, insert a new Phaser with 1 registrant. This party
will only
+      // be deregistered when the corresponding artifact source is marked as closed. Doing
so marks
+      // the end of the phase and terminates the phaser.
+      artifactSources.put(
+          artifactSource,
+          new Phaser(1) {
+            @Override
+            protected boolean onAdvance(int phase, int registeredParties) {
+              // There should only ever be a single phase. Terminate once all registered
parties
+              // have
+              // arrived.
+              return true;
+            }
+          });
+      return () -> {
+        Phaser phaser;
+        synchronized (lock) {
+          phaser = artifactSources.remove(artifactSource);
+          if (phaser == null) {
+            // We've already removed the phaser from the map and attempted to close it.
+            return;
+          }
+          // This indicates we have not yet terminated the phaser. Ensure this is the case.
+          checkState(!phaser.isTerminated());
+        }
+        phaser.arriveAndAwaitAdvance();
+        phaser.arriveAndDeregister();
+      };
+    }
   }
 
   @Override
   public Manifest getManifest() throws IOException {
-    throw new UnsupportedOperationException();
+    SourceHandle handle = getAny();
+    try {
+      return handle.getSource().getManifest();
+    } finally {
+      handle.close();
+    }
   }
 
   @Override
   public void getArtifact(String name, StreamObserver<ArtifactChunk> responseObserver)
{
-    throw new UnsupportedOperationException();
+    SourceHandle handle = getAny();
+    try {
+      handle.getSource().getArtifact(name, responseObserver);
+    } finally {
+      handle.close();
+    }
+  }
+
+  private SourceHandle getAny() {
+    synchronized (lock) {
+      checkState(!artifactSources.isEmpty());
+      Map.Entry<ArtifactSource, Phaser> entry = artifactSources.entrySet().iterator().next();
+      return new SourceHandle(entry.getKey(), entry.getValue());
+    }
+  }
+
+  private static class SourceHandle {
+    private final ArtifactSource artifactSource;
+    private final Phaser phaser;
+
+    private boolean isClosed = false;
+
+    SourceHandle(ArtifactSource artifactSource, Phaser phaser) {
+      this.artifactSource = artifactSource;
+      this.phaser = phaser;
+      int registeredPhase = this.phaser.register();
+      checkState(registeredPhase >= 0, "Artifact source already closed");
+    }
+
+    ArtifactSource getSource() {
+      checkState(!phaser.isTerminated());
+      return artifactSource;
+    }
+
+    void close() {
+      if (!isClosed) {
+        phaser.arriveAndDeregister();
+      }
+    }
   }
 }
diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/ArtifactSourcePoolTest.java
b/runners/flink/src/test/java/org/apache/beam/runners/flink/ArtifactSourcePoolTest.java
new file mode 100644
index 00000000000..15657bda4ee
--- /dev/null
+++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/ArtifactSourcePoolTest.java
@@ -0,0 +1,158 @@
+/*
+ * 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.runners.flink;
+
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.instanceOf;
+import static org.hamcrest.Matchers.sameInstance;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import io.grpc.stub.StreamObserver;
+import java.io.IOException;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import org.apache.beam.model.jobmanagement.v1.ArtifactApi.ArtifactChunk;
+import org.apache.beam.model.jobmanagement.v1.ArtifactApi.Manifest;
+import org.apache.beam.runners.fnexecution.artifact.ArtifactSource;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/** Tests for {@link ArtifactSourcePool}. */
+@RunWith(JUnit4.class)
+public class ArtifactSourcePoolTest {
+  @Rule public ExpectedException thrown = ExpectedException.none();
+
+  @Test
+  public void mustRegisterSourcesFirst() throws Exception {
+    ArtifactSourcePool pool = ArtifactSourcePool.create();
+    thrown.expect(instanceOf(IllegalStateException.class));
+    pool.getManifest();
+  }
+
+  @Test
+  public void delegatesToPool() throws Exception {
+    ArtifactSourcePool pool = ArtifactSourcePool.create();
+    ArtifactSource source = mock(ArtifactSource.class);
+    Manifest manifest = Manifest.newBuilder().build();
+    when(source.getManifest()).thenReturn(manifest);
+    AutoCloseable handle = pool.addToPool(source);
+    assertThat(pool.getManifest(), sameInstance(manifest));
+    handle.close();
+  }
+
+  @Test
+  public void delegatesAfterClose() throws Exception {
+    ArtifactSourcePool pool = ArtifactSourcePool.create();
+    ArtifactSource fooSource = mock(ArtifactSource.class);
+    ArtifactSource barSource = mock(ArtifactSource.class);
+    AutoCloseable fooHandle = pool.addToPool(fooSource);
+    AutoCloseable barHandle = pool.addToPool(barSource);
+    fooHandle.close();
+    // barSource is the only remaining valid source. Ensure this is called.
+    pool.getManifest();
+    verify(barSource).getManifest();
+    barHandle.close();
+  }
+
+  @Test
+  public void cannotServeAfterClosing() throws Exception {
+    ArtifactSourcePool pool = ArtifactSourcePool.create();
+    ArtifactSource source = mock(ArtifactSource.class);
+    AutoCloseable handle = pool.addToPool(source);
+    handle.close();
+    thrown.expect(instanceOf(IllegalStateException.class));
+    pool.getManifest();
+  }
+
+  @Test
+  public void waitsUntilConsumersFinished() throws Exception {
+    ExecutorService executor = Executors.newCachedThreadPool();
+    ArtifactSourcePool pool = ArtifactSourcePool.create();
+    CompletableFuture<Void> manifestRequested = new CompletableFuture<>();
+    CompletableFuture<Void> closeProbablyInvoked = new CompletableFuture<>();
+    ArtifactSource source =
+        new ArtifactSource() {
+          @Override
+          public Manifest getManifest() throws IOException {
+            manifestRequested.complete(null);
+            // Block until close has (probably) been invoked. This means there is a live
handle at
+            // time that close is invoked and exercises the resource monitor. We cannot test
this
+            // deterministically because the call to close blocks until completion.
+            try {
+              closeProbablyInvoked.get();
+            } catch (Exception e) {
+              throw new RuntimeException(e);
+            }
+            return null;
+          }
+
+          @Override
+          public void getArtifact(String name, StreamObserver<ArtifactChunk> responseObserver)
{
+            throw new UnsupportedOperationException();
+          }
+        };
+    AutoCloseable handle = pool.addToPool(source);
+    CompletableFuture<Manifest> manifestFuture =
+        CompletableFuture.supplyAsync(
+            () -> {
+              try {
+                return pool.getManifest();
+              } catch (IOException e) {
+                throw new RuntimeException(e);
+              }
+            },
+            executor);
+
+    CompletableFuture<Void> closedFuture =
+        manifestRequested.handleAsync(
+            (v, th) -> {
+              try {
+                handle.close();
+              } catch (Exception e) {
+                throw new RuntimeException(e);
+              }
+              return null;
+            },
+            executor);
+
+    manifestRequested.handleAsync(
+        (v, th) -> {
+          try {
+            Thread.sleep(500);
+            closeProbablyInvoked.complete(null);
+          } catch (InterruptedException e) {
+            throw new RuntimeException(e);
+          }
+          return null;
+        },
+        executor);
+
+    manifestFuture.get();
+    closedFuture.get();
+
+    executor.shutdown();
+    executor.awaitTermination(10, TimeUnit.SECONDS);
+  }
+}


 

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


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

    Worklog Id:     (was: 102671)
    Time Spent: 1.5h  (was: 1h 20m)

> Pooled artifact source
> ----------------------
>
>                 Key: BEAM-4286
>                 URL: https://issues.apache.org/jira/browse/BEAM-4286
>             Project: Beam
>          Issue Type: Bug
>          Components: runner-flink
>            Reporter: Ben Sidhom
>            Assignee: Ben Sidhom
>            Priority: Minor
>          Time Spent: 1.5h
>  Remaining Estimate: 0h
>
> Because DistributeCache lifetimes are tied to operator lifetimes in Flink, we need a
way to wrap operator-scoped artifact sources. Artifacts are inherently job-scoped and should
be the same throughout a job's lifetime. For this reason, it is safe to pool artifact sources
and serve artifacts from an arbitrary pooled source as long as the underlying source is still
in scope.
> We need a pooled source in order to satisfy the bundle factory interfaces. Using the
job-scoped and stage-scoped bundle factories allows us to cache and reuse different components
that serve SDK harnesses. Because the distributed cache lifetimes are specific to Flink, the
pooled artifact source should probably live in a runner-specific directory.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Mime
View raw message