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

Change the Java sdk harness cache timeout for bundle processors to be an hour for streaming pipelines instead of 1 minute. #33175

Open
wants to merge 1 commit into
base: master
Choose a base branch
from
Open
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 @@ -20,6 +20,7 @@
import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkNotNull;

import com.fasterxml.jackson.annotation.JsonCreator;
import java.time.Duration;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashMap;
Expand Down Expand Up @@ -386,4 +387,20 @@ static List<Logger> getConfiguredLoggerFromOptions(SdkHarnessOptions loggingOpti
}
return configuredLoggers;
}

@Description(
Copy link
Member

Choose a reason for hiding this comment

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

I suggest @Hidden so that this is not shown to users when they do --help

"Timeout used for cache of bundle processors. Defaults to a minute for batch and an hour for streaming.")
@Default.InstanceFactory(BundleProcessorCacheTimeoutFactory.class)
Duration getBundleProcessorCacheTimeout();

void setBundleProcessorCacheTimeout(Duration duration);

class BundleProcessorCacheTimeoutFactory implements DefaultValueFactory<Duration> {
@Override
public Duration create(PipelineOptions options) {
return options.as(StreamingOptions.class).isStreaming()
? Duration.ofHours(1)
: Duration.ofMinutes(1);
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -83,6 +83,7 @@
import org.apache.beam.sdk.metrics.MetricsEnvironment;
import org.apache.beam.sdk.metrics.MetricsEnvironment.MetricsEnvironmentState;
import org.apache.beam.sdk.options.PipelineOptions;
import org.apache.beam.sdk.options.SdkHarnessOptions;
import org.apache.beam.sdk.transforms.DoFn.BundleFinalizer;
import org.apache.beam.sdk.util.WindowedValue;
import org.apache.beam.sdk.util.common.ReflectHelpers;
Expand Down Expand Up @@ -188,7 +189,8 @@ public ProcessBundleHandler(
executionStateSampler,
REGISTERED_RUNNER_FACTORIES,
processWideCache,
new BundleProcessorCache(),
new BundleProcessorCache(
options.as(SdkHarnessOptions.class).getBundleProcessorCacheTimeout()),
dataSampler);
}

Expand Down Expand Up @@ -927,25 +929,25 @@ public int hashCode() {
return super.hashCode();
}

BundleProcessorCache() {
this.cachedBundleProcessors =
BundleProcessorCache(Duration timeout) {
CacheBuilder<String, ConcurrentLinkedQueue<ProcessBundleHandler.BundleProcessor>> builder =
CacheBuilder.newBuilder()
.expireAfterAccess(Duration.ofMinutes(1L))
.removalListener(
removalNotification -> {
((ConcurrentLinkedQueue<BundleProcessor>) removalNotification.getValue())
.forEach(
bundleProcessor -> {
bundleProcessor.shutdown();
});
})
.build(
new CacheLoader<String, ConcurrentLinkedQueue<BundleProcessor>>() {
@Override
public ConcurrentLinkedQueue<BundleProcessor> load(String s) throws Exception {
return new ConcurrentLinkedQueue<>();
}
});
removalNotification ->
removalNotification
.getValue()
.forEach(bundleProcessor -> bundleProcessor.shutdown()));
if (timeout.compareTo(Duration.ZERO) > 0) {
builder = builder.expireAfterAccess(timeout);
}
this.cachedBundleProcessors =
builder.build(
new CacheLoader<String, ConcurrentLinkedQueue<BundleProcessor>>() {
@Override
public ConcurrentLinkedQueue<BundleProcessor> load(String s) throws Exception {
return new ConcurrentLinkedQueue<>();
}
});
// We specifically use a weak hash map so that references will automatically go out of scope
// and not need to be freed explicitly from the cache.
this.activeBundleProcessors = Collections.synchronizedMap(new WeakHashMap<>());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -48,6 +48,7 @@
import static org.mockito.Mockito.when;

import java.io.IOException;
import java.time.Duration;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
Expand Down Expand Up @@ -354,6 +355,10 @@ void reset() throws Exception {

private static class TestBundleProcessorCache extends BundleProcessorCache {

TestBundleProcessorCache() {
super(Duration.ZERO);
}

@Override
BundleProcessor get(
InstructionRequest processBundleRequest,
Expand All @@ -376,7 +381,7 @@ public void testTrySplitBeforeBundleDoesNotFail() {
executionStateSampler,
ImmutableMap.of(),
Caches.noop(),
new BundleProcessorCache(),
new BundleProcessorCache(Duration.ZERO),
null /* dataSampler */);

BeamFnApi.InstructionResponse response =
Expand Down Expand Up @@ -407,7 +412,7 @@ public void testProgressBeforeBundleDoesNotFail() throws Exception {
executionStateSampler,
ImmutableMap.of(),
Caches.noop(),
new BundleProcessorCache(),
new BundleProcessorCache(Duration.ZERO),
null /* dataSampler */);

handler.progress(
Expand Down Expand Up @@ -487,7 +492,7 @@ public void testOrderOfStartAndFinishCalls() throws Exception {
DATA_INPUT_URN, startFinishRecorder,
DATA_OUTPUT_URN, startFinishRecorder),
Caches.noop(),
new BundleProcessorCache(),
new BundleProcessorCache(Duration.ZERO),
null /* dataSampler */);

handler.processBundle(
Expand Down Expand Up @@ -592,7 +597,7 @@ public void testOrderOfSetupTeardownCalls() throws Exception {
executionStateSampler,
urnToPTransformRunnerFactoryMap,
Caches.noop(),
new BundleProcessorCache(),
new BundleProcessorCache(Duration.ZERO),
null /* dataSampler */);

handler.processBundle(
Expand Down Expand Up @@ -699,7 +704,7 @@ private static InstructionRequest processBundleRequestFor(
public void testBundleProcessorIsFoundWhenActive() {
BundleProcessor bundleProcessor = mock(BundleProcessor.class);
when(bundleProcessor.getInstructionId()).thenReturn("known");
BundleProcessorCache cache = new BundleProcessorCache();
BundleProcessorCache cache = new BundleProcessorCache(Duration.ZERO);

// Check that an unknown bundle processor is not found
assertNull(cache.find("unknown"));
Expand Down Expand Up @@ -811,7 +816,7 @@ public void testCreatingPTransformExceptionsArePropagated() throws Exception {
throw new IllegalStateException("TestException");
}),
Caches.noop(),
new BundleProcessorCache(),
new BundleProcessorCache(Duration.ZERO),
null /* dataSampler */);
assertThrows(
"TestException",
Expand Down Expand Up @@ -862,7 +867,7 @@ public void testBundleFinalizationIsPropagated() throws Exception {
return null;
}),
Caches.noop(),
new BundleProcessorCache(),
new BundleProcessorCache(Duration.ZERO),
null /* dataSampler */);
BeamFnApi.InstructionResponse.Builder response =
handler.processBundle(
Expand Down Expand Up @@ -916,7 +921,7 @@ public void testPTransformStartExceptionsArePropagated() {
return null;
}),
Caches.noop(),
new BundleProcessorCache(),
new BundleProcessorCache(Duration.ZERO),
null /* dataSampler */);
assertThrows(
"TestException",
Expand Down Expand Up @@ -1094,7 +1099,7 @@ public void onCompleted() {}
executionStateSampler,
urnToPTransformRunnerFactoryMap,
Caches.noop(),
new BundleProcessorCache(),
new BundleProcessorCache(Duration.ZERO),
null /* dataSampler */);
}

Expand Down Expand Up @@ -1427,7 +1432,7 @@ public void testInstructionIsUnregisteredFromBeamFnDataClientOnSuccess() throws
return null;
}),
Caches.noop(),
new BundleProcessorCache(),
new BundleProcessorCache(Duration.ZERO),
null /* dataSampler */);
handler.processBundle(
BeamFnApi.InstructionRequest.newBuilder()
Expand Down Expand Up @@ -1500,7 +1505,7 @@ public void testDataProcessingExceptionsArePropagated() throws Exception {
return null;
}),
Caches.noop(),
new BundleProcessorCache(),
new BundleProcessorCache(Duration.ZERO),
null /* dataSampler */);
assertThrows(
"TestException",
Expand Down Expand Up @@ -1551,7 +1556,7 @@ public void testPTransformFinishExceptionsArePropagated() throws Exception {
return null;
}),
Caches.noop(),
new BundleProcessorCache(),
new BundleProcessorCache(Duration.ZERO),
null /* dataSampler */);
assertThrows(
"TestException",
Expand Down Expand Up @@ -1647,7 +1652,7 @@ private void doStateCalls(BeamFnStateClient beamFnStateClient) {
}
}),
Caches.noop(),
new BundleProcessorCache(),
new BundleProcessorCache(Duration.ZERO),
null /* dataSampler */);
handler.processBundle(
BeamFnApi.InstructionRequest.newBuilder()
Expand Down Expand Up @@ -1698,7 +1703,7 @@ private void doStateCalls(BeamFnStateClient beamFnStateClient) {
}
}),
Caches.noop(),
new BundleProcessorCache(),
new BundleProcessorCache(Duration.ZERO),
null /* dataSampler */);
assertThrows(
"State API calls are unsupported",
Expand Down Expand Up @@ -1787,7 +1792,7 @@ public void reset() {
return null;
};

BundleProcessorCache bundleProcessorCache = new BundleProcessorCache();
BundleProcessorCache bundleProcessorCache = new BundleProcessorCache(Duration.ZERO);
ProcessBundleHandler handler =
new ProcessBundleHandler(
PipelineOptionsFactory.create(),
Expand Down Expand Up @@ -1930,7 +1935,7 @@ public Object createRunnerForPTransform(Context context) throws IOException {
}
}),
Caches.noop(),
new BundleProcessorCache(),
new BundleProcessorCache(Duration.ZERO),
null /* dataSampler */);
assertThrows(
"Timers are unsupported",
Expand Down
Loading