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

[RRIO] [Call] Implement the Repeater #29490

Merged
merged 4 commits into from
Dec 5, 2023
Merged
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
4 changes: 2 additions & 2 deletions sdks/java/io/rrio/build.gradle
Original file line number Diff line number Diff line change
Expand Up @@ -37,6 +37,8 @@ dependencies {
implementation library.java.jackson_core
implementation library.java.jackson_databind
implementation "redis.clients:jedis:$jedisVersion"
implementation platform(library.java.google_cloud_platform_libraries_bom)
damondouglas marked this conversation as resolved.
Show resolved Hide resolved
implementation library.java.google_http_client

testImplementation project(path: ":sdks:java:core", configuration: "shadowTest")
testImplementation project(path: ":sdks:java:io:common", configuration: "testRuntimeMigration")
Expand All @@ -47,8 +49,6 @@ dependencies {
testImplementation "io.grpc:grpc-stub:${grpcVersion}"
testImplementation "com.google.protobuf:protobuf-java-util:${protobufVersion}"

testImplementation platform(library.java.google_cloud_platform_libraries_bom)
testImplementation library.java.google_http_client
testImplementation library.java.junit
testImplementation library.java.testcontainers_base

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,142 @@
/*
* 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.io.requestresponse;

import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull;

import com.google.api.client.util.BackOff;
import com.google.api.client.util.ExponentialBackOff;
import java.io.IOException;
import java.util.Set;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.Executors;
import java.util.concurrent.Future;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.TimeUnit;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet;
import org.checkerframework.checker.nullness.qual.MonotonicNonNull;

/** Repeats a method invocation when it encounters an error. */
public class Repeater<InputT, OutputT> {

/**
* {@link Set} of {@link UserCodeExecutionException}s that warrant repeating. A public modifier is
* applied to communicate to users of this class which {@link UserCodeExecutionException}s
* constitute warrant repeat execution.
*/
public static final Set<Class<? extends UserCodeExecutionException>> REPEATABLE_ERROR_TYPES =
ImmutableSet.of(
UserCodeRemoteSystemException.class,
UserCodeTimeoutException.class,
UserCodeQuotaException.class);

/** Instantiates a {@link Repeater}. */
public static <InputT, OutputT> Repeater<InputT, OutputT> of(
ThrowableFunction<InputT, OutputT> throwableFunction, Sleeper sleeper, Integer limit) {
damondouglas marked this conversation as resolved.
Show resolved Hide resolved
return new Repeater<>(throwableFunction, sleeper, limit);
}

private final ThrowableFunction<InputT, OutputT> throwableFunction;

private final Sleeper sleeper;
private final int limit;

private Repeater(
ThrowableFunction<InputT, OutputT> throwableFunction, Sleeper sleeper, int limit) {
this.throwableFunction = throwableFunction;
this.sleeper = sleeper;
this.limit = limit;
}

/**
* Applies the {@link InputT} to the {@link ThrowableFunction}. If the function throws an
* exception that {@link #REPEATABLE_ERROR_TYPES} contains, repeats the invocation up to the
* limit, otherwise throws the last exception.
*/
public OutputT apply(InputT input) throws UserCodeExecutionException, InterruptedException {
@MonotonicNonNull UserCodeExecutionException lastException = null;
for (int numAttempts = 0; numAttempts < limit; numAttempts++) {
try {
return throwableFunction.apply(input);
} catch (UserCodeExecutionException e) {
if (!REPEATABLE_ERROR_TYPES.contains(e.getClass())) {
throw e;
}
lastException = e;
sleeper.sleep();
}
}
throw checkStateNotNull(lastException);
}

/**
* A {@link FunctionalInterface} for executing a {@link UserCodeExecutionException} throwable
* function.
*/
@FunctionalInterface
public interface ThrowableFunction<InputT, OutputT> {
/** Returns the result of invoking this function on the given input. */
OutputT apply(InputT input) throws UserCodeExecutionException;
}

/** Interfaces implementation details for pausing an execution. */
public interface Sleeper {

/** Pauses the execution. */
void sleep() throws InterruptedException;
damondouglas marked this conversation as resolved.
Show resolved Hide resolved
}

/**
* A {@link Sleeper} implementation that uses a {@link BackOff} to determine how long to pause
* execution.
*/
public static class DefaultSleeper implements Sleeper {
damondouglas marked this conversation as resolved.
Show resolved Hide resolved

public static DefaultSleeper of() {
return of(new ExponentialBackOff());
}

public static DefaultSleeper of(BackOff backOff) {
return new DefaultSleeper(backOff);
}

private final BackOff backOff;

private DefaultSleeper(BackOff backOff) {
this.backOff = backOff;
}

@Override
public void sleep() throws InterruptedException {
ScheduledExecutorService executorService = Executors.newSingleThreadScheduledExecutor();
try {
Future<?> future =
executorService.schedule(() -> {}, backOff.nextBackOffMillis(), TimeUnit.MILLISECONDS);
future.get();
} catch (IOException | ExecutionException e) {
throw new RuntimeException(e);
}
executorService.shutdown();
boolean ignored = executorService.awaitTermination(1L, TimeUnit.SECONDS);
}

BackOff getBackOff() {
return backOff;
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,41 @@
/*
* 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.io.requestresponse;

/**
* A {@link UserCodeExecutionException} that signals an error with a remote system. Examples of such
* errors include an HTTP 5XX error or gRPC INTERNAL (13) error.
*/
public class UserCodeRemoteSystemException extends UserCodeExecutionException {
public UserCodeRemoteSystemException(String message) {
super(message);
}

public UserCodeRemoteSystemException(String message, Throwable cause) {
super(message, cause);
}

public UserCodeRemoteSystemException(Throwable cause) {
super(cause);
}

public UserCodeRemoteSystemException(
String message, Throwable cause, boolean enableSuppression, boolean writableStackTrace) {
super(message, cause, enableSuppression, writableStackTrace);
}
}
Loading
Loading