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 all 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,8 @@
package com.spotify.scio.transforms;

import com.google.common.util.concurrent.*;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ExecutionException;
import java.time.Duration;
import java.util.concurrent.*;
import java.util.function.Function;
import java.util.stream.StreamSupport;
import javax.annotation.Nullable;
Expand All @@ -34,19 +34,33 @@ 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> {
Executor getCallbackExecutor();

@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 All @@ -57,6 +71,15 @@ default ListenableFuture<V> addCallback(
// Futures#transform doesn't allow onFailure callback while Futures#addCallback doesn't
// guarantee that callbacks are called before ListenableFuture#get() unblocks
SettableFuture<V> f = SettableFuture.create();
// if executor rejects the callback, we have to fail the future
Executor rejectPropagationExecutor =
command -> {
try {
getCallbackExecutor().execute(command);
} catch (RejectedExecutionException e) {
f.setException(e);
}
};
Futures.addCallback(
future,
new FutureCallback<V>() {
Expand Down Expand Up @@ -87,7 +110,7 @@ public void onFailure(Throwable t) {
}
}
},
MoreExecutors.directExecutor());
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

MoreExecutors.directExecutor() doc says callback may not be executed if the exception terminates the executing thread

In other cases, no code will catch the exception, and it may terminate whichever thread happens to trigger the execution.

I've changed to use another executor that also completes the future when if it refuses to execute the callback. This is the strategy used by guava when transforming futures.

rejectPropagationExecutor);

return f;
}
Expand All @@ -97,10 +120,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 @@ -20,6 +20,8 @@
import com.spotify.scio.transforms.BaseAsyncLookupDoFn.CacheSupplier;
import com.spotify.scio.transforms.BaseAsyncLookupDoFn.Try;
import java.util.List;
import java.util.concurrent.Executor;
import java.util.concurrent.Executors;
import org.apache.beam.sdk.transforms.SerializableFunction;
import org.apache.commons.lang3.tuple.Pair;

Expand All @@ -42,6 +44,8 @@ public abstract class GuavaAsyncBatchLookupDoFn<
Try<Output>>
implements FutureHandlers.Guava<BatchResponse> {

private transient Executor executor;

public GuavaAsyncBatchLookupDoFn(
int batchSize,
SerializableFunction<List<Input>, BatchRequest> batchRequestFn,
Expand All @@ -67,6 +71,14 @@ public GuavaAsyncBatchLookupDoFn(
cacheSupplier);
}

@Override
public Executor getCallbackExecutor() {
if (executor == null) {
executor = Executors.newSingleThreadExecutor();
}
return executor;
}

@Override
public Try<Output> success(Output output) {
return new Try<>(output);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,8 @@
package com.spotify.scio.transforms;

import com.google.common.util.concurrent.ListenableFuture;
import java.util.concurrent.Executor;
import java.util.concurrent.Executors;
import org.apache.beam.sdk.transforms.DoFn;

/**
Expand All @@ -26,4 +28,14 @@
*/
public abstract class GuavaAsyncDoFn<InputT, OutputT, ResourceT>
extends BaseAsyncDoFn<InputT, OutputT, ResourceT, ListenableFuture<OutputT>>
implements FutureHandlers.Guava<OutputT> {}
implements FutureHandlers.Guava<OutputT> {
private transient Executor executor;

@Override
public Executor getCallbackExecutor() {
if (executor == null) {
executor = Executors.newSingleThreadExecutor();
}
return executor;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,8 @@
package com.spotify.scio.transforms;

import com.google.common.util.concurrent.ListenableFuture;
import java.util.concurrent.Executor;
import java.util.concurrent.Executors;
import org.apache.beam.sdk.transforms.DoFn;

/**
Expand All @@ -32,6 +34,8 @@ public abstract class GuavaAsyncLookupDoFn<A, B, C>
extends BaseAsyncLookupDoFn<A, B, C, ListenableFuture<B>, BaseAsyncLookupDoFn.Try<B>>
implements FutureHandlers.Guava<B> {

private transient Executor executor;

/** Create a {@link GuavaAsyncLookupDoFn} instance. */
public GuavaAsyncLookupDoFn() {
super();
Expand Down Expand Up @@ -75,6 +79,14 @@ public GuavaAsyncLookupDoFn(
super(maxPendingRequests, deduplicate, cacheSupplier);
}

@Override
public Executor getCallbackExecutor() {
if (executor == null) {
executor = Executors.newSingleThreadExecutor();
}
return executor;
}

@Override
public BaseAsyncLookupDoFn.Try<B> success(B output) {
return new Try<>(output);
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
Loading
Loading