Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Allow timeout for async DoFn #5201

Closed
wants to merge 4 commits into from
Closed
Show file tree
Hide file tree
Changes from 3 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -26,11 +26,7 @@
import java.util.List;
import java.util.Queue;
import java.util.UUID;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentLinkedQueue;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.Semaphore;
import java.util.concurrent.*;
import java.util.function.Consumer;
import java.util.stream.Collectors;
import org.apache.beam.sdk.transforms.DoFn;
Expand Down Expand Up @@ -208,7 +204,7 @@ public void finishBundle(FinishBundleContext context) {
Thread.currentThread().interrupt();
LOG.error("Failed to process futures", e);
throw new RuntimeException("Failed to process futures", e);
} catch (ExecutionException e) {
} catch (ExecutionException | TimeoutException e) {
LOG.error("Failed to process futures", e);
throw new RuntimeException("Failed to process futures", e);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,10 +18,7 @@
package com.spotify.scio.transforms;

import java.util.UUID;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentLinkedQueue;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.*;
import org.apache.beam.sdk.transforms.DoFn;
import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
import org.joda.time.Instant;
Expand Down Expand Up @@ -57,7 +54,7 @@ public void finishBundle(FinishBundleContext context) {
Thread.currentThread().interrupt();
LOG.error("Failed to process futures", e);
throw new RuntimeException("Failed to process futures", e);
} catch (ExecutionException e) {
} catch (ExecutionException | TimeoutException e) {
LOG.error("Failed to process futures", e);
throw new RuntimeException("Failed to process futures", e);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,11 +22,7 @@
import java.io.Serializable;
import java.util.Objects;
import java.util.UUID;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentLinkedQueue;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.Semaphore;
import java.util.concurrent.*;
import java.util.function.Consumer;
import java.util.function.Supplier;
import javax.annotation.CheckForNull;
Expand Down Expand Up @@ -201,7 +197,7 @@ public void finishBundle(FinishBundleContext context) {
Thread.currentThread().interrupt();
LOG.error("Failed to process futures", e);
throw new RuntimeException("Failed to process futures", e);
} catch (ExecutionException e) {
} catch (ExecutionException | TimeoutException e) {
LOG.error("Failed to process futures", e);
throw new RuntimeException("Failed to process futures", e);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,8 +18,11 @@
package com.spotify.scio.transforms;

import com.google.common.util.concurrent.*;
import java.time.Duration;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import java.util.function.Function;
import java.util.stream.StreamSupport;
import javax.annotation.Nullable;
Expand All @@ -34,19 +37,32 @@ public class FutureHandlers {
* @param <V> value type.
*/
public interface Base<F, V> {
void waitForFutures(Iterable<F> futures) throws InterruptedException, ExecutionException;

default Duration getTimeout() {
return null;
}

void waitForFutures(Iterable<F> futures)
throws InterruptedException, ExecutionException, TimeoutException;

F addCallback(F future, Function<V, Void> onSuccess, Function<Throwable, Void> onFailure);
}

/** A {@link Base} implementation for Guava {@link ListenableFuture}. */
public interface Guava<V> extends Base<ListenableFuture<V>, V> {

@Override
default void waitForFutures(Iterable<ListenableFuture<V>> futures)
throws InterruptedException, ExecutionException {
throws InterruptedException, ExecutionException, TimeoutException {
// use Future#successfulAsList instead of Futures#allAsList which only works if all
// futures succeed
Futures.successfulAsList(futures).get();
ListenableFuture<?> f = Futures.successfulAsList(futures);
Duration timeout = getTimeout();
if (timeout != null) {
f.get(timeout.toMillis(), TimeUnit.MILLISECONDS);
} else {
f.get();
}
}

@Override
Expand Down Expand Up @@ -97,10 +113,16 @@ public void onFailure(Throwable t) {
public interface Java<V> extends Base<CompletableFuture<V>, V> {
@Override
default void waitForFutures(Iterable<CompletableFuture<V>> futures)
throws InterruptedException, ExecutionException {
throws InterruptedException, ExecutionException, TimeoutException {
CompletableFuture[] array =
StreamSupport.stream(futures.spliterator(), false).toArray(CompletableFuture[]::new);
CompletableFuture.allOf(array).exceptionally(t -> null).get();
CompletableFuture<?> f = CompletableFuture.allOf(array).exceptionally(t -> null);
Duration timeout = getTimeout();
if (timeout != null) {
f.get(timeout.toMillis(), TimeUnit.MILLISECONDS);
} else {
f.get();
}
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@
import com.spotify.scio.transforms.BaseAsyncLookupDoFn;
import com.spotify.scio.transforms.GuavaAsyncLookupDoFn;
import java.io.IOException;
import java.time.Duration;
import org.apache.beam.sdk.transforms.DoFn;

/**
Expand Down Expand Up @@ -99,6 +100,11 @@ public ResourceType getResourceType() {
return ResourceType.PER_INSTANCE;
}

@Override
public Duration getTimeout() {
return Duration.ofMillis(options.getCallOptionsConfig().getMutateRpcTimeoutMs());
}

protected BigtableSession newClient() {
try {
return new BigtableSession(options);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,8 @@

package com.spotify.scio.bigtable

import com.google.cloud.bigtable.config.BigtableOptions

import java.util.concurrent.ConcurrentLinkedQueue
import com.google.cloud.bigtable.grpc.BigtableSession
import com.google.common.cache.{Cache, CacheBuilder}
Expand Down Expand Up @@ -66,21 +68,26 @@ object BigtableDoFnTest {
val queue: ConcurrentLinkedQueue[Int] = new ConcurrentLinkedQueue[Int]()
}

class TestBigtableDoFn extends BigtableDoFn[Int, String](null) {
class TestBigtableDoFn extends BigtableDoFn[Int, String](BigtableOptions.getDefaultOptions) {
override def newClient(): BigtableSession = null
override def asyncLookup(session: BigtableSession, input: Int): ListenableFuture[String] =
Futures.immediateFuture(input.toString)
}

class TestCachingBigtableDoFn extends BigtableDoFn[Int, String](null, 100, new TestCacheSupplier) {
class TestCachingBigtableDoFn
extends BigtableDoFn[Int, String](
BigtableOptions.getDefaultOptions,
100,
new TestCacheSupplier
) {
override def newClient(): BigtableSession = null
override def asyncLookup(session: BigtableSession, input: Int): ListenableFuture[String] = {
BigtableDoFnTest.queue.add(input)
Futures.immediateFuture(input.toString)
}
}

class TestFailingBigtableDoFn extends BigtableDoFn[Int, String](null) {
class TestFailingBigtableDoFn extends BigtableDoFn[Int, String](BigtableOptions.getDefaultOptions) {
override def newClient(): BigtableSession = null
override def asyncLookup(session: BigtableSession, input: Int): ListenableFuture[String] =
if (input % 2 == 0) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,15 +20,20 @@ import com.google.common.util.concurrent.{ListenableFuture, SettableFuture}
import org.scalatest.flatspec.AnyFlatSpec
import org.scalatest.matchers.should.Matchers

import java.time.{Duration => JDuration}
import java.util.concurrent.CompletableFuture
import scala.concurrent.ExecutionContext.Implicits.global
import scala.concurrent.duration._
import scala.concurrent._
import scala.jdk.CollectionConverters._
import scala.util.{Failure, Success, Try}

class GuavaFutureHandler extends FutureHandlers.Guava[String]
class JavaFutureHandler extends FutureHandlers.Java[String]
class GuavaFutureHandler extends FutureHandlers.Guava[String] {
override def getTimeout: JDuration = JDuration.ofMillis(500)
}
class JavaFutureHandler extends FutureHandlers.Java[String] {
override def getTimeout: JDuration = JDuration.ofMillis(500)
}

class FutureHandlersTest extends AnyFlatSpec with Matchers {

Expand All @@ -37,6 +42,7 @@ class FutureHandlersTest extends AnyFlatSpec with Matchers {
create: () => I,
complete: I => String => Unit,
fail: I => Throwable => Unit,
cancel: I => Unit,
access: F => String
): Unit = {
it should "block until all futures complete" in {
Expand Down Expand Up @@ -106,6 +112,25 @@ class FutureHandlersTest extends AnyFlatSpec with Matchers {
an[Exception] shouldBe thrownBy(access(chainedFuture))
}

it should "should execute onFailure if cancelled" in {
val f = create()
var result: Try[String] = null
val chainedFuture = handler.addCallback(
f,
{ value =>
result = Success(value)
null
},
{ e =>
result = Failure(e)
null
}
)
cancel(f)
result shouldBe a[Failure[_]]
an[Exception] shouldBe thrownBy(access(chainedFuture))
}

it should "should execute onSuccess and propagate callback exception" in {
val f = create()
var result: Try[String] = null
Expand Down Expand Up @@ -154,6 +179,26 @@ class FutureHandlersTest extends AnyFlatSpec with Matchers {
}
cause.getSuppressed.headOption.map(_.getMessage) shouldBe expectedSuppressed
}

it should "wait for futures to complete" in {
import scala.concurrent.ExecutionContext.Implicits.global
val successFuture = create()
val failureFuture = create()
val cancelFuture = create()
Future {
Thread.sleep(100)
complete(successFuture)("success")
fail(failureFuture)(new Exception("failure"))
cancel(cancelFuture)
}
handler.waitForFutures(Iterable[F](successFuture, failureFuture, cancelFuture).asJava)
}

it should "throw a timeout exception " in {
val f = create()
a[TimeoutException] shouldBe thrownBy(handler.waitForFutures(Iterable[F](f).asJava))
}

}

"Guava handler" should behave like futureHandler[
Expand All @@ -164,6 +209,7 @@ class FutureHandlersTest extends AnyFlatSpec with Matchers {
SettableFuture.create[String],
_.set,
_.setException,
_.cancel(true),
_.get()
)

Expand All @@ -175,6 +221,7 @@ class FutureHandlersTest extends AnyFlatSpec with Matchers {
() => new CompletableFuture[String](),
_.complete,
_.completeExceptionally,
_.cancel(true),
_.get()
)
}