From 8b61a94b03bc7d3e143b192a93937959493f7a15 Mon Sep 17 00:00:00 2001 From: Arun Pandian Date: Fri, 6 Sep 2024 16:40:37 -0700 Subject: [PATCH 01/13] [Dataflow Streaming] Add support to read user worker settings from backend --- .../beam/gradle/BeamModulePlugin.groovy | 2 +- .../dataflow/worker/OperationalLimits.java | 39 ++--- .../worker/StreamingDataflowWorker.java | 63 +++---- .../worker/StreamingModeExecutionContext.java | 14 +- .../streaming/ComputationWorkExecutor.java | 5 +- ...reamingEngineComputationConfigFetcher.java | 46 ++++-- .../config/StreamingEnginePipelineConfig.java | 33 ++-- .../StreamingEnginePipelineConfigManager.java | 88 ++++++++++ .../harness/StreamingWorkerStatusPages.java | 21 ++- .../client/grpc/GrpcDispatcherClient.java | 9 + .../ComputationWorkExecutorFactory.java | 7 +- .../processing/StreamingWorkScheduler.java | 25 ++- .../worker/StreamingDataflowWorkerTest.java | 45 +++-- .../StreamingModeExecutionContextTest.java | 6 +- .../worker/WorkerCustomSourcesTest.java | 9 +- ...ingEngineComputationConfigFetcherTest.java | 93 +++++++---- ...eamingEnginePipelineConfigManagerTest.java | 154 ++++++++++++++++++ .../windmill/src/main/proto/windmill.proto | 9 + 18 files changed, 487 insertions(+), 181 deletions(-) create mode 100644 runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingEnginePipelineConfigManager.java create mode 100644 runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingEnginePipelineConfigManagerTest.java diff --git a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy index 9e5ffa8a1c42..a03645f7b953 100644 --- a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy +++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy @@ -738,7 +738,7 @@ class BeamModulePlugin implements Plugin { google_api_common : "com.google.api:api-common", // google_cloud_platform_libraries_bom sets version google_api_services_bigquery : "com.google.apis:google-api-services-bigquery:v2-rev20240323-2.0.0", // [bomupgrader] sets version google_api_services_cloudresourcemanager : "com.google.apis:google-api-services-cloudresourcemanager:v1-rev20240310-2.0.0", // [bomupgrader] sets version - google_api_services_dataflow : "com.google.apis:google-api-services-dataflow:v1b3-rev20240624-$google_clients_version", + google_api_services_dataflow : "com.google.apis:google-api-services-dataflow:v1b3-rev20240817-$google_clients_version", google_api_services_healthcare : "com.google.apis:google-api-services-healthcare:v1-rev20240130-$google_clients_version", google_api_services_pubsub : "com.google.apis:google-api-services-pubsub:v1-rev20220904-$google_clients_version", google_api_services_storage : "com.google.apis:google-api-services-storage:v1-rev20240319-2.0.0", // [bomupgrader] sets version diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/OperationalLimits.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/OperationalLimits.java index 47e36e498507..84f41c473fe0 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/OperationalLimits.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/OperationalLimits.java @@ -17,37 +17,38 @@ */ package org.apache.beam.runners.dataflow.worker; -import com.google.auto.value.AutoBuilder; +import com.google.auto.value.AutoValue; +import org.apache.beam.sdk.annotations.Internal; /** Keep track of any operational limits required by the backend. */ -public class OperationalLimits { +@AutoValue +@Internal +public abstract class OperationalLimits { + + private static final long DEFAULT_MAX_WORK_ITEM_COMMIT_BYTES = 180 << 20; + // Maximum size of a commit from a single work item. - public final long maxWorkItemCommitBytes; + public abstract long getMaxWorkItemCommitBytes(); // Maximum size of a single output element's serialized key. - public final long maxOutputKeyBytes; + public abstract long getMaxOutputKeyBytes(); // Maximum size of a single output element's serialized value. - public final long maxOutputValueBytes; + public abstract long getMaxOutputValueBytes(); - OperationalLimits(long maxWorkItemCommitBytes, long maxOutputKeyBytes, long maxOutputValueBytes) { - this.maxWorkItemCommitBytes = maxWorkItemCommitBytes; - this.maxOutputKeyBytes = maxOutputKeyBytes; - this.maxOutputValueBytes = maxOutputValueBytes; - } + @AutoValue.Builder + public abstract static class Builder { - @AutoBuilder(ofClass = OperationalLimits.class) - public interface Builder { - Builder setMaxWorkItemCommitBytes(long bytes); + public abstract Builder setMaxWorkItemCommitBytes(long bytes); - Builder setMaxOutputKeyBytes(long bytes); + public abstract Builder setMaxOutputKeyBytes(long bytes); - Builder setMaxOutputValueBytes(long bytes); + public abstract Builder setMaxOutputValueBytes(long bytes); - OperationalLimits build(); + public abstract OperationalLimits build(); } - public static Builder builder() { - return new AutoBuilder_OperationalLimits_Builder() - .setMaxWorkItemCommitBytes(Long.MAX_VALUE) + public static OperationalLimits.Builder builder() { + return new AutoValue_OperationalLimits.Builder() + .setMaxWorkItemCommitBytes(DEFAULT_MAX_WORK_ITEM_COMMIT_BYTES) .setMaxOutputKeyBytes(Long.MAX_VALUE) .setMaxOutputValueBytes(Long.MAX_VALUE); } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java index 1af677382092..fd328a5657a9 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java @@ -34,8 +34,6 @@ import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicReference; -import java.util.function.Consumer; import java.util.function.Function; import java.util.function.Supplier; import org.apache.beam.runners.core.metrics.MetricsLogger; @@ -52,6 +50,7 @@ import org.apache.beam.runners.dataflow.worker.streaming.config.StreamingApplianceComputationConfigFetcher; import org.apache.beam.runners.dataflow.worker.streaming.config.StreamingEngineComputationConfigFetcher; import org.apache.beam.runners.dataflow.worker.streaming.config.StreamingEnginePipelineConfig; +import org.apache.beam.runners.dataflow.worker.streaming.config.StreamingEnginePipelineConfigManager; import org.apache.beam.runners.dataflow.worker.streaming.harness.SingleSourceWorkerHarness; import org.apache.beam.runners.dataflow.worker.streaming.harness.SingleSourceWorkerHarness.GetWorkSender; import org.apache.beam.runners.dataflow.worker.streaming.harness.StreamingCounters; @@ -103,9 +102,7 @@ import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.ManagedChannel; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.cache.CacheStats; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.net.HostAndPort; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.ThreadFactoryBuilder; import org.joda.time.Duration; import org.joda.time.Instant; @@ -181,7 +178,7 @@ private StreamingDataflowWorker( WorkFailureProcessor workFailureProcessor, StreamingCounters streamingCounters, MemoryMonitor memoryMonitor, - AtomicReference operationalLimits, + StreamingEnginePipelineConfigManager configManager, GrpcWindmillStreamFactory windmillStreamFactory, Function executorSupplier, ConcurrentMap stageInfoMap) { @@ -237,8 +234,8 @@ private StreamingDataflowWorker( streamingCounters, hotKeyLogger, sampler, - operationalLimits, ID_GENERATOR, + configManager, stageInfoMap); ThrottlingGetDataMetricTracker getDataMetricTracker = @@ -298,6 +295,7 @@ private StreamingDataflowWorker( .setCurrentActiveCommitBytes(workCommitter::currentActiveCommitBytes) .setGetDataStatusProvider(getDataClient::printHtml) .setWorkUnitExecutor(workUnitExecutor) + .setConfigManager(configManager) .build(); Windmill.GetWorkRequest request = @@ -335,8 +333,6 @@ public static StreamingDataflowWorker fromOptions(DataflowWorkerHarnessOptions o StreamingCounters streamingCounters = StreamingCounters.create(); WorkUnitClient dataflowServiceClient = new DataflowWorkUnitClient(options, LOG); BoundedQueueExecutor workExecutor = createWorkUnitExecutor(options); - AtomicReference operationalLimits = - new AtomicReference<>(OperationalLimits.builder().build()); WindmillStateCache windmillStateCache = WindmillStateCache.builder() .setSizeMb(options.getWorkerCacheMb()) @@ -348,14 +344,18 @@ public static StreamingDataflowWorker fromOptions(DataflowWorkerHarnessOptions o new ThreadFactoryBuilder().setNameFormat(threadName).build()); GrpcWindmillStreamFactory.Builder windmillStreamFactoryBuilder = createGrpcwindmillStreamFactoryBuilder(options, clientId); + StreamingEnginePipelineConfigManager configManager = + new StreamingEnginePipelineConfigManager( + /*initializeWithDefaults=*/ !options + .isEnableStreamingEngine()); // appliance is initialized with default settings ConfigFetcherComputationStateCacheAndWindmillClient configFetcherComputationStateCacheAndWindmillClient = createConfigFetcherComputationStateCacheAndWindmillClient( options, dataflowServiceClient, - operationalLimits, windmillStreamFactoryBuilder, + configManager, configFetcher -> ComputationStateCache.create( configFetcher, @@ -412,7 +412,7 @@ public static StreamingDataflowWorker fromOptions(DataflowWorkerHarnessOptions o workFailureProcessor, streamingCounters, memoryMonitor, - operationalLimits, + configManager, configFetcherComputationStateCacheAndWindmillClient.windmillStreamFactory(), executorSupplier, stageInfo); @@ -428,8 +428,8 @@ public static StreamingDataflowWorker fromOptions(DataflowWorkerHarnessOptions o createConfigFetcherComputationStateCacheAndWindmillClient( DataflowWorkerHarnessOptions options, WorkUnitClient dataflowServiceClient, - AtomicReference operationalLimits, GrpcWindmillStreamFactory.Builder windmillStreamFactoryBuilder, + StreamingEnginePipelineConfigManager configManager, Function computationStateCacheFactory) { ComputationConfig.Fetcher configFetcher; WindmillServerStub windmillServer; @@ -438,15 +438,12 @@ public static StreamingDataflowWorker fromOptions(DataflowWorkerHarnessOptions o if (options.isEnableStreamingEngine()) { GrpcDispatcherClient dispatcherClient = GrpcDispatcherClient.create(createStubFactory(options)); + configManager.onConfig(dispatcherClient::onJobConfig); configFetcher = StreamingEngineComputationConfigFetcher.create( options.getGlobalConfigRefreshPeriod().getMillis(), dataflowServiceClient, - config -> - onPipelineConfig( - config, - dispatcherClient::consumeWindmillDispatcherEndpoints, - operationalLimits::set)); + configManager); computationStateCache = computationStateCacheFactory.apply(configFetcher); windmillStreamFactory = windmillStreamFactoryBuilder @@ -494,16 +491,19 @@ static StreamingDataflowWorker forTesting( HotKeyLogger hotKeyLogger, Supplier clock, Function executorSupplier, - int localRetryTimeoutMs, - OperationalLimits limits) { + StreamingEnginePipelineConfigManager configManager, + int localRetryTimeoutMs) { ConcurrentMap stageInfo = new ConcurrentHashMap<>(); - AtomicReference operationalLimits = new AtomicReference<>(limits); BoundedQueueExecutor workExecutor = createWorkUnitExecutor(options); WindmillStateCache stateCache = WindmillStateCache.builder() .setSizeMb(options.getWorkerCacheMb()) .setSupportMapViaMultimap(options.isEnableStreamingEngine()) .build(); + StreamingEnginePipelineConfig config = configManager.getConfig(); + if (!config.windmillServiceEndpoints().isEmpty()) { + windmillServer.setWindmillServiceEndpoints(config.windmillServiceEndpoints()); + } ComputationConfig.Fetcher configFetcher = options.isEnableStreamingEngine() ? StreamingEngineComputationConfigFetcher.forTesting( @@ -511,11 +511,7 @@ static StreamingDataflowWorker forTesting( options.getGlobalConfigRefreshPeriod().getMillis(), workUnitClient, executorSupplier, - config -> - onPipelineConfig( - config, - windmillServer::setWindmillServiceEndpoints, - operationalLimits::set)) + configManager) : new StreamingApplianceComputationConfigFetcher(windmillServer::getConfig); ConcurrentMap stateNameMap = new ConcurrentHashMap<>(prePopulatedStateNameMappings); @@ -583,7 +579,7 @@ static StreamingDataflowWorker forTesting( workFailureProcessor, streamingCounters, memoryMonitor, - operationalLimits, + configManager, options.isEnableStreamingEngine() ? windmillStreamFactory .setHealthCheckIntervalMillis( @@ -594,23 +590,6 @@ static StreamingDataflowWorker forTesting( stageInfo); } - private static void onPipelineConfig( - StreamingEnginePipelineConfig config, - Consumer> consumeWindmillServiceEndpoints, - Consumer operationalLimits) { - - operationalLimits.accept( - OperationalLimits.builder() - .setMaxWorkItemCommitBytes(config.maxWorkItemCommitBytes()) - .setMaxOutputKeyBytes(config.maxOutputKeyBytes()) - .setMaxOutputValueBytes(config.maxOutputValueBytes()) - .build()); - - if (!config.windmillServiceEndpoints().isEmpty()) { - consumeWindmillServiceEndpoints.accept(config.windmillServiceEndpoints()); - } - } - private static GrpcWindmillStreamFactory.Builder createGrpcwindmillStreamFactoryBuilder( DataflowWorkerHarnessOptions options, long clientId) { Duration maxBackoff = diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContext.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContext.java index f25f6294da86..d08aad4df515 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContext.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContext.java @@ -50,6 +50,7 @@ import org.apache.beam.runners.dataflow.worker.profiler.ScopedProfiler.ProfileScope; import org.apache.beam.runners.dataflow.worker.streaming.Watermarks; import org.apache.beam.runners.dataflow.worker.streaming.Work; +import org.apache.beam.runners.dataflow.worker.streaming.config.StreamingEnginePipelineConfigManager; import org.apache.beam.runners.dataflow.worker.streaming.sideinput.SideInput; import org.apache.beam.runners.dataflow.worker.streaming.sideinput.SideInputState; import org.apache.beam.runners.dataflow.worker.streaming.sideinput.SideInputStateFetcher; @@ -107,6 +108,7 @@ public class StreamingModeExecutionContext extends DataflowExecutionContext stateNameMap; private final WindmillStateCache.ForComputation stateCache; private final ReaderCache readerCache; + private final StreamingEnginePipelineConfigManager configManager; private final boolean throwExceptionOnLargeOutput; private volatile long backlogBytes; @@ -130,10 +132,6 @@ public class StreamingModeExecutionContext extends DataflowExecutionContext metricsContainerRegistry, DataflowExecutionStateTracker executionStateTracker, StreamingModeExecutionStateRegistry executionStateRegistry, + StreamingEnginePipelineConfigManager configManager, long sinkByteLimit, boolean throwExceptionOnLargeOutput) { super( @@ -163,6 +162,7 @@ public StreamingModeExecutionContext( sinkByteLimit); this.computationId = computationId; this.readerCache = readerCache; + this.configManager = configManager; this.sideInputCache = new HashMap<>(); this.stateNameMap = ImmutableMap.copyOf(stateNameMap); this.stateCache = stateCache; @@ -176,11 +176,11 @@ public final long getBacklogBytes() { } public long getMaxOutputKeyBytes() { - return operationalLimits.maxOutputKeyBytes; + return configManager.getConfig().operationalLimits().getMaxOutputKeyBytes(); } public long getMaxOutputValueBytes() { - return operationalLimits.maxOutputValueBytes; + return configManager.getConfig().operationalLimits().getMaxOutputValueBytes(); } public boolean throwExceptionsForLargeOutput() { @@ -196,13 +196,11 @@ public void start( Work work, WindmillStateReader stateReader, SideInputStateFetcher sideInputStateFetcher, - OperationalLimits operationalLimits, Windmill.WorkItemCommitRequest.Builder outputBuilder) { this.key = key; this.work = work; this.computationKey = WindmillComputationKey.create(computationId, work.getShardedKey()); this.sideInputStateFetcher = sideInputStateFetcher; - this.operationalLimits = operationalLimits; this.outputBuilder = outputBuilder; this.sideInputCache.clear(); clearSinkFullHint(); diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/ComputationWorkExecutor.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/ComputationWorkExecutor.java index 8a00194887da..8dc681fc640c 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/ComputationWorkExecutor.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/ComputationWorkExecutor.java @@ -24,7 +24,6 @@ import org.apache.beam.runners.core.metrics.ExecutionStateTracker; import org.apache.beam.runners.dataflow.worker.DataflowMapTaskExecutor; import org.apache.beam.runners.dataflow.worker.DataflowWorkExecutor; -import org.apache.beam.runners.dataflow.worker.OperationalLimits; import org.apache.beam.runners.dataflow.worker.StreamingModeExecutionContext; import org.apache.beam.runners.dataflow.worker.streaming.sideinput.SideInputStateFetcher; import org.apache.beam.runners.dataflow.worker.util.common.worker.ElementCounter; @@ -73,11 +72,9 @@ public final void executeWork( Work work, WindmillStateReader stateReader, SideInputStateFetcher sideInputStateFetcher, - OperationalLimits operationalLimits, Windmill.WorkItemCommitRequest.Builder outputBuilder) throws Exception { - context() - .start(key, work, stateReader, sideInputStateFetcher, operationalLimits, outputBuilder); + context().start(key, work, stateReader, sideInputStateFetcher, outputBuilder); workExecutor().execute(); } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingEngineComputationConfigFetcher.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingEngineComputationConfigFetcher.java index d230aac54c63..d9377a258dc7 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingEngineComputationConfigFetcher.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingEngineComputationConfigFetcher.java @@ -30,16 +30,18 @@ import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; -import java.util.function.Consumer; import java.util.function.Function; import java.util.stream.StreamSupport; import javax.annotation.concurrent.ThreadSafe; +import org.apache.beam.runners.dataflow.worker.OperationalLimits; import org.apache.beam.runners.dataflow.worker.WorkUnitClient; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill.UserWorkerRunnerV1Settings; import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.sdk.util.BackOff; import org.apache.beam.sdk.util.BackOffUtils; import org.apache.beam.sdk.util.FluentBackoff; import org.apache.beam.sdk.util.Sleeper; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.InvalidProtocolBufferException; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Splitter; @@ -72,7 +74,7 @@ public final class StreamingEngineComputationConfigFetcher implements Computatio private final long globalConfigRefreshPeriodMillis; private final WorkUnitClient dataflowServiceClient; private final ScheduledExecutorService globalConfigRefresher; - private final Consumer onStreamingConfig; + private final StreamingEnginePipelineConfigManager configManager; private final AtomicBoolean hasReceivedGlobalConfig; private StreamingEngineComputationConfigFetcher( @@ -80,25 +82,25 @@ private StreamingEngineComputationConfigFetcher( long globalConfigRefreshPeriodMillis, WorkUnitClient dataflowServiceClient, ScheduledExecutorService globalConfigRefresher, - Consumer onStreamingConfig) { + StreamingEnginePipelineConfigManager configManager) { this.globalConfigRefreshPeriodMillis = globalConfigRefreshPeriodMillis; this.dataflowServiceClient = dataflowServiceClient; this.globalConfigRefresher = globalConfigRefresher; - this.onStreamingConfig = onStreamingConfig; + this.configManager = configManager; this.hasReceivedGlobalConfig = new AtomicBoolean(hasReceivedGlobalConfig); } public static StreamingEngineComputationConfigFetcher create( long globalConfigRefreshPeriodMillis, WorkUnitClient dataflowServiceClient, - Consumer onStreamingConfig) { + StreamingEnginePipelineConfigManager configManager) { return new StreamingEngineComputationConfigFetcher( /* hasReceivedGlobalConfig= */ false, globalConfigRefreshPeriodMillis, dataflowServiceClient, Executors.newSingleThreadScheduledExecutor( new ThreadFactoryBuilder().setNameFormat(CONFIG_REFRESHER_THREAD_NAME).build()), - onStreamingConfig); + configManager); } @VisibleForTesting @@ -107,13 +109,13 @@ public static StreamingEngineComputationConfigFetcher forTesting( long globalConfigRefreshPeriodMillis, WorkUnitClient dataflowServiceClient, Function executorSupplier, - Consumer onStreamingConfig) { + StreamingEnginePipelineConfigManager configManager) { return new StreamingEngineComputationConfigFetcher( hasReceivedGlobalConfig, globalConfigRefreshPeriodMillis, dataflowServiceClient, executorSupplier.apply(CONFIG_REFRESHER_THREAD_NAME), - onStreamingConfig); + configManager); } @VisibleForTesting @@ -159,9 +161,7 @@ private static Optional fetchConfigWithRetry( private StreamingEnginePipelineConfig createPipelineConfig(StreamingConfigTask config) { StreamingEnginePipelineConfig.Builder pipelineConfig = StreamingEnginePipelineConfig.builder(); - if (config.getUserStepToStateFamilyNameMap() != null) { - pipelineConfig.setUserStepToStateFamilyNameMap(config.getUserStepToStateFamilyNameMap()); - } + OperationalLimits.Builder operationalLimits = OperationalLimits.builder(); if (config.getWindmillServiceEndpoint() != null && !config.getWindmillServiceEndpoint().isEmpty()) { @@ -184,23 +184,36 @@ private StreamingEnginePipelineConfig createPipelineConfig(StreamingConfigTask c if (config.getMaxWorkItemCommitBytes() != null && config.getMaxWorkItemCommitBytes() > 0 && config.getMaxWorkItemCommitBytes() <= Integer.MAX_VALUE) { - pipelineConfig.setMaxWorkItemCommitBytes(config.getMaxWorkItemCommitBytes().intValue()); + operationalLimits.setMaxWorkItemCommitBytes(config.getMaxWorkItemCommitBytes().intValue()); } if (config.getOperationalLimits() != null) { if (config.getOperationalLimits().getMaxKeyBytes() != null && config.getOperationalLimits().getMaxKeyBytes() > 0 && config.getOperationalLimits().getMaxKeyBytes() <= Integer.MAX_VALUE) { - pipelineConfig.setMaxOutputKeyBytes(config.getOperationalLimits().getMaxKeyBytes()); + operationalLimits.setMaxOutputKeyBytes(config.getOperationalLimits().getMaxKeyBytes()); } if (config.getOperationalLimits().getMaxProductionOutputBytes() != null && config.getOperationalLimits().getMaxProductionOutputBytes() > 0 && config.getOperationalLimits().getMaxProductionOutputBytes() <= Integer.MAX_VALUE) { - pipelineConfig.setMaxOutputValueBytes( + operationalLimits.setMaxOutputValueBytes( config.getOperationalLimits().getMaxProductionOutputBytes()); } } + pipelineConfig.setOperationalLimits(operationalLimits.build()); + + byte[] settings_bytes = config.decodeUserWorkerRunnerV1Settings(); + if (settings_bytes != null) { + UserWorkerRunnerV1Settings settings = UserWorkerRunnerV1Settings.newBuilder().build(); + try { + settings = UserWorkerRunnerV1Settings.parseFrom(settings_bytes); + } catch (InvalidProtocolBufferException e) { + LOG.error("Parsing UserWorkerRunnerV1Settings failed", e); + } + pipelineConfig.setUserWorkerJobSettings(settings); + } + return pipelineConfig.build(); } @@ -259,7 +272,7 @@ public void stop() { @SuppressWarnings("FutureReturnValueIgnored") private void schedulePeriodicGlobalConfigRequests() { globalConfigRefresher.scheduleWithFixedDelay( - () -> fetchGlobalConfig().ifPresent(onStreamingConfig), + () -> fetchGlobalConfig().ifPresent(configManager::setConfig), 0, globalConfigRefreshPeriodMillis, TimeUnit.MILLISECONDS); @@ -274,7 +287,7 @@ private synchronized void fetchInitialPipelineGlobalConfig() { LOG.info("Sending request to get initial global configuration for this worker."); Optional globalConfig = fetchGlobalConfig(); if (globalConfig.isPresent()) { - onStreamingConfig.accept(globalConfig.get()); + configManager.setConfig(globalConfig.get()); hasReceivedGlobalConfig.set(true); break; } @@ -292,6 +305,7 @@ private Optional fetchGlobalConfig() { @FunctionalInterface private interface ThrowingFetchWorkItemFn { + Optional fetchWorkItem() throws IOException; } } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingEnginePipelineConfig.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingEnginePipelineConfig.java index 8f1ff93f6a49..d64f8b4d10cc 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingEnginePipelineConfig.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingEnginePipelineConfig.java @@ -18,8 +18,8 @@ package org.apache.beam.runners.dataflow.worker.streaming.config; import com.google.auto.value.AutoValue; -import java.util.HashMap; -import java.util.Map; +import org.apache.beam.runners.dataflow.worker.OperationalLimits; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill.UserWorkerRunnerV1Settings; import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.net.HostAndPort; @@ -29,38 +29,27 @@ @Internal public abstract class StreamingEnginePipelineConfig { - private static final long DEFAULT_MAX_WORK_ITEM_COMMIT_BYTES = 180 << 20; - public static StreamingEnginePipelineConfig.Builder builder() { return new AutoValue_StreamingEnginePipelineConfig.Builder() - .setMaxWorkItemCommitBytes(DEFAULT_MAX_WORK_ITEM_COMMIT_BYTES) - .setMaxOutputKeyBytes(Long.MAX_VALUE) - .setMaxOutputValueBytes(Long.MAX_VALUE) - .setUserStepToStateFamilyNameMap(new HashMap<>()) - .setWindmillServiceEndpoints(ImmutableSet.of()); + .setWindmillServiceEndpoints(ImmutableSet.of()) + .setUserWorkerJobSettings(UserWorkerRunnerV1Settings.newBuilder().build()) + .setOperationalLimits(OperationalLimits.builder().build()); } - public abstract long maxWorkItemCommitBytes(); - - public abstract long maxOutputKeyBytes(); - - public abstract long maxOutputValueBytes(); - - public abstract Map userStepToStateFamilyNameMap(); + public abstract OperationalLimits operationalLimits(); public abstract ImmutableSet windmillServiceEndpoints(); + public abstract UserWorkerRunnerV1Settings userWorkerJobSettings(); + @AutoValue.Builder public abstract static class Builder { - public abstract Builder setMaxWorkItemCommitBytes(long value); - - public abstract Builder setMaxOutputKeyBytes(long value); - public abstract Builder setMaxOutputValueBytes(long value); + public abstract Builder setWindmillServiceEndpoints(ImmutableSet value); - public abstract Builder setUserStepToStateFamilyNameMap(Map value); + public abstract Builder setOperationalLimits(OperationalLimits operationalLimits); - public abstract Builder setWindmillServiceEndpoints(ImmutableSet value); + public abstract Builder setUserWorkerJobSettings(UserWorkerRunnerV1Settings settings); public abstract StreamingEnginePipelineConfig build(); } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingEnginePipelineConfigManager.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingEnginePipelineConfigManager.java new file mode 100644 index 000000000000..7b9ae5b1eb40 --- /dev/null +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingEnginePipelineConfigManager.java @@ -0,0 +1,88 @@ +/* + * 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.dataflow.worker.streaming.config; + +import com.google.common.base.Preconditions; +import java.util.Iterator; +import java.util.concurrent.CopyOnWriteArrayList; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Consumer; +import javax.annotation.Nonnull; +import javax.annotation.concurrent.ThreadSafe; +import org.apache.beam.sdk.annotations.Internal; + +@Internal +@ThreadSafe +public class StreamingEnginePipelineConfigManager { + + private final AtomicReference streamingEngineConfig = + new AtomicReference<>(); + + private final CopyOnWriteArrayList> config_callbacks = + new CopyOnWriteArrayList<>(); + + public StreamingEnginePipelineConfigManager(boolean initializeWithDefaults) { + if (initializeWithDefaults) { + streamingEngineConfig.set(StreamingEnginePipelineConfig.builder().build()); + } + } + + /* + * Returns the latest StreamingEnginePipelineConfig + */ + public StreamingEnginePipelineConfig getConfig() { + Preconditions.checkState( + streamingEngineConfig.get() != null, + "Global config should be set before any processing is done"); + return streamingEngineConfig.get(); + } + + /* + * Subscribe to config updates by registering a callback. + * Callback will be called the first time with settings, if any, inline before the method returns. + */ + public void onConfig(@Nonnull Consumer callback) { + StreamingEnginePipelineConfig config; + synchronized (this) { + config_callbacks.add(callback); + config = streamingEngineConfig.get(); + } + if (config != null) { + callback.accept(config); + } + } + + /* + * Package private setter for setting config + */ + void setConfig(@Nonnull StreamingEnginePipelineConfig config) { + Iterator> iterator; + synchronized (this) { + if (config.equals(streamingEngineConfig.get())) { + return; + } + streamingEngineConfig.set(config); + // iterator of CopyOnWriteArrayList provides + // snapshot semantics + iterator = config_callbacks.iterator(); + } + while (iterator.hasNext()) { + iterator.next().accept(config); + } + } +} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/harness/StreamingWorkerStatusPages.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/harness/StreamingWorkerStatusPages.java index d305e25af7e5..5088ef334069 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/harness/StreamingWorkerStatusPages.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/harness/StreamingWorkerStatusPages.java @@ -28,6 +28,7 @@ import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicReference; import java.util.function.Consumer; import java.util.function.Supplier; import javax.servlet.http.HttpServletRequest; @@ -38,6 +39,8 @@ import org.apache.beam.runners.dataflow.worker.status.LastExceptionDataProvider; import org.apache.beam.runners.dataflow.worker.status.WorkerStatusPages; import org.apache.beam.runners.dataflow.worker.streaming.ComputationStateCache; +import org.apache.beam.runners.dataflow.worker.streaming.config.StreamingEnginePipelineConfig; +import org.apache.beam.runners.dataflow.worker.streaming.config.StreamingEnginePipelineConfigManager; import org.apache.beam.runners.dataflow.worker.util.BoundedQueueExecutor; import org.apache.beam.runners.dataflow.worker.windmill.client.grpc.ChannelzServlet; import org.apache.beam.runners.dataflow.worker.windmill.client.grpc.GrpcWindmillStreamFactory; @@ -77,6 +80,8 @@ public final class StreamingWorkerStatusPages { private final DebugCapture.@Nullable Manager debugCapture; private final @Nullable ChannelzServlet channelzServlet; + private final AtomicReference seConfig = new AtomicReference<>(); + StreamingWorkerStatusPages( Supplier clock, long clientId, @@ -90,7 +95,8 @@ public final class StreamingWorkerStatusPages { @Nullable GrpcWindmillStreamFactory windmillStreamFactory, Consumer getDataStatusProvider, BoundedQueueExecutor workUnitExecutor, - ScheduledExecutorService statusPageDumper) { + ScheduledExecutorService statusPageDumper, + StreamingEnginePipelineConfigManager configManager) { this.clock = clock; this.clientId = clientId; this.isRunning = isRunning; @@ -104,6 +110,7 @@ public final class StreamingWorkerStatusPages { this.getDataStatusProvider = getDataStatusProvider; this.workUnitExecutor = workUnitExecutor; this.statusPageDumper = statusPageDumper; + configManager.onConfig(seConfig::set); } public static StreamingWorkerStatusPages.Builder builder() { @@ -150,6 +157,16 @@ private void addStreamingEngineStatusPages() { statusPages.addCapturePage(Preconditions.checkNotNull(channelzServlet)); statusPages.addStatusDataProvider( "streaming", "Streaming RPCs", Preconditions.checkNotNull(windmillStreamFactory)); + statusPages.addStatusDataProvider( + "jobSettings", + "User Worker Job Settings", + writer -> { + if (seConfig.get() == null) { + writer.println("Job Settings not loaded."); + return; + } + writer.println(seConfig.get().userWorkerJobSettings().toString()); + }); } private boolean isStreamingEngine() { @@ -256,6 +273,8 @@ public interface Builder { Builder setStatusPageDumper(ScheduledExecutorService statusPageDumper); + Builder setConfigManager(StreamingEnginePipelineConfigManager configManager); + StreamingWorkerStatusPages build(); } } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcDispatcherClient.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcDispatcherClient.java index cf2e7260592d..d34826a9b9a8 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcDispatcherClient.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcDispatcherClient.java @@ -29,6 +29,7 @@ import java.util.concurrent.atomic.AtomicReference; import javax.annotation.concurrent.GuardedBy; import javax.annotation.concurrent.ThreadSafe; +import org.apache.beam.runners.dataflow.worker.streaming.config.StreamingEnginePipelineConfig; import org.apache.beam.runners.dataflow.worker.windmill.CloudWindmillMetadataServiceV1Alpha1Grpc; import org.apache.beam.runners.dataflow.worker.windmill.CloudWindmillMetadataServiceV1Alpha1Grpc.CloudWindmillMetadataServiceV1Alpha1Stub; import org.apache.beam.runners.dataflow.worker.windmill.CloudWindmillServiceV1Alpha1Grpc; @@ -46,6 +47,7 @@ /** Manages endpoints and stubs for connecting to the Windmill Dispatcher. */ @ThreadSafe public class GrpcDispatcherClient { + private static final Logger LOG = LoggerFactory.getLogger(GrpcDispatcherClient.class); private final WindmillStubFactory windmillStubFactory; private final CountDownLatch onInitializedEndpoints; @@ -146,6 +148,13 @@ public boolean hasInitializedEndpoints() { return dispatcherStubs.get().hasInitializedEndpoints(); } + public void onJobConfig(StreamingEnginePipelineConfig config) { + if (config.windmillServiceEndpoints().isEmpty()) { + return; + } + consumeWindmillDispatcherEndpoints(config.windmillServiceEndpoints()); + } + public synchronized void consumeWindmillDispatcherEndpoints( ImmutableSet dispatcherEndpoints) { ImmutableSet currentDispatcherEndpoints = diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/processing/ComputationWorkExecutorFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/processing/ComputationWorkExecutorFactory.java index 20c1247b2168..cd08677573be 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/processing/ComputationWorkExecutorFactory.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/processing/ComputationWorkExecutorFactory.java @@ -47,6 +47,7 @@ import org.apache.beam.runners.dataflow.worker.streaming.ComputationState; import org.apache.beam.runners.dataflow.worker.streaming.ComputationWorkExecutor; import org.apache.beam.runners.dataflow.worker.streaming.StageInfo; +import org.apache.beam.runners.dataflow.worker.streaming.config.StreamingEnginePipelineConfigManager; import org.apache.beam.runners.dataflow.worker.util.common.worker.MapTaskExecutor; import org.apache.beam.runners.dataflow.worker.util.common.worker.OutputObjectAndByteCounter; import org.apache.beam.runners.dataflow.worker.util.common.worker.ReadOperation; @@ -94,6 +95,7 @@ final class ComputationWorkExecutorFactory { private final long maxSinkBytes; private final IdGenerator idGenerator; + private final StreamingEnginePipelineConfigManager configManager; private final boolean throwExceptionOnLargeOutput; ComputationWorkExecutorFactory( @@ -103,12 +105,14 @@ final class ComputationWorkExecutorFactory { Function stateCacheFactory, DataflowExecutionStateSampler sampler, CounterSet pendingDeltaCounters, - IdGenerator idGenerator) { + IdGenerator idGenerator, + StreamingEnginePipelineConfigManager configManager) { this.options = options; this.mapTaskExecutorFactory = mapTaskExecutorFactory; this.readerCache = readerCache; this.stateCacheFactory = stateCacheFactory; this.idGenerator = idGenerator; + this.configManager = configManager; this.readerRegistry = ReaderRegistry.defaultRegistry(); this.sinkRegistry = SinkRegistry.defaultRegistry(); this.sampler = sampler; @@ -262,6 +266,7 @@ private StreamingModeExecutionContext createExecutionContext( stageInfo.metricsContainerRegistry(), executionStateTracker, stageInfo.executionStateRegistry(), + configManager, maxSinkBytes, throwExceptionOnLargeOutput); } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/processing/StreamingWorkScheduler.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/processing/StreamingWorkScheduler.java index 86f2cffe604c..6df49bba9b30 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/processing/StreamingWorkScheduler.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/processing/StreamingWorkScheduler.java @@ -25,7 +25,6 @@ import java.util.Optional; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicReference; import java.util.function.Function; import java.util.function.Supplier; import javax.annotation.concurrent.ThreadSafe; @@ -33,7 +32,6 @@ import org.apache.beam.runners.dataflow.worker.DataflowExecutionStateSampler; import org.apache.beam.runners.dataflow.worker.DataflowMapTaskExecutorFactory; import org.apache.beam.runners.dataflow.worker.HotKeyLogger; -import org.apache.beam.runners.dataflow.worker.OperationalLimits; import org.apache.beam.runners.dataflow.worker.ReaderCache; import org.apache.beam.runners.dataflow.worker.WorkItemCancelledException; import org.apache.beam.runners.dataflow.worker.logging.DataflowWorkerLoggingMDC; @@ -44,6 +42,7 @@ import org.apache.beam.runners.dataflow.worker.streaming.StageInfo; import org.apache.beam.runners.dataflow.worker.streaming.Watermarks; import org.apache.beam.runners.dataflow.worker.streaming.Work; +import org.apache.beam.runners.dataflow.worker.streaming.config.StreamingEnginePipelineConfigManager; import org.apache.beam.runners.dataflow.worker.streaming.harness.StreamingCounters; import org.apache.beam.runners.dataflow.worker.streaming.sideinput.SideInputStateFetcher; import org.apache.beam.runners.dataflow.worker.streaming.sideinput.SideInputStateFetcherFactory; @@ -85,7 +84,7 @@ public final class StreamingWorkScheduler { private final HotKeyLogger hotKeyLogger; private final ConcurrentMap stageInfoMap; private final DataflowExecutionStateSampler sampler; - private final AtomicReference operationalLimits; + private final StreamingEnginePipelineConfigManager configManager; public StreamingWorkScheduler( DataflowWorkerHarnessOptions options, @@ -99,7 +98,7 @@ public StreamingWorkScheduler( HotKeyLogger hotKeyLogger, ConcurrentMap stageInfoMap, DataflowExecutionStateSampler sampler, - AtomicReference operationalLimits) { + StreamingEnginePipelineConfigManager configManager) { this.options = options; this.clock = clock; this.computationWorkExecutorFactory = computationWorkExecutorFactory; @@ -111,7 +110,7 @@ public StreamingWorkScheduler( this.hotKeyLogger = hotKeyLogger; this.stageInfoMap = stageInfoMap; this.sampler = sampler; - this.operationalLimits = operationalLimits; + this.configManager = configManager; } public static StreamingWorkScheduler create( @@ -126,8 +125,8 @@ public static StreamingWorkScheduler create( StreamingCounters streamingCounters, HotKeyLogger hotKeyLogger, DataflowExecutionStateSampler sampler, - AtomicReference operationalLimits, IdGenerator idGenerator, + StreamingEnginePipelineConfigManager configManager, ConcurrentMap stageInfoMap) { ComputationWorkExecutorFactory computationWorkExecutorFactory = new ComputationWorkExecutorFactory( @@ -137,7 +136,8 @@ public static StreamingWorkScheduler create( stateCacheFactory, sampler, streamingCounters.pendingDeltaCounters(), - idGenerator); + idGenerator, + configManager); return new StreamingWorkScheduler( options, @@ -151,7 +151,7 @@ public static StreamingWorkScheduler create( hotKeyLogger, stageInfoMap, sampler, - operationalLimits); + configManager); } private static long computeShuffleBytesRead(Windmill.WorkItem workItem) { @@ -295,7 +295,7 @@ private Windmill.WorkItemCommitRequest validateCommitRequestSize( Windmill.WorkItemCommitRequest commitRequest, String computationId, Windmill.WorkItem workItem) { - long byteLimit = operationalLimits.get().maxWorkItemCommitBytes; + long byteLimit = configManager.getConfig().operationalLimits().getMaxWorkItemCommitBytes(); int commitSize = commitRequest.getSerializedSize(); int estimatedCommitSize = commitSize < 0 ? Integer.MAX_VALUE : commitSize; @@ -380,12 +380,7 @@ private ExecuteWorkResult executeWork( // Blocks while executing work. computationWorkExecutor.executeWork( - executionKey, - work, - stateReader, - localSideInputStateFetcher, - operationalLimits.get(), - outputBuilder); + executionKey, work, stateReader, localSideInputStateFetcher, outputBuilder); if (work.isFailed()) { throw new WorkItemCancelledException(workItem.getShardingKey()); diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorkerTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorkerTest.java index b41ad391d878..06adb93f2a79 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorkerTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorkerTest.java @@ -102,6 +102,8 @@ import org.apache.beam.runners.dataflow.worker.streaming.ShardedKey; import org.apache.beam.runners.dataflow.worker.streaming.Watermarks; import org.apache.beam.runners.dataflow.worker.streaming.Work; +import org.apache.beam.runners.dataflow.worker.streaming.config.StreamingEnginePipelineConfig; +import org.apache.beam.runners.dataflow.worker.streaming.config.StreamingEnginePipelineConfigManager; import org.apache.beam.runners.dataflow.worker.testing.RestoreDataflowLoggingMDC; import org.apache.beam.runners.dataflow.worker.testing.TestCountingSource; import org.apache.beam.runners.dataflow.worker.util.BoundedQueueExecutor; @@ -275,6 +277,8 @@ public Long get() { @Rule public TestRule restoreMDC = new RestoreDataflowLoggingMDC(); @Rule public ErrorCollector errorCollector = new ErrorCollector(); WorkUnitClient mockWorkUnitClient = mock(WorkUnitClient.class); + StreamingEnginePipelineConfigManager mockConfigManager = + mock(StreamingEnginePipelineConfigManager.class); HotKeyLogger hotKeyLogger = mock(HotKeyLogger.class); private @Nullable ComputationStateCache computationStateCache = null; @@ -750,7 +754,9 @@ private StringBuilder initializeExpectedCommitRequest( requestBuilder.append("cache_token: "); requestBuilder.append(index + 1); requestBuilder.append(" "); - if (hasSourceBytesProcessed) requestBuilder.append("source_bytes_processed: 0 "); + if (hasSourceBytesProcessed) { + requestBuilder.append("source_bytes_processed: 0 "); + } return requestBuilder; } @@ -834,6 +840,8 @@ private DataflowWorkerHarnessOptions createTestingPipelineOptions(String... args private StreamingDataflowWorker makeWorker( StreamingDataflowWorkerTestParams streamingDataflowWorkerTestParams) { + when(mockConfigManager.getConfig()) + .thenReturn(streamingDataflowWorkerTestParams.streamingEnginePipelineConfig()); StreamingDataflowWorker worker = StreamingDataflowWorker.forTesting( streamingDataflowWorkerTestParams.stateNameMappings(), @@ -847,8 +855,8 @@ private StreamingDataflowWorker makeWorker( hotKeyLogger, streamingDataflowWorkerTestParams.clock(), streamingDataflowWorkerTestParams.executorSupplier(), - streamingDataflowWorkerTestParams.localRetryTimeoutMs(), - streamingDataflowWorkerTestParams.operationalLimits()); + mockConfigManager, + streamingDataflowWorkerTestParams.localRetryTimeoutMs()); this.computationStateCache = worker.getComputationStateCache(); return worker; } @@ -1210,8 +1218,11 @@ public void testKeyCommitTooLargeException() throws Exception { makeWorker( defaultWorkerParams() .setInstructions(instructions) - .setOperationalLimits( - OperationalLimits.builder().setMaxWorkItemCommitBytes(1000).build()) + .setStreamingEnginePipelineConfig( + StreamingEnginePipelineConfig.builder() + .setOperationalLimits( + OperationalLimits.builder().setMaxWorkItemCommitBytes(1000).build()) + .build()) .publishCounters() .build()); worker.start(); @@ -1282,7 +1293,11 @@ public void testOutputKeyTooLargeException() throws Exception { makeWorker( defaultWorkerParams("--experiments=throw_exceptions_on_large_output") .setInstructions(instructions) - .setOperationalLimits(OperationalLimits.builder().setMaxOutputKeyBytes(15).build()) + .setStreamingEnginePipelineConfig( + StreamingEnginePipelineConfig.builder() + .setOperationalLimits( + OperationalLimits.builder().setMaxOutputKeyBytes(15).build()) + .build()) .build()); worker.start(); @@ -1315,8 +1330,11 @@ public void testOutputValueTooLargeException() throws Exception { makeWorker( defaultWorkerParams("--experiments=throw_exceptions_on_large_output") .setInstructions(instructions) - .setOperationalLimits( - OperationalLimits.builder().setMaxOutputValueBytes(15).build()) + .setStreamingEnginePipelineConfig( + StreamingEnginePipelineConfig.builder() + .setOperationalLimits( + OperationalLimits.builder().setMaxOutputValueBytes(15).build()) + .build()) .build()); worker.start(); @@ -4412,7 +4430,9 @@ Duration getLatencyAttributionDuration(long workToken, LatencyAttribution.State } boolean isActiveWorkRefresh(GetDataRequest request) { - if (request.getComputationHeartbeatRequestCount() > 0) return true; + if (request.getComputationHeartbeatRequestCount() > 0) { + return true; + } for (ComputationGetDataRequest computationRequest : request.getRequestsList()) { if (!computationRequest.getComputationId().equals(DEFAULT_COMPUTATION_ID)) { return false; @@ -4508,7 +4528,7 @@ private static StreamingDataflowWorkerTestParams.Builder builder() { .setLocalRetryTimeoutMs(-1) .setPublishCounters(false) .setClock(Instant::now) - .setOperationalLimits(OperationalLimits.builder().build()); + .setStreamingEnginePipelineConfig(StreamingEnginePipelineConfig.builder().build()); } abstract ImmutableMap stateNameMappings(); @@ -4525,10 +4545,11 @@ private static StreamingDataflowWorkerTestParams.Builder builder() { abstract int localRetryTimeoutMs(); - abstract OperationalLimits operationalLimits(); + abstract StreamingEnginePipelineConfig streamingEnginePipelineConfig(); @AutoValue.Builder abstract static class Builder { + abstract Builder setStateNameMappings(ImmutableMap value); abstract ImmutableMap.Builder stateNameMappingsBuilder(); @@ -4559,7 +4580,7 @@ final Builder publishCounters() { abstract Builder setLocalRetryTimeoutMs(int value); - abstract Builder setOperationalLimits(OperationalLimits operationalLimits); + abstract Builder setStreamingEnginePipelineConfig(StreamingEnginePipelineConfig config); abstract StreamingDataflowWorkerTestParams build(); } diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContextTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContextTest.java index 86ed8f552d16..472500cb9b20 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContextTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContextTest.java @@ -59,6 +59,7 @@ import org.apache.beam.runners.dataflow.worker.profiler.ScopedProfiler.ProfileScope; import org.apache.beam.runners.dataflow.worker.streaming.Watermarks; import org.apache.beam.runners.dataflow.worker.streaming.Work; +import org.apache.beam.runners.dataflow.worker.streaming.config.StreamingEnginePipelineConfigManager; import org.apache.beam.runners.dataflow.worker.streaming.sideinput.SideInputStateFetcher; import org.apache.beam.runners.dataflow.worker.windmill.Windmill; import org.apache.beam.runners.dataflow.worker.windmill.client.getdata.FakeGetDataClient; @@ -107,6 +108,8 @@ public void setUp() { options = PipelineOptionsFactory.as(DataflowWorkerHarnessOptions.class); CounterSet counterSet = new CounterSet(); ConcurrentHashMap stateNameMap = new ConcurrentHashMap<>(); + StreamingEnginePipelineConfigManager configManager = + new StreamingEnginePipelineConfigManager(/*initializeWithDefaults=*/ true); stateNameMap.put(NameContextsForTests.nameContextForTest().userName(), "testStateFamily"); executionContext = new StreamingModeExecutionContext( @@ -127,6 +130,7 @@ public void setUp() { PipelineOptionsFactory.create(), "test-work-item-id"), executionStateRegistry, + configManager, Long.MAX_VALUE, /*throwExceptionOnLargeOutput=*/ false); } @@ -158,7 +162,6 @@ public void testTimerInternalsSetTimer() { Watermarks.builder().setInputDataWatermark(new Instant(1000)).build()), stateReader, sideInputStateFetcher, - OperationalLimits.builder().build(), outputBuilder); TimerInternals timerInternals = stepContext.timerInternals(); @@ -208,7 +211,6 @@ public void testTimerInternalsProcessingTimeSkew() { Watermarks.builder().setInputDataWatermark(new Instant(1000)).build()), stateReader, sideInputStateFetcher, - OperationalLimits.builder().build(), outputBuilder); TimerInternals timerInternals = stepContext.timerInternals(); assertTrue(timerTimestamp.isBefore(timerInternals.currentProcessingTime())); diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSourcesTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSourcesTest.java index f2e03b453fd8..f75b9bb504b9 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSourcesTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSourcesTest.java @@ -90,6 +90,7 @@ import org.apache.beam.runners.dataflow.worker.profiler.ScopedProfiler.NoopProfileScope; import org.apache.beam.runners.dataflow.worker.streaming.Watermarks; import org.apache.beam.runners.dataflow.worker.streaming.Work; +import org.apache.beam.runners.dataflow.worker.streaming.config.StreamingEnginePipelineConfigManager; import org.apache.beam.runners.dataflow.worker.streaming.sideinput.SideInputStateFetcher; import org.apache.beam.runners.dataflow.worker.testing.TestCountingSource; import org.apache.beam.runners.dataflow.worker.util.common.worker.NativeReader; @@ -594,6 +595,8 @@ public void testReadUnboundedReader() throws Exception { StreamingModeExecutionStateRegistry executionStateRegistry = new StreamingModeExecutionStateRegistry(); ReaderCache readerCache = new ReaderCache(Duration.standardMinutes(1), Runnable::run); + StreamingEnginePipelineConfigManager configManager = + new StreamingEnginePipelineConfigManager(/*initializeWithDefaults=*/ true); StreamingModeExecutionContext context = new StreamingModeExecutionContext( counterSet, @@ -610,6 +613,7 @@ public void testReadUnboundedReader() throws Exception { PipelineOptionsFactory.create(), "test-work-item-id"), executionStateRegistry, + configManager, Long.MAX_VALUE, /*throwExceptionOnLargeOutput=*/ false); @@ -635,7 +639,6 @@ public void testReadUnboundedReader() throws Exception { Watermarks.builder().setInputDataWatermark(new Instant(0)).build()), mock(WindmillStateReader.class), mock(SideInputStateFetcher.class), - OperationalLimits.builder().build(), Windmill.WorkItemCommitRequest.newBuilder()); @SuppressWarnings({"unchecked", "rawtypes"}) @@ -960,6 +963,8 @@ public void testFailedWorkItemsAbort() throws Exception { CounterSet counterSet = new CounterSet(); StreamingModeExecutionStateRegistry executionStateRegistry = new StreamingModeExecutionStateRegistry(); + StreamingEnginePipelineConfigManager configManager = + new StreamingEnginePipelineConfigManager(/*initializeWithDefaults=*/ true); StreamingModeExecutionContext context = new StreamingModeExecutionContext( counterSet, @@ -979,6 +984,7 @@ public void testFailedWorkItemsAbort() throws Exception { PipelineOptionsFactory.create(), "test-work-item-id"), executionStateRegistry, + configManager, Long.MAX_VALUE, /*throwExceptionOnLargeOutput=*/ false); @@ -1012,7 +1018,6 @@ public void testFailedWorkItemsAbort() throws Exception { dummyWork, mock(WindmillStateReader.class), mock(SideInputStateFetcher.class), - OperationalLimits.builder().build(), Windmill.WorkItemCommitRequest.newBuilder()); @SuppressWarnings({"unchecked", "rawtypes"}) diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingEngineComputationConfigFetcherTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingEngineComputationConfigFetcherTest.java index 59fd092adcba..7e676824658c 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingEngineComputationConfigFetcherTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingEngineComputationConfigFetcherTest.java @@ -34,7 +34,7 @@ import java.util.Set; import java.util.concurrent.CountDownLatch; import java.util.concurrent.Executors; -import java.util.function.Consumer; +import org.apache.beam.runners.dataflow.worker.OperationalLimits; import org.apache.beam.runners.dataflow.worker.WorkUnitClient; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; @@ -47,6 +47,7 @@ @RunWith(JUnit4.class) public class StreamingEngineComputationConfigFetcherTest { + private final WorkUnitClient mockDataflowServiceClient = mock(WorkUnitClient.class, new Returns(Optional.empty())); private StreamingEngineComputationConfigFetcher streamingEngineConfigFetcher; @@ -54,13 +55,13 @@ public class StreamingEngineComputationConfigFetcherTest { private StreamingEngineComputationConfigFetcher createConfigFetcher( boolean waitForInitialConfig, long globalConfigRefreshPeriod, - Consumer onPipelineConfig) { + StreamingEnginePipelineConfigManager configManager) { return StreamingEngineComputationConfigFetcher.forTesting( !waitForInitialConfig, globalConfigRefreshPeriod, mockDataflowServiceClient, ignored -> Executors.newSingleThreadScheduledExecutor(), - onPipelineConfig); + configManager); } @After @@ -78,28 +79,31 @@ public void testStart_requiresInitialConfig() throws IOException, InterruptedExc Set receivedPipelineConfig = new HashSet<>(); when(mockDataflowServiceClient.getGlobalStreamingConfigWorkItem()) .thenReturn(Optional.of(initialConfig)); + StreamingEnginePipelineConfigManager configManager = + new StreamingEnginePipelineConfigManager(/*initializeWithDefaults=*/ false); + configManager.onConfig( + config -> { + try { + receivedPipelineConfig.add(config); + waitForInitialConfig.await(); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + }); streamingEngineConfigFetcher = - createConfigFetcher( - /* waitForInitialConfig= */ true, - 0, - config -> { - try { - receivedPipelineConfig.add(config); - waitForInitialConfig.await(); - } catch (InterruptedException e) { - throw new RuntimeException(e); - } - }); + createConfigFetcher(/* waitForInitialConfig= */ true, 0, configManager); Thread asyncStartConfigLoader = new Thread(streamingEngineConfigFetcher::start); asyncStartConfigLoader.start(); waitForInitialConfig.countDown(); asyncStartConfigLoader.join(); - assertThat(receivedPipelineConfig) - .containsExactly( - StreamingEnginePipelineConfig.builder() - .setMaxWorkItemCommitBytes( - initialConfig.getStreamingConfigTask().getMaxWorkItemCommitBytes()) - .build()); + StreamingEnginePipelineConfig.Builder configBuilder = + StreamingEnginePipelineConfig.builder() + .setOperationalLimits( + OperationalLimits.builder() + .setMaxWorkItemCommitBytes( + initialConfig.getStreamingConfigTask().getMaxWorkItemCommitBytes()) + .build()); + assertThat(receivedPipelineConfig).containsExactly(configBuilder.build()); } @Test @@ -127,15 +131,16 @@ public void testStart_startsPeriodicConfigRequests() throws IOException, Interru // ConfigFetcher should not do anything with a config that doesn't contain a // StreamingConfigTask. .thenReturn(Optional.of(new WorkItem().setJobId("jobId"))); - + StreamingEnginePipelineConfigManager configManager = + new StreamingEnginePipelineConfigManager(/*initializeWithDefaults=*/ false); + configManager.onConfig( + config -> { + receivedPipelineConfig.add(config); + numExpectedRefreshes.countDown(); + }); streamingEngineConfigFetcher = createConfigFetcher( - /* waitForInitialConfig= */ true, - Duration.millis(100).getMillis(), - config -> { - receivedPipelineConfig.add(config); - numExpectedRefreshes.countDown(); - }); + /* waitForInitialConfig= */ true, Duration.millis(100).getMillis(), configManager); Thread asyncStartConfigLoader = new Thread(streamingEngineConfigFetcher::start); asyncStartConfigLoader.start(); @@ -144,23 +149,34 @@ public void testStart_startsPeriodicConfigRequests() throws IOException, Interru assertThat(receivedPipelineConfig) .containsExactly( StreamingEnginePipelineConfig.builder() - .setMaxWorkItemCommitBytes( - firstConfig.getStreamingConfigTask().getMaxWorkItemCommitBytes()) + .setOperationalLimits( + OperationalLimits.builder() + .setMaxWorkItemCommitBytes( + firstConfig.getStreamingConfigTask().getMaxWorkItemCommitBytes()) + .build()) .build(), StreamingEnginePipelineConfig.builder() - .setMaxWorkItemCommitBytes( - secondConfig.getStreamingConfigTask().getMaxWorkItemCommitBytes()) + .setOperationalLimits( + OperationalLimits.builder() + .setMaxWorkItemCommitBytes( + secondConfig.getStreamingConfigTask().getMaxWorkItemCommitBytes()) + .build()) .build(), StreamingEnginePipelineConfig.builder() - .setMaxWorkItemCommitBytes( - thirdConfig.getStreamingConfigTask().getMaxWorkItemCommitBytes()) + .setOperationalLimits( + OperationalLimits.builder() + .setMaxWorkItemCommitBytes( + thirdConfig.getStreamingConfigTask().getMaxWorkItemCommitBytes()) + .build()) .build()); } @Test public void testGetComputationConfig() throws IOException { + StreamingEnginePipelineConfigManager configManager = + new StreamingEnginePipelineConfigManager(/*initializeWithDefaults=*/ false); streamingEngineConfigFetcher = - createConfigFetcher(/* waitForInitialConfig= */ false, 0, ignored -> {}); + createConfigFetcher(/* waitForInitialConfig= */ false, 0, configManager); String computationId = "computationId"; String stageName = "stageName"; String systemName = "systemName"; @@ -194,8 +210,11 @@ public void testGetComputationConfig() throws IOException { @Test public void testGetComputationConfig_noComputationPresent() throws IOException { Set receivedPipelineConfig = new HashSet<>(); + StreamingEnginePipelineConfigManager configManager = + new StreamingEnginePipelineConfigManager(/*initializeWithDefaults=*/ false); + configManager.onConfig(receivedPipelineConfig::add); streamingEngineConfigFetcher = - createConfigFetcher(/* waitForInitialConfig= */ false, 0, receivedPipelineConfig::add); + createConfigFetcher(/* waitForInitialConfig= */ false, 0, configManager); when(mockDataflowServiceClient.getStreamingConfigWorkItem(anyString())) .thenReturn(Optional.empty()); Optional pipelineConfig = @@ -206,8 +225,10 @@ public void testGetComputationConfig_noComputationPresent() throws IOException { @Test public void testGetComputationConfig_fetchConfigFromDataflowError() throws IOException { + StreamingEnginePipelineConfigManager configManager = + new StreamingEnginePipelineConfigManager(/*initializeWithDefaults=*/ false); streamingEngineConfigFetcher = - createConfigFetcher(/* waitForInitialConfig= */ false, 0, ignored -> {}); + createConfigFetcher(/* waitForInitialConfig= */ false, 0, configManager); RuntimeException e = new RuntimeException("something bad happened."); when(mockDataflowServiceClient.getStreamingConfigWorkItem(anyString())).thenThrow(e); Throwable fetchConfigError = diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingEnginePipelineConfigManagerTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingEnginePipelineConfigManagerTest.java new file mode 100644 index 000000000000..78a6b289be53 --- /dev/null +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingEnginePipelineConfigManagerTest.java @@ -0,0 +1,154 @@ +/* + * 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.dataflow.worker.streaming.config; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Supplier; +import org.apache.beam.runners.dataflow.worker.OperationalLimits; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill.UserWorkerRunnerV1Settings; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.net.HostAndPort; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +@RunWith(JUnit4.class) +public class StreamingEnginePipelineConfigManagerTest { + + @Test + public void getConfig() { + StreamingEnginePipelineConfigManager configManager = + new StreamingEnginePipelineConfigManager(/*initializeWithDefaults=*/ false); + StreamingEnginePipelineConfig config = + StreamingEnginePipelineConfig.builder() + .setOperationalLimits( + OperationalLimits.builder() + .setMaxOutputValueBytes(123) + .setMaxOutputKeyBytes(324) + .setMaxWorkItemCommitBytes(456) + .build()) + .setWindmillServiceEndpoints(ImmutableSet.of(HostAndPort.fromHost("windmillHost"))) + .setUserWorkerJobSettings( + UserWorkerRunnerV1Settings.newBuilder() + .setUseSeparateWindmillHeartbeatStreams(false) + .build()) + .build(); + configManager.setConfig(config); + assertEquals(config, configManager.getConfig()); + } + + @Test + public void onConfig_configSetAfterRegisteringCallback() throws InterruptedException { + CountDownLatch latch = new CountDownLatch(1); + StreamingEnginePipelineConfigManager configManager = + new StreamingEnginePipelineConfigManager(/*initializeWithDefaults=*/ false); + StreamingEnginePipelineConfig configToSet = + StreamingEnginePipelineConfig.builder() + .setOperationalLimits( + OperationalLimits.builder() + .setMaxOutputValueBytes(123) + .setMaxOutputKeyBytes(324) + .setMaxWorkItemCommitBytes(456) + .build()) + .setWindmillServiceEndpoints(ImmutableSet.of(HostAndPort.fromHost("windmillHost"))) + .setUserWorkerJobSettings( + UserWorkerRunnerV1Settings.newBuilder() + .setUseSeparateWindmillHeartbeatStreams(false) + .build()) + .build(); + AtomicReference configFromCallback = new AtomicReference<>(); + configManager.onConfig( + config -> { + configFromCallback.set(config); + latch.countDown(); + }); + configManager.setConfig(configToSet); + assertTrue(latch.await(10, TimeUnit.SECONDS)); + assertEquals(configFromCallback.get(), configManager.getConfig()); + } + + @Test + public void onConfig_configSetBeforeRegisteringCallback() throws InterruptedException { + CountDownLatch latch = new CountDownLatch(1); + StreamingEnginePipelineConfigManager configManager = + new StreamingEnginePipelineConfigManager(/*initializeWithDefaults=*/ false); + StreamingEnginePipelineConfig configToSet = + StreamingEnginePipelineConfig.builder() + .setOperationalLimits( + OperationalLimits.builder() + .setMaxOutputValueBytes(123) + .setMaxOutputKeyBytes(324) + .setMaxWorkItemCommitBytes(456) + .build()) + .setWindmillServiceEndpoints(ImmutableSet.of(HostAndPort.fromHost("windmillHost"))) + .setUserWorkerJobSettings( + UserWorkerRunnerV1Settings.newBuilder() + .setUseSeparateWindmillHeartbeatStreams(false) + .build()) + .build(); + AtomicReference configFromCallback = new AtomicReference<>(); + configManager.setConfig(configToSet); + configManager.onConfig( + config -> { + configFromCallback.set(config); + latch.countDown(); + }); + assertTrue(latch.await(10, TimeUnit.SECONDS)); + assertEquals(configFromCallback.get(), configManager.getConfig()); + } + + @Test + public void onConfig_shouldNotCallCallbackForIfConfigRemainsSame() throws InterruptedException { + CountDownLatch latch = new CountDownLatch(1); + AtomicInteger callbackCount = new AtomicInteger(0); + StreamingEnginePipelineConfigManager configManager = + new StreamingEnginePipelineConfigManager(/*initializeWithDefaults=*/ false); + Supplier configToSet = + () -> { + return StreamingEnginePipelineConfig.builder() + .setOperationalLimits( + OperationalLimits.builder() + .setMaxOutputValueBytes(123) + .setMaxOutputKeyBytes(324) + .setMaxWorkItemCommitBytes(456) + .build()) + .setWindmillServiceEndpoints(ImmutableSet.of(HostAndPort.fromHost("windmillHost"))) + .setUserWorkerJobSettings( + UserWorkerRunnerV1Settings.newBuilder() + .setUseSeparateWindmillHeartbeatStreams(false) + .build()) + .build(); + }; + configManager.onConfig( + config -> { + callbackCount.incrementAndGet(); + latch.countDown(); + }); + configManager.setConfig(configToSet.get()); + // call setter again with same config + configManager.setConfig(configToSet.get()); + assertTrue(latch.await(10, TimeUnit.SECONDS)); + assertEquals(1, callbackCount.get()); + } +} diff --git a/runners/google-cloud-dataflow-java/worker/windmill/src/main/proto/windmill.proto b/runners/google-cloud-dataflow-java/worker/windmill/src/main/proto/windmill.proto index 4677ff9dcc9a..3b3348dbc3fa 100644 --- a/runners/google-cloud-dataflow-java/worker/windmill/src/main/proto/windmill.proto +++ b/runners/google-cloud-dataflow-java/worker/windmill/src/main/proto/windmill.proto @@ -923,6 +923,15 @@ message WorkerMetadataResponse { reserved 4; } +// Settings to control runtime behavior of the java runner v1 user worker. +message UserWorkerRunnerV1Settings { + // If true, use separate channels for each windmill RPC. + optional bool use_windmill_isolated_channels = 1 [default = true]; + + // If true, use separate streaming RPC for windmill heartbeats and state reads. + optional bool use_separate_windmill_heartbeat_streams = 2 [default = true]; +} + service WindmillAppliance { // Gets streaming Dataflow work. rpc GetWork(.windmill.GetWorkRequest) returns (.windmill.GetWorkResponse); From e7b705da1934776e3dd64bb5d5bc16c5fbc3fea1 Mon Sep 17 00:00:00 2001 From: Arun Pandian Date: Mon, 9 Sep 2024 08:54:12 -0700 Subject: [PATCH 02/13] address review comments --- .../worker/StreamingDataflowWorker.java | 10 ++- .../worker/StreamingModeExecutionContext.java | 9 +- .../FixedPipelineConfigManagerImpl.java | 44 ++++++++++ .../StreamingEnginePipelineConfigManager.java | 57 +------------ ...eamingEnginePipelineConfigManagerImpl.java | 83 +++++++++++++++++++ .../StreamingModeExecutionContextTest.java | 4 +- .../worker/WorkerCustomSourcesTest.java | 6 +- .../FixedPipelineConfigManagerImplTest.java | 52 ++++++++++++ ...ingEngineComputationConfigFetcherTest.java | 22 ++--- ...gEnginePipelineConfigManagerImplTest.java} | 48 +++++------ 10 files changed, 238 insertions(+), 97 deletions(-) create mode 100644 runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/FixedPipelineConfigManagerImpl.java create mode 100644 runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingEnginePipelineConfigManagerImpl.java create mode 100644 runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/config/FixedPipelineConfigManagerImplTest.java rename runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/config/{StreamingEnginePipelineConfigManagerTest.java => StreamingEnginePipelineConfigManagerImplTest.java} (80%) diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java index fd328a5657a9..aceedb95665d 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java @@ -47,10 +47,12 @@ import org.apache.beam.runners.dataflow.worker.streaming.StageInfo; import org.apache.beam.runners.dataflow.worker.streaming.WorkHeartbeatResponseProcessor; import org.apache.beam.runners.dataflow.worker.streaming.config.ComputationConfig; +import org.apache.beam.runners.dataflow.worker.streaming.config.FixedPipelineConfigManagerImpl; import org.apache.beam.runners.dataflow.worker.streaming.config.StreamingApplianceComputationConfigFetcher; import org.apache.beam.runners.dataflow.worker.streaming.config.StreamingEngineComputationConfigFetcher; import org.apache.beam.runners.dataflow.worker.streaming.config.StreamingEnginePipelineConfig; import org.apache.beam.runners.dataflow.worker.streaming.config.StreamingEnginePipelineConfigManager; +import org.apache.beam.runners.dataflow.worker.streaming.config.StreamingEnginePipelineConfigManagerImpl; import org.apache.beam.runners.dataflow.worker.streaming.harness.SingleSourceWorkerHarness; import org.apache.beam.runners.dataflow.worker.streaming.harness.SingleSourceWorkerHarness.GetWorkSender; import org.apache.beam.runners.dataflow.worker.streaming.harness.StreamingCounters; @@ -345,9 +347,11 @@ public static StreamingDataflowWorker fromOptions(DataflowWorkerHarnessOptions o GrpcWindmillStreamFactory.Builder windmillStreamFactoryBuilder = createGrpcwindmillStreamFactoryBuilder(options, clientId); StreamingEnginePipelineConfigManager configManager = - new StreamingEnginePipelineConfigManager( - /*initializeWithDefaults=*/ !options - .isEnableStreamingEngine()); // appliance is initialized with default settings + options.isEnableStreamingEngine() + ? new StreamingEnginePipelineConfigManagerImpl() + : new FixedPipelineConfigManagerImpl( + StreamingEnginePipelineConfig.builder() + .build()); // appliance is initialized with default settings ConfigFetcherComputationStateCacheAndWindmillClient configFetcherComputationStateCacheAndWindmillClient = diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContext.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContext.java index d08aad4df515..35426e3b0ab0 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContext.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContext.java @@ -132,6 +132,10 @@ public class StreamingModeExecutionContext extends DataflowExecutionContext callback) {} +} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingEnginePipelineConfigManager.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingEnginePipelineConfigManager.java index 7b9ae5b1eb40..3ab842ac2c7a 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingEnginePipelineConfigManager.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingEnginePipelineConfigManager.java @@ -17,10 +17,6 @@ */ package org.apache.beam.runners.dataflow.worker.streaming.config; -import com.google.common.base.Preconditions; -import java.util.Iterator; -import java.util.concurrent.CopyOnWriteArrayList; -import java.util.concurrent.atomic.AtomicReference; import java.util.function.Consumer; import javax.annotation.Nonnull; import javax.annotation.concurrent.ThreadSafe; @@ -28,61 +24,16 @@ @Internal @ThreadSafe -public class StreamingEnginePipelineConfigManager { - - private final AtomicReference streamingEngineConfig = - new AtomicReference<>(); - - private final CopyOnWriteArrayList> config_callbacks = - new CopyOnWriteArrayList<>(); - - public StreamingEnginePipelineConfigManager(boolean initializeWithDefaults) { - if (initializeWithDefaults) { - streamingEngineConfig.set(StreamingEnginePipelineConfig.builder().build()); - } - } +public interface StreamingEnginePipelineConfigManager { /* * Returns the latest StreamingEnginePipelineConfig */ - public StreamingEnginePipelineConfig getConfig() { - Preconditions.checkState( - streamingEngineConfig.get() != null, - "Global config should be set before any processing is done"); - return streamingEngineConfig.get(); - } + StreamingEnginePipelineConfig getConfig(); /* * Subscribe to config updates by registering a callback. - * Callback will be called the first time with settings, if any, inline before the method returns. - */ - public void onConfig(@Nonnull Consumer callback) { - StreamingEnginePipelineConfig config; - synchronized (this) { - config_callbacks.add(callback); - config = streamingEngineConfig.get(); - } - if (config != null) { - callback.accept(config); - } - } - - /* - * Package private setter for setting config + * Callback should be called the first time with settings, if any, inline before the method returns. */ - void setConfig(@Nonnull StreamingEnginePipelineConfig config) { - Iterator> iterator; - synchronized (this) { - if (config.equals(streamingEngineConfig.get())) { - return; - } - streamingEngineConfig.set(config); - // iterator of CopyOnWriteArrayList provides - // snapshot semantics - iterator = config_callbacks.iterator(); - } - while (iterator.hasNext()) { - iterator.next().accept(config); - } - } + void onConfig(@Nonnull Consumer callback); } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingEnginePipelineConfigManagerImpl.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingEnginePipelineConfigManagerImpl.java new file mode 100644 index 000000000000..1a8228e1c74e --- /dev/null +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingEnginePipelineConfigManagerImpl.java @@ -0,0 +1,83 @@ +/* + * 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.dataflow.worker.streaming.config; + +import com.google.common.base.Preconditions; +import java.util.Iterator; +import java.util.concurrent.CopyOnWriteArrayList; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Consumer; +import javax.annotation.Nonnull; +import javax.annotation.concurrent.ThreadSafe; +import org.apache.beam.sdk.annotations.Internal; + +@Internal +@ThreadSafe +public class StreamingEnginePipelineConfigManagerImpl + implements StreamingEnginePipelineConfigManager { + + private final AtomicReference streamingEngineConfig = + new AtomicReference<>(); + + private final CopyOnWriteArrayList> config_callbacks = + new CopyOnWriteArrayList<>(); + + /* + * Returns the latest StreamingEnginePipelineConfig + */ + public StreamingEnginePipelineConfig getConfig() { + Preconditions.checkState( + streamingEngineConfig.get() != null, + "Global config should be set before any processing is done"); + return streamingEngineConfig.get(); + } + + /* + * Subscribe to config updates by registering a callback. + * Callback will be called the first time with settings, if any, inline before the method returns. + */ + public void onConfig(@Nonnull Consumer callback) { + StreamingEnginePipelineConfig config; + synchronized (this) { + config_callbacks.add(callback); + config = streamingEngineConfig.get(); + } + if (config != null) { + callback.accept(config); + } + } + + /* + * Package private setter for setting config + */ + void setConfig(@Nonnull StreamingEnginePipelineConfig config) { + Iterator> iterator; + synchronized (this) { + if (config.equals(streamingEngineConfig.get())) { + return; + } + streamingEngineConfig.set(config); + // iterator of CopyOnWriteArrayList provides + // snapshot semantics + iterator = config_callbacks.iterator(); + } + while (iterator.hasNext()) { + iterator.next().accept(config); + } + } +} diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContextTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContextTest.java index 472500cb9b20..3263a2c942a0 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContextTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContextTest.java @@ -59,6 +59,8 @@ import org.apache.beam.runners.dataflow.worker.profiler.ScopedProfiler.ProfileScope; import org.apache.beam.runners.dataflow.worker.streaming.Watermarks; import org.apache.beam.runners.dataflow.worker.streaming.Work; +import org.apache.beam.runners.dataflow.worker.streaming.config.FixedPipelineConfigManagerImpl; +import org.apache.beam.runners.dataflow.worker.streaming.config.StreamingEnginePipelineConfig; import org.apache.beam.runners.dataflow.worker.streaming.config.StreamingEnginePipelineConfigManager; import org.apache.beam.runners.dataflow.worker.streaming.sideinput.SideInputStateFetcher; import org.apache.beam.runners.dataflow.worker.windmill.Windmill; @@ -109,7 +111,7 @@ public void setUp() { CounterSet counterSet = new CounterSet(); ConcurrentHashMap stateNameMap = new ConcurrentHashMap<>(); StreamingEnginePipelineConfigManager configManager = - new StreamingEnginePipelineConfigManager(/*initializeWithDefaults=*/ true); + new FixedPipelineConfigManagerImpl(StreamingEnginePipelineConfig.builder().build()); stateNameMap.put(NameContextsForTests.nameContextForTest().userName(), "testStateFamily"); executionContext = new StreamingModeExecutionContext( diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSourcesTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSourcesTest.java index f75b9bb504b9..82e209605e3c 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSourcesTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSourcesTest.java @@ -90,6 +90,8 @@ import org.apache.beam.runners.dataflow.worker.profiler.ScopedProfiler.NoopProfileScope; import org.apache.beam.runners.dataflow.worker.streaming.Watermarks; import org.apache.beam.runners.dataflow.worker.streaming.Work; +import org.apache.beam.runners.dataflow.worker.streaming.config.FixedPipelineConfigManagerImpl; +import org.apache.beam.runners.dataflow.worker.streaming.config.StreamingEnginePipelineConfig; import org.apache.beam.runners.dataflow.worker.streaming.config.StreamingEnginePipelineConfigManager; import org.apache.beam.runners.dataflow.worker.streaming.sideinput.SideInputStateFetcher; import org.apache.beam.runners.dataflow.worker.testing.TestCountingSource; @@ -596,7 +598,7 @@ public void testReadUnboundedReader() throws Exception { new StreamingModeExecutionStateRegistry(); ReaderCache readerCache = new ReaderCache(Duration.standardMinutes(1), Runnable::run); StreamingEnginePipelineConfigManager configManager = - new StreamingEnginePipelineConfigManager(/*initializeWithDefaults=*/ true); + new FixedPipelineConfigManagerImpl(StreamingEnginePipelineConfig.builder().build()); StreamingModeExecutionContext context = new StreamingModeExecutionContext( counterSet, @@ -964,7 +966,7 @@ public void testFailedWorkItemsAbort() throws Exception { StreamingModeExecutionStateRegistry executionStateRegistry = new StreamingModeExecutionStateRegistry(); StreamingEnginePipelineConfigManager configManager = - new StreamingEnginePipelineConfigManager(/*initializeWithDefaults=*/ true); + new FixedPipelineConfigManagerImpl(StreamingEnginePipelineConfig.builder().build()); StreamingModeExecutionContext context = new StreamingModeExecutionContext( counterSet, diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/config/FixedPipelineConfigManagerImplTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/config/FixedPipelineConfigManagerImplTest.java new file mode 100644 index 000000000000..13dd635da182 --- /dev/null +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/config/FixedPipelineConfigManagerImplTest.java @@ -0,0 +1,52 @@ +/* + * 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.dataflow.worker.streaming.config; + +import static org.junit.Assert.assertEquals; + +import org.apache.beam.runners.dataflow.worker.OperationalLimits; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill.UserWorkerRunnerV1Settings; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.net.HostAndPort; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +@RunWith(JUnit4.class) +public class FixedPipelineConfigManagerImplTest { + + @Test + public void getConfig() { + StreamingEnginePipelineConfig config = + StreamingEnginePipelineConfig.builder() + .setOperationalLimits( + OperationalLimits.builder() + .setMaxOutputValueBytes(123) + .setMaxOutputKeyBytes(324) + .setMaxWorkItemCommitBytes(456) + .build()) + .setWindmillServiceEndpoints(ImmutableSet.of(HostAndPort.fromHost("windmillHost"))) + .setUserWorkerJobSettings( + UserWorkerRunnerV1Settings.newBuilder() + .setUseSeparateWindmillHeartbeatStreams(false) + .build()) + .build(); + FixedPipelineConfigManagerImpl configManager = new FixedPipelineConfigManagerImpl(config); + assertEquals(config, configManager.getConfig()); + } +} diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingEngineComputationConfigFetcherTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingEngineComputationConfigFetcherTest.java index 7e676824658c..01df8f6a95c1 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingEngineComputationConfigFetcherTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingEngineComputationConfigFetcherTest.java @@ -55,7 +55,7 @@ public class StreamingEngineComputationConfigFetcherTest { private StreamingEngineComputationConfigFetcher createConfigFetcher( boolean waitForInitialConfig, long globalConfigRefreshPeriod, - StreamingEnginePipelineConfigManager configManager) { + StreamingEnginePipelineConfigManagerImpl configManager) { return StreamingEngineComputationConfigFetcher.forTesting( !waitForInitialConfig, globalConfigRefreshPeriod, @@ -79,8 +79,8 @@ public void testStart_requiresInitialConfig() throws IOException, InterruptedExc Set receivedPipelineConfig = new HashSet<>(); when(mockDataflowServiceClient.getGlobalStreamingConfigWorkItem()) .thenReturn(Optional.of(initialConfig)); - StreamingEnginePipelineConfigManager configManager = - new StreamingEnginePipelineConfigManager(/*initializeWithDefaults=*/ false); + StreamingEnginePipelineConfigManagerImpl configManager = + new StreamingEnginePipelineConfigManagerImpl(); configManager.onConfig( config -> { try { @@ -131,8 +131,8 @@ public void testStart_startsPeriodicConfigRequests() throws IOException, Interru // ConfigFetcher should not do anything with a config that doesn't contain a // StreamingConfigTask. .thenReturn(Optional.of(new WorkItem().setJobId("jobId"))); - StreamingEnginePipelineConfigManager configManager = - new StreamingEnginePipelineConfigManager(/*initializeWithDefaults=*/ false); + StreamingEnginePipelineConfigManagerImpl configManager = + new StreamingEnginePipelineConfigManagerImpl(); configManager.onConfig( config -> { receivedPipelineConfig.add(config); @@ -173,8 +173,8 @@ public void testStart_startsPeriodicConfigRequests() throws IOException, Interru @Test public void testGetComputationConfig() throws IOException { - StreamingEnginePipelineConfigManager configManager = - new StreamingEnginePipelineConfigManager(/*initializeWithDefaults=*/ false); + StreamingEnginePipelineConfigManagerImpl configManager = + new StreamingEnginePipelineConfigManagerImpl(); streamingEngineConfigFetcher = createConfigFetcher(/* waitForInitialConfig= */ false, 0, configManager); String computationId = "computationId"; @@ -210,8 +210,8 @@ public void testGetComputationConfig() throws IOException { @Test public void testGetComputationConfig_noComputationPresent() throws IOException { Set receivedPipelineConfig = new HashSet<>(); - StreamingEnginePipelineConfigManager configManager = - new StreamingEnginePipelineConfigManager(/*initializeWithDefaults=*/ false); + StreamingEnginePipelineConfigManagerImpl configManager = + new StreamingEnginePipelineConfigManagerImpl(); configManager.onConfig(receivedPipelineConfig::add); streamingEngineConfigFetcher = createConfigFetcher(/* waitForInitialConfig= */ false, 0, configManager); @@ -225,8 +225,8 @@ public void testGetComputationConfig_noComputationPresent() throws IOException { @Test public void testGetComputationConfig_fetchConfigFromDataflowError() throws IOException { - StreamingEnginePipelineConfigManager configManager = - new StreamingEnginePipelineConfigManager(/*initializeWithDefaults=*/ false); + StreamingEnginePipelineConfigManagerImpl configManager = + new StreamingEnginePipelineConfigManagerImpl(); streamingEngineConfigFetcher = createConfigFetcher(/* waitForInitialConfig= */ false, 0, configManager); RuntimeException e = new RuntimeException("something bad happened."); diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingEnginePipelineConfigManagerTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingEnginePipelineConfigManagerImplTest.java similarity index 80% rename from runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingEnginePipelineConfigManagerTest.java rename to runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingEnginePipelineConfigManagerImplTest.java index 78a6b289be53..fe99331eeda0 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingEnginePipelineConfigManagerTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingEnginePipelineConfigManagerImplTest.java @@ -34,12 +34,11 @@ import org.junit.runners.JUnit4; @RunWith(JUnit4.class) -public class StreamingEnginePipelineConfigManagerTest { - +public class StreamingEnginePipelineConfigManagerImplTest { @Test public void getConfig() { - StreamingEnginePipelineConfigManager configManager = - new StreamingEnginePipelineConfigManager(/*initializeWithDefaults=*/ false); + StreamingEnginePipelineConfigManagerImpl configManager = + new StreamingEnginePipelineConfigManagerImpl(); StreamingEnginePipelineConfig config = StreamingEnginePipelineConfig.builder() .setOperationalLimits( @@ -61,8 +60,8 @@ public void getConfig() { @Test public void onConfig_configSetAfterRegisteringCallback() throws InterruptedException { CountDownLatch latch = new CountDownLatch(1); - StreamingEnginePipelineConfigManager configManager = - new StreamingEnginePipelineConfigManager(/*initializeWithDefaults=*/ false); + StreamingEnginePipelineConfigManagerImpl configManager = + new StreamingEnginePipelineConfigManagerImpl(); StreamingEnginePipelineConfig configToSet = StreamingEnginePipelineConfig.builder() .setOperationalLimits( @@ -91,8 +90,8 @@ public void onConfig_configSetAfterRegisteringCallback() throws InterruptedExcep @Test public void onConfig_configSetBeforeRegisteringCallback() throws InterruptedException { CountDownLatch latch = new CountDownLatch(1); - StreamingEnginePipelineConfigManager configManager = - new StreamingEnginePipelineConfigManager(/*initializeWithDefaults=*/ false); + StreamingEnginePipelineConfigManagerImpl configManager = + new StreamingEnginePipelineConfigManagerImpl(); StreamingEnginePipelineConfig configToSet = StreamingEnginePipelineConfig.builder() .setOperationalLimits( @@ -122,24 +121,23 @@ public void onConfig_configSetBeforeRegisteringCallback() throws InterruptedExce public void onConfig_shouldNotCallCallbackForIfConfigRemainsSame() throws InterruptedException { CountDownLatch latch = new CountDownLatch(1); AtomicInteger callbackCount = new AtomicInteger(0); - StreamingEnginePipelineConfigManager configManager = - new StreamingEnginePipelineConfigManager(/*initializeWithDefaults=*/ false); + StreamingEnginePipelineConfigManagerImpl configManager = + new StreamingEnginePipelineConfigManagerImpl(); Supplier configToSet = - () -> { - return StreamingEnginePipelineConfig.builder() - .setOperationalLimits( - OperationalLimits.builder() - .setMaxOutputValueBytes(123) - .setMaxOutputKeyBytes(324) - .setMaxWorkItemCommitBytes(456) - .build()) - .setWindmillServiceEndpoints(ImmutableSet.of(HostAndPort.fromHost("windmillHost"))) - .setUserWorkerJobSettings( - UserWorkerRunnerV1Settings.newBuilder() - .setUseSeparateWindmillHeartbeatStreams(false) - .build()) - .build(); - }; + () -> + StreamingEnginePipelineConfig.builder() + .setOperationalLimits( + OperationalLimits.builder() + .setMaxOutputValueBytes(123) + .setMaxOutputKeyBytes(324) + .setMaxWorkItemCommitBytes(456) + .build()) + .setWindmillServiceEndpoints(ImmutableSet.of(HostAndPort.fromHost("windmillHost"))) + .setUserWorkerJobSettings( + UserWorkerRunnerV1Settings.newBuilder() + .setUseSeparateWindmillHeartbeatStreams(false) + .build()) + .build(); configManager.onConfig( config -> { callbackCount.incrementAndGet(); From a8bb92f2206f65e1cca7ab8334f3a576364a53c6 Mon Sep 17 00:00:00 2001 From: Arun Pandian Date: Mon, 9 Sep 2024 12:41:49 -0700 Subject: [PATCH 03/13] fix class hierarchy --- .../worker/StreamingDataflowWorker.java | 50 +++++++---------- .../worker/StreamingModeExecutionContext.java | 6 +- .../streaming/config/ComputationConfig.java | 3 +- ...Impl.java => FixedGlobalConfigHandle.java} | 14 +++-- ...mingApplianceComputationConfigFetcher.java | 10 +++- ...reamingEngineComputationConfigFetcher.java | 41 +++++++------- ...Config.java => StreamingGlobalConfig.java} | 8 +-- ....java => StreamingGlobalConfigHandle.java} | 6 +- ...a => StreamingGlobalConfigHandleImpl.java} | 19 ++++--- .../harness/StreamingWorkerStatusPages.java | 12 ++-- .../client/grpc/GrpcDispatcherClient.java | 4 +- .../ComputationWorkExecutorFactory.java | 6 +- .../processing/StreamingWorkScheduler.java | 8 +-- .../worker/StreamingDataflowWorkerTest.java | 27 +++++---- .../StreamingModeExecutionContextTest.java | 10 ++-- .../worker/WorkerCustomSourcesTest.java | 14 ++--- ....java => FixedGlobalConfigHandleTest.java} | 40 +++++++++++-- ...ApplianceComputationConfigFetcherTest.java | 4 +- ...ingEngineComputationConfigFetcherTest.java | 53 ++++++++---------- ... StreamingGlobalConfigHandleImplTest.java} | 56 +++++++++---------- 20 files changed, 211 insertions(+), 180 deletions(-) rename runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/{FixedPipelineConfigManagerImpl.java => FixedGlobalConfigHandle.java} (77%) rename runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/{StreamingEnginePipelineConfig.java => StreamingGlobalConfig.java} (89%) rename runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/{StreamingEnginePipelineConfigManager.java => StreamingGlobalConfigHandle.java} (88%) rename runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/{StreamingEnginePipelineConfigManagerImpl.java => StreamingGlobalConfigHandleImpl.java} (79%) rename runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/config/{FixedPipelineConfigManagerImplTest.java => FixedGlobalConfigHandleTest.java} (56%) rename runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/config/{StreamingEnginePipelineConfigManagerImplTest.java => StreamingGlobalConfigHandleImplTest.java} (75%) diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java index aceedb95665d..092e43bf3a6f 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java @@ -47,12 +47,11 @@ import org.apache.beam.runners.dataflow.worker.streaming.StageInfo; import org.apache.beam.runners.dataflow.worker.streaming.WorkHeartbeatResponseProcessor; import org.apache.beam.runners.dataflow.worker.streaming.config.ComputationConfig; -import org.apache.beam.runners.dataflow.worker.streaming.config.FixedPipelineConfigManagerImpl; +import org.apache.beam.runners.dataflow.worker.streaming.config.FixedGlobalConfigHandle; import org.apache.beam.runners.dataflow.worker.streaming.config.StreamingApplianceComputationConfigFetcher; import org.apache.beam.runners.dataflow.worker.streaming.config.StreamingEngineComputationConfigFetcher; -import org.apache.beam.runners.dataflow.worker.streaming.config.StreamingEnginePipelineConfig; -import org.apache.beam.runners.dataflow.worker.streaming.config.StreamingEnginePipelineConfigManager; -import org.apache.beam.runners.dataflow.worker.streaming.config.StreamingEnginePipelineConfigManagerImpl; +import org.apache.beam.runners.dataflow.worker.streaming.config.StreamingGlobalConfig; +import org.apache.beam.runners.dataflow.worker.streaming.config.StreamingGlobalConfigHandleImpl; import org.apache.beam.runners.dataflow.worker.streaming.harness.SingleSourceWorkerHarness; import org.apache.beam.runners.dataflow.worker.streaming.harness.SingleSourceWorkerHarness.GetWorkSender; import org.apache.beam.runners.dataflow.worker.streaming.harness.StreamingCounters; @@ -180,7 +179,6 @@ private StreamingDataflowWorker( WorkFailureProcessor workFailureProcessor, StreamingCounters streamingCounters, MemoryMonitor memoryMonitor, - StreamingEnginePipelineConfigManager configManager, GrpcWindmillStreamFactory windmillStreamFactory, Function executorSupplier, ConcurrentMap stageInfoMap) { @@ -237,7 +235,7 @@ private StreamingDataflowWorker( hotKeyLogger, sampler, ID_GENERATOR, - configManager, + configFetcher.getGlobalConfigHandle(), stageInfoMap); ThrottlingGetDataMetricTracker getDataMetricTracker = @@ -297,7 +295,7 @@ private StreamingDataflowWorker( .setCurrentActiveCommitBytes(workCommitter::currentActiveCommitBytes) .setGetDataStatusProvider(getDataClient::printHtml) .setWorkUnitExecutor(workUnitExecutor) - .setConfigManager(configManager) + .setglobalConfigHandle(configFetcher.getGlobalConfigHandle()) .build(); Windmill.GetWorkRequest request = @@ -346,12 +344,6 @@ public static StreamingDataflowWorker fromOptions(DataflowWorkerHarnessOptions o new ThreadFactoryBuilder().setNameFormat(threadName).build()); GrpcWindmillStreamFactory.Builder windmillStreamFactoryBuilder = createGrpcwindmillStreamFactoryBuilder(options, clientId); - StreamingEnginePipelineConfigManager configManager = - options.isEnableStreamingEngine() - ? new StreamingEnginePipelineConfigManagerImpl() - : new FixedPipelineConfigManagerImpl( - StreamingEnginePipelineConfig.builder() - .build()); // appliance is initialized with default settings ConfigFetcherComputationStateCacheAndWindmillClient configFetcherComputationStateCacheAndWindmillClient = @@ -359,7 +351,6 @@ public static StreamingDataflowWorker fromOptions(DataflowWorkerHarnessOptions o options, dataflowServiceClient, windmillStreamFactoryBuilder, - configManager, configFetcher -> ComputationStateCache.create( configFetcher, @@ -416,7 +407,6 @@ public static StreamingDataflowWorker fromOptions(DataflowWorkerHarnessOptions o workFailureProcessor, streamingCounters, memoryMonitor, - configManager, configFetcherComputationStateCacheAndWindmillClient.windmillStreamFactory(), executorSupplier, stageInfo); @@ -433,7 +423,6 @@ public static StreamingDataflowWorker fromOptions(DataflowWorkerHarnessOptions o DataflowWorkerHarnessOptions options, WorkUnitClient dataflowServiceClient, GrpcWindmillStreamFactory.Builder windmillStreamFactoryBuilder, - StreamingEnginePipelineConfigManager configManager, Function computationStateCacheFactory) { ComputationConfig.Fetcher configFetcher; WindmillServerStub windmillServer; @@ -442,12 +431,10 @@ public static StreamingDataflowWorker fromOptions(DataflowWorkerHarnessOptions o if (options.isEnableStreamingEngine()) { GrpcDispatcherClient dispatcherClient = GrpcDispatcherClient.create(createStubFactory(options)); - configManager.onConfig(dispatcherClient::onJobConfig); configFetcher = StreamingEngineComputationConfigFetcher.create( - options.getGlobalConfigRefreshPeriod().getMillis(), - dataflowServiceClient, - configManager); + options.getGlobalConfigRefreshPeriod().getMillis(), dataflowServiceClient); + configFetcher.getGlobalConfigHandle().onConfig(dispatcherClient::onJobConfig); computationStateCache = computationStateCacheFactory.apply(configFetcher); windmillStreamFactory = windmillStreamFactoryBuilder @@ -475,7 +462,10 @@ public static StreamingDataflowWorker fromOptions(DataflowWorkerHarnessOptions o windmillServer = new JniWindmillApplianceServer(options.getLocalWindmillHostport()); } - configFetcher = new StreamingApplianceComputationConfigFetcher(windmillServer::getConfig); + configFetcher = + new StreamingApplianceComputationConfigFetcher( + windmillServer::getConfig, + new FixedGlobalConfigHandle(StreamingGlobalConfig.builder().build())); computationStateCache = computationStateCacheFactory.apply(configFetcher); } @@ -495,7 +485,7 @@ static StreamingDataflowWorker forTesting( HotKeyLogger hotKeyLogger, Supplier clock, Function executorSupplier, - StreamingEnginePipelineConfigManager configManager, + StreamingGlobalConfigHandleImpl globalConfigHandle, int localRetryTimeoutMs) { ConcurrentMap stageInfo = new ConcurrentHashMap<>(); BoundedQueueExecutor workExecutor = createWorkUnitExecutor(options); @@ -504,19 +494,20 @@ static StreamingDataflowWorker forTesting( .setSizeMb(options.getWorkerCacheMb()) .setSupportMapViaMultimap(options.isEnableStreamingEngine()) .build(); - StreamingEnginePipelineConfig config = configManager.getConfig(); - if (!config.windmillServiceEndpoints().isEmpty()) { - windmillServer.setWindmillServiceEndpoints(config.windmillServiceEndpoints()); - } ComputationConfig.Fetcher configFetcher = options.isEnableStreamingEngine() ? StreamingEngineComputationConfigFetcher.forTesting( /* hasReceivedGlobalConfig= */ true, options.getGlobalConfigRefreshPeriod().getMillis(), workUnitClient, - executorSupplier, - configManager) - : new StreamingApplianceComputationConfigFetcher(windmillServer::getConfig); + globalConfigHandle, + executorSupplier) + : new StreamingApplianceComputationConfigFetcher( + windmillServer::getConfig, globalConfigHandle); + StreamingGlobalConfig config = configFetcher.getGlobalConfigHandle().getConfig(); + if (!config.windmillServiceEndpoints().isEmpty()) { + windmillServer.setWindmillServiceEndpoints(config.windmillServiceEndpoints()); + } ConcurrentMap stateNameMap = new ConcurrentHashMap<>(prePopulatedStateNameMappings); ComputationStateCache computationStateCache = @@ -583,7 +574,6 @@ static StreamingDataflowWorker forTesting( workFailureProcessor, streamingCounters, memoryMonitor, - configManager, options.isEnableStreamingEngine() ? windmillStreamFactory .setHealthCheckIntervalMillis( diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContext.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContext.java index 35426e3b0ab0..85af41577358 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContext.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContext.java @@ -50,7 +50,7 @@ import org.apache.beam.runners.dataflow.worker.profiler.ScopedProfiler.ProfileScope; import org.apache.beam.runners.dataflow.worker.streaming.Watermarks; import org.apache.beam.runners.dataflow.worker.streaming.Work; -import org.apache.beam.runners.dataflow.worker.streaming.config.StreamingEnginePipelineConfigManager; +import org.apache.beam.runners.dataflow.worker.streaming.config.StreamingGlobalConfigHandle; import org.apache.beam.runners.dataflow.worker.streaming.sideinput.SideInput; import org.apache.beam.runners.dataflow.worker.streaming.sideinput.SideInputState; import org.apache.beam.runners.dataflow.worker.streaming.sideinput.SideInputStateFetcher; @@ -108,7 +108,7 @@ public class StreamingModeExecutionContext extends DataflowExecutionContext stateNameMap; private final WindmillStateCache.ForComputation stateCache; private final ReaderCache readerCache; - private final StreamingEnginePipelineConfigManager configManager; + private final StreamingGlobalConfigHandle configManager; private final boolean throwExceptionOnLargeOutput; private volatile long backlogBytes; @@ -155,7 +155,7 @@ public StreamingModeExecutionContext( MetricsContainerRegistry metricsContainerRegistry, DataflowExecutionStateTracker executionStateTracker, StreamingModeExecutionStateRegistry executionStateRegistry, - StreamingEnginePipelineConfigManager configManager, + StreamingGlobalConfigHandle configManager, long sinkByteLimit, boolean throwExceptionOnLargeOutput) { super( diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/ComputationConfig.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/ComputationConfig.java index fb8bcf7edbfb..9702751aeb98 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/ComputationConfig.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/ComputationConfig.java @@ -48,12 +48,13 @@ public static ComputationConfig create( public abstract ImmutableMap stateNameMap(); /** Interface to fetch configurations for a specific computation. */ - @FunctionalInterface public interface Fetcher { default void start() {} default void stop() {} Optional fetchConfig(String computationId); + + StreamingGlobalConfigHandle getGlobalConfigHandle(); } } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/FixedPipelineConfigManagerImpl.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/FixedGlobalConfigHandle.java similarity index 77% rename from runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/FixedPipelineConfigManagerImpl.java rename to runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/FixedGlobalConfigHandle.java index a2026c5d1632..e0485ae77b71 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/FixedPipelineConfigManagerImpl.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/FixedGlobalConfigHandle.java @@ -28,17 +28,21 @@ * StreamingEnginePipelineConfigManager returning a fixed config * initialized during construction. Used for Appliance and Tests. */ -public class FixedPipelineConfigManagerImpl implements StreamingEnginePipelineConfigManager { +public class FixedGlobalConfigHandle implements StreamingGlobalConfigHandle { - private final StreamingEnginePipelineConfig config; + private final StreamingGlobalConfig config; - public FixedPipelineConfigManagerImpl(StreamingEnginePipelineConfig config) { + public FixedGlobalConfigHandle(StreamingGlobalConfig config) { this.config = config; } - public StreamingEnginePipelineConfig getConfig() { + @Override + public StreamingGlobalConfig getConfig() { return config; } - public void onConfig(@Nonnull Consumer callback) {} + @Override + public void onConfig(@Nonnull Consumer callback) { + callback.accept(config); + } } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingApplianceComputationConfigFetcher.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingApplianceComputationConfigFetcher.java index 786ded09498a..025e66be79c1 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingApplianceComputationConfigFetcher.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingApplianceComputationConfigFetcher.java @@ -48,11 +48,14 @@ public final class StreamingApplianceComputationConfigFetcher implements Computa private final ApplianceComputationConfigFetcher applianceComputationConfigFetcher; private final ConcurrentHashMap systemNameToComputationIdMap; + private final StreamingGlobalConfigHandle globalConfigHandle; public StreamingApplianceComputationConfigFetcher( - ApplianceComputationConfigFetcher applianceComputationConfigFetcher) { + ApplianceComputationConfigFetcher applianceComputationConfigFetcher, + StreamingGlobalConfigHandle globalConfigHandle) { this.applianceComputationConfigFetcher = applianceComputationConfigFetcher; this.systemNameToComputationIdMap = new ConcurrentHashMap<>(); + this.globalConfigHandle = globalConfigHandle; } /** Returns a {@code Table} */ @@ -112,6 +115,11 @@ public Optional fetchConfig(String computationId) { .collect(toImmutableMap(NameMapEntry::getUserName, NameMapEntry::getSystemName))); } + @Override + public StreamingGlobalConfigHandle getGlobalConfigHandle() { + return globalConfigHandle; + } + private Optional createComputationConfig( String serializedMapTask, Table transformUserNameToStateFamilyByComputationId, diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingEngineComputationConfigFetcher.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingEngineComputationConfigFetcher.java index d9377a258dc7..22b0dac6eb22 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingEngineComputationConfigFetcher.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingEngineComputationConfigFetcher.java @@ -74,33 +74,31 @@ public final class StreamingEngineComputationConfigFetcher implements Computatio private final long globalConfigRefreshPeriodMillis; private final WorkUnitClient dataflowServiceClient; private final ScheduledExecutorService globalConfigRefresher; - private final StreamingEnginePipelineConfigManager configManager; + private final StreamingGlobalConfigHandleImpl globalConfigHandle; private final AtomicBoolean hasReceivedGlobalConfig; private StreamingEngineComputationConfigFetcher( boolean hasReceivedGlobalConfig, long globalConfigRefreshPeriodMillis, WorkUnitClient dataflowServiceClient, - ScheduledExecutorService globalConfigRefresher, - StreamingEnginePipelineConfigManager configManager) { + StreamingGlobalConfigHandleImpl globalConfigHandle, + ScheduledExecutorService globalConfigRefresher) { this.globalConfigRefreshPeriodMillis = globalConfigRefreshPeriodMillis; this.dataflowServiceClient = dataflowServiceClient; this.globalConfigRefresher = globalConfigRefresher; - this.configManager = configManager; + this.globalConfigHandle = globalConfigHandle; this.hasReceivedGlobalConfig = new AtomicBoolean(hasReceivedGlobalConfig); } public static StreamingEngineComputationConfigFetcher create( - long globalConfigRefreshPeriodMillis, - WorkUnitClient dataflowServiceClient, - StreamingEnginePipelineConfigManager configManager) { + long globalConfigRefreshPeriodMillis, WorkUnitClient dataflowServiceClient) { return new StreamingEngineComputationConfigFetcher( /* hasReceivedGlobalConfig= */ false, globalConfigRefreshPeriodMillis, dataflowServiceClient, + new StreamingGlobalConfigHandleImpl(), Executors.newSingleThreadScheduledExecutor( - new ThreadFactoryBuilder().setNameFormat(CONFIG_REFRESHER_THREAD_NAME).build()), - configManager); + new ThreadFactoryBuilder().setNameFormat(CONFIG_REFRESHER_THREAD_NAME).build())); } @VisibleForTesting @@ -108,14 +106,14 @@ public static StreamingEngineComputationConfigFetcher forTesting( boolean hasReceivedGlobalConfig, long globalConfigRefreshPeriodMillis, WorkUnitClient dataflowServiceClient, - Function executorSupplier, - StreamingEnginePipelineConfigManager configManager) { + StreamingGlobalConfigHandleImpl globalConfigHandle, + Function executorSupplier) { return new StreamingEngineComputationConfigFetcher( hasReceivedGlobalConfig, globalConfigRefreshPeriodMillis, dataflowServiceClient, - executorSupplier.apply(CONFIG_REFRESHER_THREAD_NAME), - configManager); + globalConfigHandle, + executorSupplier.apply(CONFIG_REFRESHER_THREAD_NAME)); } @VisibleForTesting @@ -159,8 +157,8 @@ private static Optional fetchConfigWithRetry( } } - private StreamingEnginePipelineConfig createPipelineConfig(StreamingConfigTask config) { - StreamingEnginePipelineConfig.Builder pipelineConfig = StreamingEnginePipelineConfig.builder(); + private StreamingGlobalConfig createPipelineConfig(StreamingConfigTask config) { + StreamingGlobalConfig.Builder pipelineConfig = StreamingGlobalConfig.builder(); OperationalLimits.Builder operationalLimits = OperationalLimits.builder(); if (config.getWindmillServiceEndpoint() != null @@ -246,6 +244,11 @@ public Optional fetchConfig(String computationId) { .flatMap(StreamingEngineComputationConfigFetcher::createComputationConfig); } + @Override + public StreamingGlobalConfigHandle getGlobalConfigHandle() { + return globalConfigHandle; + } + @Override public void stop() { // We have already shutdown or start has not been called. @@ -272,7 +275,7 @@ public void stop() { @SuppressWarnings("FutureReturnValueIgnored") private void schedulePeriodicGlobalConfigRequests() { globalConfigRefresher.scheduleWithFixedDelay( - () -> fetchGlobalConfig().ifPresent(configManager::setConfig), + () -> fetchGlobalConfig().ifPresent(globalConfigHandle::setConfig), 0, globalConfigRefreshPeriodMillis, TimeUnit.MILLISECONDS); @@ -285,9 +288,9 @@ private void schedulePeriodicGlobalConfigRequests() { private synchronized void fetchInitialPipelineGlobalConfig() { while (!hasReceivedGlobalConfig.get()) { LOG.info("Sending request to get initial global configuration for this worker."); - Optional globalConfig = fetchGlobalConfig(); + Optional globalConfig = fetchGlobalConfig(); if (globalConfig.isPresent()) { - configManager.setConfig(globalConfig.get()); + globalConfigHandle.setConfig(globalConfig.get()); hasReceivedGlobalConfig.set(true); break; } @@ -298,7 +301,7 @@ private synchronized void fetchInitialPipelineGlobalConfig() { LOG.info("Initial global configuration received, harness is now ready"); } - private Optional fetchGlobalConfig() { + private Optional fetchGlobalConfig() { return fetchConfigWithRetry(dataflowServiceClient::getGlobalStreamingConfigWorkItem) .map(config -> createPipelineConfig(config)); } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingEnginePipelineConfig.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingGlobalConfig.java similarity index 89% rename from runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingEnginePipelineConfig.java rename to runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingGlobalConfig.java index d64f8b4d10cc..8f76f5ec27af 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingEnginePipelineConfig.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingGlobalConfig.java @@ -27,10 +27,10 @@ /** Global pipeline config for pipelines running in Streaming Engine mode. */ @AutoValue @Internal -public abstract class StreamingEnginePipelineConfig { +public abstract class StreamingGlobalConfig { - public static StreamingEnginePipelineConfig.Builder builder() { - return new AutoValue_StreamingEnginePipelineConfig.Builder() + public static StreamingGlobalConfig.Builder builder() { + return new AutoValue_StreamingGlobalConfig.Builder() .setWindmillServiceEndpoints(ImmutableSet.of()) .setUserWorkerJobSettings(UserWorkerRunnerV1Settings.newBuilder().build()) .setOperationalLimits(OperationalLimits.builder().build()); @@ -51,6 +51,6 @@ public abstract static class Builder { public abstract Builder setUserWorkerJobSettings(UserWorkerRunnerV1Settings settings); - public abstract StreamingEnginePipelineConfig build(); + public abstract StreamingGlobalConfig build(); } } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingEnginePipelineConfigManager.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingGlobalConfigHandle.java similarity index 88% rename from runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingEnginePipelineConfigManager.java rename to runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingGlobalConfigHandle.java index 3ab842ac2c7a..646f3e212e31 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingEnginePipelineConfigManager.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingGlobalConfigHandle.java @@ -24,16 +24,16 @@ @Internal @ThreadSafe -public interface StreamingEnginePipelineConfigManager { +public interface StreamingGlobalConfigHandle { /* * Returns the latest StreamingEnginePipelineConfig */ - StreamingEnginePipelineConfig getConfig(); + StreamingGlobalConfig getConfig(); /* * Subscribe to config updates by registering a callback. * Callback should be called the first time with settings, if any, inline before the method returns. */ - void onConfig(@Nonnull Consumer callback); + void onConfig(@Nonnull Consumer callback); } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingEnginePipelineConfigManagerImpl.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingGlobalConfigHandleImpl.java similarity index 79% rename from runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingEnginePipelineConfigManagerImpl.java rename to runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingGlobalConfigHandleImpl.java index 1a8228e1c74e..7c9e4ab8f6dc 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingEnginePipelineConfigManagerImpl.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingGlobalConfigHandleImpl.java @@ -28,19 +28,19 @@ @Internal @ThreadSafe -public class StreamingEnginePipelineConfigManagerImpl - implements StreamingEnginePipelineConfigManager { +public class StreamingGlobalConfigHandleImpl implements StreamingGlobalConfigHandle { - private final AtomicReference streamingEngineConfig = + private final AtomicReference streamingEngineConfig = new AtomicReference<>(); - private final CopyOnWriteArrayList> config_callbacks = + private final CopyOnWriteArrayList> config_callbacks = new CopyOnWriteArrayList<>(); /* * Returns the latest StreamingEnginePipelineConfig */ - public StreamingEnginePipelineConfig getConfig() { + @Override + public StreamingGlobalConfig getConfig() { Preconditions.checkState( streamingEngineConfig.get() != null, "Global config should be set before any processing is done"); @@ -51,8 +51,9 @@ public StreamingEnginePipelineConfig getConfig() { * Subscribe to config updates by registering a callback. * Callback will be called the first time with settings, if any, inline before the method returns. */ - public void onConfig(@Nonnull Consumer callback) { - StreamingEnginePipelineConfig config; + @Override + public void onConfig(@Nonnull Consumer callback) { + StreamingGlobalConfig config; synchronized (this) { config_callbacks.add(callback); config = streamingEngineConfig.get(); @@ -65,8 +66,8 @@ public void onConfig(@Nonnull Consumer callback) /* * Package private setter for setting config */ - void setConfig(@Nonnull StreamingEnginePipelineConfig config) { - Iterator> iterator; + void setConfig(@Nonnull StreamingGlobalConfig config) { + Iterator> iterator; synchronized (this) { if (config.equals(streamingEngineConfig.get())) { return; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/harness/StreamingWorkerStatusPages.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/harness/StreamingWorkerStatusPages.java index 5088ef334069..4654845c83da 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/harness/StreamingWorkerStatusPages.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/harness/StreamingWorkerStatusPages.java @@ -39,8 +39,8 @@ import org.apache.beam.runners.dataflow.worker.status.LastExceptionDataProvider; import org.apache.beam.runners.dataflow.worker.status.WorkerStatusPages; import org.apache.beam.runners.dataflow.worker.streaming.ComputationStateCache; -import org.apache.beam.runners.dataflow.worker.streaming.config.StreamingEnginePipelineConfig; -import org.apache.beam.runners.dataflow.worker.streaming.config.StreamingEnginePipelineConfigManager; +import org.apache.beam.runners.dataflow.worker.streaming.config.StreamingGlobalConfig; +import org.apache.beam.runners.dataflow.worker.streaming.config.StreamingGlobalConfigHandle; import org.apache.beam.runners.dataflow.worker.util.BoundedQueueExecutor; import org.apache.beam.runners.dataflow.worker.windmill.client.grpc.ChannelzServlet; import org.apache.beam.runners.dataflow.worker.windmill.client.grpc.GrpcWindmillStreamFactory; @@ -80,7 +80,7 @@ public final class StreamingWorkerStatusPages { private final DebugCapture.@Nullable Manager debugCapture; private final @Nullable ChannelzServlet channelzServlet; - private final AtomicReference seConfig = new AtomicReference<>(); + private final AtomicReference seConfig = new AtomicReference<>(); StreamingWorkerStatusPages( Supplier clock, @@ -96,7 +96,7 @@ public final class StreamingWorkerStatusPages { Consumer getDataStatusProvider, BoundedQueueExecutor workUnitExecutor, ScheduledExecutorService statusPageDumper, - StreamingEnginePipelineConfigManager configManager) { + StreamingGlobalConfigHandle globalConfigHandle) { this.clock = clock; this.clientId = clientId; this.isRunning = isRunning; @@ -110,7 +110,7 @@ public final class StreamingWorkerStatusPages { this.getDataStatusProvider = getDataStatusProvider; this.workUnitExecutor = workUnitExecutor; this.statusPageDumper = statusPageDumper; - configManager.onConfig(seConfig::set); + globalConfigHandle.onConfig(seConfig::set); } public static StreamingWorkerStatusPages.Builder builder() { @@ -273,7 +273,7 @@ public interface Builder { Builder setStatusPageDumper(ScheduledExecutorService statusPageDumper); - Builder setConfigManager(StreamingEnginePipelineConfigManager configManager); + Builder setglobalConfigHandle(StreamingGlobalConfigHandle globalConfigHandle); StreamingWorkerStatusPages build(); } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcDispatcherClient.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcDispatcherClient.java index d34826a9b9a8..4a264b84db9c 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcDispatcherClient.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcDispatcherClient.java @@ -29,7 +29,7 @@ import java.util.concurrent.atomic.AtomicReference; import javax.annotation.concurrent.GuardedBy; import javax.annotation.concurrent.ThreadSafe; -import org.apache.beam.runners.dataflow.worker.streaming.config.StreamingEnginePipelineConfig; +import org.apache.beam.runners.dataflow.worker.streaming.config.StreamingGlobalConfig; import org.apache.beam.runners.dataflow.worker.windmill.CloudWindmillMetadataServiceV1Alpha1Grpc; import org.apache.beam.runners.dataflow.worker.windmill.CloudWindmillMetadataServiceV1Alpha1Grpc.CloudWindmillMetadataServiceV1Alpha1Stub; import org.apache.beam.runners.dataflow.worker.windmill.CloudWindmillServiceV1Alpha1Grpc; @@ -148,7 +148,7 @@ public boolean hasInitializedEndpoints() { return dispatcherStubs.get().hasInitializedEndpoints(); } - public void onJobConfig(StreamingEnginePipelineConfig config) { + public void onJobConfig(StreamingGlobalConfig config) { if (config.windmillServiceEndpoints().isEmpty()) { return; } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/processing/ComputationWorkExecutorFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/processing/ComputationWorkExecutorFactory.java index cd08677573be..c1a761c8b41a 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/processing/ComputationWorkExecutorFactory.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/processing/ComputationWorkExecutorFactory.java @@ -47,7 +47,7 @@ import org.apache.beam.runners.dataflow.worker.streaming.ComputationState; import org.apache.beam.runners.dataflow.worker.streaming.ComputationWorkExecutor; import org.apache.beam.runners.dataflow.worker.streaming.StageInfo; -import org.apache.beam.runners.dataflow.worker.streaming.config.StreamingEnginePipelineConfigManager; +import org.apache.beam.runners.dataflow.worker.streaming.config.StreamingGlobalConfigHandle; import org.apache.beam.runners.dataflow.worker.util.common.worker.MapTaskExecutor; import org.apache.beam.runners.dataflow.worker.util.common.worker.OutputObjectAndByteCounter; import org.apache.beam.runners.dataflow.worker.util.common.worker.ReadOperation; @@ -95,7 +95,7 @@ final class ComputationWorkExecutorFactory { private final long maxSinkBytes; private final IdGenerator idGenerator; - private final StreamingEnginePipelineConfigManager configManager; + private final StreamingGlobalConfigHandle configManager; private final boolean throwExceptionOnLargeOutput; ComputationWorkExecutorFactory( @@ -106,7 +106,7 @@ final class ComputationWorkExecutorFactory { DataflowExecutionStateSampler sampler, CounterSet pendingDeltaCounters, IdGenerator idGenerator, - StreamingEnginePipelineConfigManager configManager) { + StreamingGlobalConfigHandle configManager) { this.options = options; this.mapTaskExecutorFactory = mapTaskExecutorFactory; this.readerCache = readerCache; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/processing/StreamingWorkScheduler.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/processing/StreamingWorkScheduler.java index 6df49bba9b30..3c968f71bf44 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/processing/StreamingWorkScheduler.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/processing/StreamingWorkScheduler.java @@ -42,7 +42,7 @@ import org.apache.beam.runners.dataflow.worker.streaming.StageInfo; import org.apache.beam.runners.dataflow.worker.streaming.Watermarks; import org.apache.beam.runners.dataflow.worker.streaming.Work; -import org.apache.beam.runners.dataflow.worker.streaming.config.StreamingEnginePipelineConfigManager; +import org.apache.beam.runners.dataflow.worker.streaming.config.StreamingGlobalConfigHandle; import org.apache.beam.runners.dataflow.worker.streaming.harness.StreamingCounters; import org.apache.beam.runners.dataflow.worker.streaming.sideinput.SideInputStateFetcher; import org.apache.beam.runners.dataflow.worker.streaming.sideinput.SideInputStateFetcherFactory; @@ -84,7 +84,7 @@ public final class StreamingWorkScheduler { private final HotKeyLogger hotKeyLogger; private final ConcurrentMap stageInfoMap; private final DataflowExecutionStateSampler sampler; - private final StreamingEnginePipelineConfigManager configManager; + private final StreamingGlobalConfigHandle configManager; public StreamingWorkScheduler( DataflowWorkerHarnessOptions options, @@ -98,7 +98,7 @@ public StreamingWorkScheduler( HotKeyLogger hotKeyLogger, ConcurrentMap stageInfoMap, DataflowExecutionStateSampler sampler, - StreamingEnginePipelineConfigManager configManager) { + StreamingGlobalConfigHandle configManager) { this.options = options; this.clock = clock; this.computationWorkExecutorFactory = computationWorkExecutorFactory; @@ -126,7 +126,7 @@ public static StreamingWorkScheduler create( HotKeyLogger hotKeyLogger, DataflowExecutionStateSampler sampler, IdGenerator idGenerator, - StreamingEnginePipelineConfigManager configManager, + StreamingGlobalConfigHandle configManager, ConcurrentMap stageInfoMap) { ComputationWorkExecutorFactory computationWorkExecutorFactory = new ComputationWorkExecutorFactory( diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorkerTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorkerTest.java index 06adb93f2a79..eafd0d1f6f49 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorkerTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorkerTest.java @@ -102,8 +102,8 @@ import org.apache.beam.runners.dataflow.worker.streaming.ShardedKey; import org.apache.beam.runners.dataflow.worker.streaming.Watermarks; import org.apache.beam.runners.dataflow.worker.streaming.Work; -import org.apache.beam.runners.dataflow.worker.streaming.config.StreamingEnginePipelineConfig; -import org.apache.beam.runners.dataflow.worker.streaming.config.StreamingEnginePipelineConfigManager; +import org.apache.beam.runners.dataflow.worker.streaming.config.StreamingGlobalConfig; +import org.apache.beam.runners.dataflow.worker.streaming.config.StreamingGlobalConfigHandleImpl; import org.apache.beam.runners.dataflow.worker.testing.RestoreDataflowLoggingMDC; import org.apache.beam.runners.dataflow.worker.testing.TestCountingSource; import org.apache.beam.runners.dataflow.worker.util.BoundedQueueExecutor; @@ -277,8 +277,7 @@ public Long get() { @Rule public TestRule restoreMDC = new RestoreDataflowLoggingMDC(); @Rule public ErrorCollector errorCollector = new ErrorCollector(); WorkUnitClient mockWorkUnitClient = mock(WorkUnitClient.class); - StreamingEnginePipelineConfigManager mockConfigManager = - mock(StreamingEnginePipelineConfigManager.class); + StreamingGlobalConfigHandleImpl mockConfigManager = mock(StreamingGlobalConfigHandleImpl.class); HotKeyLogger hotKeyLogger = mock(HotKeyLogger.class); private @Nullable ComputationStateCache computationStateCache = null; @@ -841,7 +840,7 @@ private DataflowWorkerHarnessOptions createTestingPipelineOptions(String... args private StreamingDataflowWorker makeWorker( StreamingDataflowWorkerTestParams streamingDataflowWorkerTestParams) { when(mockConfigManager.getConfig()) - .thenReturn(streamingDataflowWorkerTestParams.streamingEnginePipelineConfig()); + .thenReturn(streamingDataflowWorkerTestParams.streamingGlobalConfig()); StreamingDataflowWorker worker = StreamingDataflowWorker.forTesting( streamingDataflowWorkerTestParams.stateNameMappings(), @@ -1218,8 +1217,8 @@ public void testKeyCommitTooLargeException() throws Exception { makeWorker( defaultWorkerParams() .setInstructions(instructions) - .setStreamingEnginePipelineConfig( - StreamingEnginePipelineConfig.builder() + .setStreamingGlobalConfig( + StreamingGlobalConfig.builder() .setOperationalLimits( OperationalLimits.builder().setMaxWorkItemCommitBytes(1000).build()) .build()) @@ -1293,8 +1292,8 @@ public void testOutputKeyTooLargeException() throws Exception { makeWorker( defaultWorkerParams("--experiments=throw_exceptions_on_large_output") .setInstructions(instructions) - .setStreamingEnginePipelineConfig( - StreamingEnginePipelineConfig.builder() + .setStreamingGlobalConfig( + StreamingGlobalConfig.builder() .setOperationalLimits( OperationalLimits.builder().setMaxOutputKeyBytes(15).build()) .build()) @@ -1330,8 +1329,8 @@ public void testOutputValueTooLargeException() throws Exception { makeWorker( defaultWorkerParams("--experiments=throw_exceptions_on_large_output") .setInstructions(instructions) - .setStreamingEnginePipelineConfig( - StreamingEnginePipelineConfig.builder() + .setStreamingGlobalConfig( + StreamingGlobalConfig.builder() .setOperationalLimits( OperationalLimits.builder().setMaxOutputValueBytes(15).build()) .build()) @@ -4528,7 +4527,7 @@ private static StreamingDataflowWorkerTestParams.Builder builder() { .setLocalRetryTimeoutMs(-1) .setPublishCounters(false) .setClock(Instant::now) - .setStreamingEnginePipelineConfig(StreamingEnginePipelineConfig.builder().build()); + .setStreamingGlobalConfig(StreamingGlobalConfig.builder().build()); } abstract ImmutableMap stateNameMappings(); @@ -4545,7 +4544,7 @@ private static StreamingDataflowWorkerTestParams.Builder builder() { abstract int localRetryTimeoutMs(); - abstract StreamingEnginePipelineConfig streamingEnginePipelineConfig(); + abstract StreamingGlobalConfig streamingGlobalConfig(); @AutoValue.Builder abstract static class Builder { @@ -4580,7 +4579,7 @@ final Builder publishCounters() { abstract Builder setLocalRetryTimeoutMs(int value); - abstract Builder setStreamingEnginePipelineConfig(StreamingEnginePipelineConfig config); + abstract Builder setStreamingGlobalConfig(StreamingGlobalConfig config); abstract StreamingDataflowWorkerTestParams build(); } diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContextTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContextTest.java index 3263a2c942a0..f178576c085f 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContextTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContextTest.java @@ -59,9 +59,9 @@ import org.apache.beam.runners.dataflow.worker.profiler.ScopedProfiler.ProfileScope; import org.apache.beam.runners.dataflow.worker.streaming.Watermarks; import org.apache.beam.runners.dataflow.worker.streaming.Work; -import org.apache.beam.runners.dataflow.worker.streaming.config.FixedPipelineConfigManagerImpl; -import org.apache.beam.runners.dataflow.worker.streaming.config.StreamingEnginePipelineConfig; -import org.apache.beam.runners.dataflow.worker.streaming.config.StreamingEnginePipelineConfigManager; +import org.apache.beam.runners.dataflow.worker.streaming.config.FixedGlobalConfigHandle; +import org.apache.beam.runners.dataflow.worker.streaming.config.StreamingGlobalConfig; +import org.apache.beam.runners.dataflow.worker.streaming.config.StreamingGlobalConfigHandle; import org.apache.beam.runners.dataflow.worker.streaming.sideinput.SideInputStateFetcher; import org.apache.beam.runners.dataflow.worker.windmill.Windmill; import org.apache.beam.runners.dataflow.worker.windmill.client.getdata.FakeGetDataClient; @@ -110,8 +110,8 @@ public void setUp() { options = PipelineOptionsFactory.as(DataflowWorkerHarnessOptions.class); CounterSet counterSet = new CounterSet(); ConcurrentHashMap stateNameMap = new ConcurrentHashMap<>(); - StreamingEnginePipelineConfigManager configManager = - new FixedPipelineConfigManagerImpl(StreamingEnginePipelineConfig.builder().build()); + StreamingGlobalConfigHandle configManager = + new FixedGlobalConfigHandle(StreamingGlobalConfig.builder().build()); stateNameMap.put(NameContextsForTests.nameContextForTest().userName(), "testStateFamily"); executionContext = new StreamingModeExecutionContext( diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSourcesTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSourcesTest.java index 82e209605e3c..6d56ec78f4eb 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSourcesTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSourcesTest.java @@ -90,9 +90,9 @@ import org.apache.beam.runners.dataflow.worker.profiler.ScopedProfiler.NoopProfileScope; import org.apache.beam.runners.dataflow.worker.streaming.Watermarks; import org.apache.beam.runners.dataflow.worker.streaming.Work; -import org.apache.beam.runners.dataflow.worker.streaming.config.FixedPipelineConfigManagerImpl; -import org.apache.beam.runners.dataflow.worker.streaming.config.StreamingEnginePipelineConfig; -import org.apache.beam.runners.dataflow.worker.streaming.config.StreamingEnginePipelineConfigManager; +import org.apache.beam.runners.dataflow.worker.streaming.config.FixedGlobalConfigHandle; +import org.apache.beam.runners.dataflow.worker.streaming.config.StreamingGlobalConfig; +import org.apache.beam.runners.dataflow.worker.streaming.config.StreamingGlobalConfigHandle; import org.apache.beam.runners.dataflow.worker.streaming.sideinput.SideInputStateFetcher; import org.apache.beam.runners.dataflow.worker.testing.TestCountingSource; import org.apache.beam.runners.dataflow.worker.util.common.worker.NativeReader; @@ -597,8 +597,8 @@ public void testReadUnboundedReader() throws Exception { StreamingModeExecutionStateRegistry executionStateRegistry = new StreamingModeExecutionStateRegistry(); ReaderCache readerCache = new ReaderCache(Duration.standardMinutes(1), Runnable::run); - StreamingEnginePipelineConfigManager configManager = - new FixedPipelineConfigManagerImpl(StreamingEnginePipelineConfig.builder().build()); + StreamingGlobalConfigHandle configManager = + new FixedGlobalConfigHandle(StreamingGlobalConfig.builder().build()); StreamingModeExecutionContext context = new StreamingModeExecutionContext( counterSet, @@ -965,8 +965,8 @@ public void testFailedWorkItemsAbort() throws Exception { CounterSet counterSet = new CounterSet(); StreamingModeExecutionStateRegistry executionStateRegistry = new StreamingModeExecutionStateRegistry(); - StreamingEnginePipelineConfigManager configManager = - new FixedPipelineConfigManagerImpl(StreamingEnginePipelineConfig.builder().build()); + StreamingGlobalConfigHandle configManager = + new FixedGlobalConfigHandle(StreamingGlobalConfig.builder().build()); StreamingModeExecutionContext context = new StreamingModeExecutionContext( counterSet, diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/config/FixedPipelineConfigManagerImplTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/config/FixedGlobalConfigHandleTest.java similarity index 56% rename from runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/config/FixedPipelineConfigManagerImplTest.java rename to runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/config/FixedGlobalConfigHandleTest.java index 13dd635da182..823e0c42822c 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/config/FixedPipelineConfigManagerImplTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/config/FixedGlobalConfigHandleTest.java @@ -18,7 +18,11 @@ package org.apache.beam.runners.dataflow.worker.streaming.config; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; import org.apache.beam.runners.dataflow.worker.OperationalLimits; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.UserWorkerRunnerV1Settings; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; @@ -28,12 +32,12 @@ import org.junit.runners.JUnit4; @RunWith(JUnit4.class) -public class FixedPipelineConfigManagerImplTest { +public class FixedGlobalConfigHandleTest { @Test public void getConfig() { - StreamingEnginePipelineConfig config = - StreamingEnginePipelineConfig.builder() + StreamingGlobalConfig config = + StreamingGlobalConfig.builder() .setOperationalLimits( OperationalLimits.builder() .setMaxOutputValueBytes(123) @@ -46,7 +50,35 @@ public void getConfig() { .setUseSeparateWindmillHeartbeatStreams(false) .build()) .build(); - FixedPipelineConfigManagerImpl configManager = new FixedPipelineConfigManagerImpl(config); + FixedGlobalConfigHandle configManager = new FixedGlobalConfigHandle(config); + assertEquals(config, configManager.getConfig()); + } + + @Test + public void onConfig() throws InterruptedException { + StreamingGlobalConfig config = + StreamingGlobalConfig.builder() + .setOperationalLimits( + OperationalLimits.builder() + .setMaxOutputValueBytes(123) + .setMaxOutputKeyBytes(324) + .setMaxWorkItemCommitBytes(456) + .build()) + .setWindmillServiceEndpoints(ImmutableSet.of(HostAndPort.fromHost("windmillHost"))) + .setUserWorkerJobSettings( + UserWorkerRunnerV1Settings.newBuilder() + .setUseSeparateWindmillHeartbeatStreams(false) + .build()) + .build(); + FixedGlobalConfigHandle configManager = new FixedGlobalConfigHandle(config); + AtomicReference configFromCallback = new AtomicReference<>(); + CountDownLatch latch = new CountDownLatch(1); + configManager.onConfig( + cbConfig -> { + configFromCallback.set(cbConfig); + latch.countDown(); + }); + assertTrue(latch.await(10, TimeUnit.SECONDS)); assertEquals(config, configManager.getConfig()); } } diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingApplianceComputationConfigFetcherTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingApplianceComputationConfigFetcherTest.java index f39c98c61b19..2586ae2be86f 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingApplianceComputationConfigFetcherTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingApplianceComputationConfigFetcherTest.java @@ -137,6 +137,8 @@ public void testGetComputationConfig_onFetchConfigError() { } private StreamingApplianceComputationConfigFetcher createStreamingApplianceConfigLoader() { - return new StreamingApplianceComputationConfigFetcher(mockWindmillServer::getConfig); + return new StreamingApplianceComputationConfigFetcher( + mockWindmillServer::getConfig, + new FixedGlobalConfigHandle(StreamingGlobalConfig.builder().build())); } } diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingEngineComputationConfigFetcherTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingEngineComputationConfigFetcherTest.java index 01df8f6a95c1..24fe85aa5471 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingEngineComputationConfigFetcherTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingEngineComputationConfigFetcherTest.java @@ -55,13 +55,13 @@ public class StreamingEngineComputationConfigFetcherTest { private StreamingEngineComputationConfigFetcher createConfigFetcher( boolean waitForInitialConfig, long globalConfigRefreshPeriod, - StreamingEnginePipelineConfigManagerImpl configManager) { + StreamingGlobalConfigHandleImpl globalConfigHandle) { return StreamingEngineComputationConfigFetcher.forTesting( !waitForInitialConfig, globalConfigRefreshPeriod, mockDataflowServiceClient, - ignored -> Executors.newSingleThreadScheduledExecutor(), - configManager); + globalConfigHandle, + ignored -> Executors.newSingleThreadScheduledExecutor()); } @After @@ -76,12 +76,11 @@ public void testStart_requiresInitialConfig() throws IOException, InterruptedExc .setJobId("job") .setStreamingConfigTask(new StreamingConfigTask().setMaxWorkItemCommitBytes(10L)); CountDownLatch waitForInitialConfig = new CountDownLatch(1); - Set receivedPipelineConfig = new HashSet<>(); + Set receivedPipelineConfig = new HashSet<>(); when(mockDataflowServiceClient.getGlobalStreamingConfigWorkItem()) .thenReturn(Optional.of(initialConfig)); - StreamingEnginePipelineConfigManagerImpl configManager = - new StreamingEnginePipelineConfigManagerImpl(); - configManager.onConfig( + StreamingGlobalConfigHandleImpl globalConfigHandle = new StreamingGlobalConfigHandleImpl(); + globalConfigHandle.onConfig( config -> { try { receivedPipelineConfig.add(config); @@ -91,13 +90,13 @@ public void testStart_requiresInitialConfig() throws IOException, InterruptedExc } }); streamingEngineConfigFetcher = - createConfigFetcher(/* waitForInitialConfig= */ true, 0, configManager); + createConfigFetcher(/* waitForInitialConfig= */ true, 0, globalConfigHandle); Thread asyncStartConfigLoader = new Thread(streamingEngineConfigFetcher::start); asyncStartConfigLoader.start(); waitForInitialConfig.countDown(); asyncStartConfigLoader.join(); - StreamingEnginePipelineConfig.Builder configBuilder = - StreamingEnginePipelineConfig.builder() + StreamingGlobalConfig.Builder configBuilder = + StreamingGlobalConfig.builder() .setOperationalLimits( OperationalLimits.builder() .setMaxWorkItemCommitBytes( @@ -121,7 +120,7 @@ public void testStart_startsPeriodicConfigRequests() throws IOException, Interru .setJobId("job") .setStreamingConfigTask(new StreamingConfigTask().setMaxWorkItemCommitBytes(100L)); CountDownLatch numExpectedRefreshes = new CountDownLatch(3); - Set receivedPipelineConfig = new HashSet<>(); + Set receivedPipelineConfig = new HashSet<>(); when(mockDataflowServiceClient.getGlobalStreamingConfigWorkItem()) .thenReturn(Optional.of(firstConfig)) .thenReturn(Optional.of(secondConfig)) @@ -131,16 +130,15 @@ public void testStart_startsPeriodicConfigRequests() throws IOException, Interru // ConfigFetcher should not do anything with a config that doesn't contain a // StreamingConfigTask. .thenReturn(Optional.of(new WorkItem().setJobId("jobId"))); - StreamingEnginePipelineConfigManagerImpl configManager = - new StreamingEnginePipelineConfigManagerImpl(); - configManager.onConfig( + StreamingGlobalConfigHandleImpl globalConfigHandle = new StreamingGlobalConfigHandleImpl(); + globalConfigHandle.onConfig( config -> { receivedPipelineConfig.add(config); numExpectedRefreshes.countDown(); }); streamingEngineConfigFetcher = createConfigFetcher( - /* waitForInitialConfig= */ true, Duration.millis(100).getMillis(), configManager); + /* waitForInitialConfig= */ true, Duration.millis(100).getMillis(), globalConfigHandle); Thread asyncStartConfigLoader = new Thread(streamingEngineConfigFetcher::start); asyncStartConfigLoader.start(); @@ -148,21 +146,21 @@ public void testStart_startsPeriodicConfigRequests() throws IOException, Interru asyncStartConfigLoader.join(); assertThat(receivedPipelineConfig) .containsExactly( - StreamingEnginePipelineConfig.builder() + StreamingGlobalConfig.builder() .setOperationalLimits( OperationalLimits.builder() .setMaxWorkItemCommitBytes( firstConfig.getStreamingConfigTask().getMaxWorkItemCommitBytes()) .build()) .build(), - StreamingEnginePipelineConfig.builder() + StreamingGlobalConfig.builder() .setOperationalLimits( OperationalLimits.builder() .setMaxWorkItemCommitBytes( secondConfig.getStreamingConfigTask().getMaxWorkItemCommitBytes()) .build()) .build(), - StreamingEnginePipelineConfig.builder() + StreamingGlobalConfig.builder() .setOperationalLimits( OperationalLimits.builder() .setMaxWorkItemCommitBytes( @@ -173,10 +171,9 @@ public void testStart_startsPeriodicConfigRequests() throws IOException, Interru @Test public void testGetComputationConfig() throws IOException { - StreamingEnginePipelineConfigManagerImpl configManager = - new StreamingEnginePipelineConfigManagerImpl(); + StreamingGlobalConfigHandleImpl globalConfigHandle = new StreamingGlobalConfigHandleImpl(); streamingEngineConfigFetcher = - createConfigFetcher(/* waitForInitialConfig= */ false, 0, configManager); + createConfigFetcher(/* waitForInitialConfig= */ false, 0, globalConfigHandle); String computationId = "computationId"; String stageName = "stageName"; String systemName = "systemName"; @@ -209,12 +206,11 @@ public void testGetComputationConfig() throws IOException { @Test public void testGetComputationConfig_noComputationPresent() throws IOException { - Set receivedPipelineConfig = new HashSet<>(); - StreamingEnginePipelineConfigManagerImpl configManager = - new StreamingEnginePipelineConfigManagerImpl(); - configManager.onConfig(receivedPipelineConfig::add); + Set receivedPipelineConfig = new HashSet<>(); + StreamingGlobalConfigHandleImpl globalConfigHandle = new StreamingGlobalConfigHandleImpl(); + globalConfigHandle.onConfig(receivedPipelineConfig::add); streamingEngineConfigFetcher = - createConfigFetcher(/* waitForInitialConfig= */ false, 0, configManager); + createConfigFetcher(/* waitForInitialConfig= */ false, 0, globalConfigHandle); when(mockDataflowServiceClient.getStreamingConfigWorkItem(anyString())) .thenReturn(Optional.empty()); Optional pipelineConfig = @@ -225,10 +221,9 @@ public void testGetComputationConfig_noComputationPresent() throws IOException { @Test public void testGetComputationConfig_fetchConfigFromDataflowError() throws IOException { - StreamingEnginePipelineConfigManagerImpl configManager = - new StreamingEnginePipelineConfigManagerImpl(); + StreamingGlobalConfigHandleImpl globalConfigHandle = new StreamingGlobalConfigHandleImpl(); streamingEngineConfigFetcher = - createConfigFetcher(/* waitForInitialConfig= */ false, 0, configManager); + createConfigFetcher(/* waitForInitialConfig= */ false, 0, globalConfigHandle); RuntimeException e = new RuntimeException("something bad happened."); when(mockDataflowServiceClient.getStreamingConfigWorkItem(anyString())).thenThrow(e); Throwable fetchConfigError = diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingEnginePipelineConfigManagerImplTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingGlobalConfigHandleImplTest.java similarity index 75% rename from runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingEnginePipelineConfigManagerImplTest.java rename to runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingGlobalConfigHandleImplTest.java index fe99331eeda0..c025dbc476c0 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingEnginePipelineConfigManagerImplTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingGlobalConfigHandleImplTest.java @@ -34,13 +34,12 @@ import org.junit.runners.JUnit4; @RunWith(JUnit4.class) -public class StreamingEnginePipelineConfigManagerImplTest { +public class StreamingGlobalConfigHandleImplTest { @Test public void getConfig() { - StreamingEnginePipelineConfigManagerImpl configManager = - new StreamingEnginePipelineConfigManagerImpl(); - StreamingEnginePipelineConfig config = - StreamingEnginePipelineConfig.builder() + StreamingGlobalConfigHandleImpl globalConfigHandle = new StreamingGlobalConfigHandleImpl(); + StreamingGlobalConfig config = + StreamingGlobalConfig.builder() .setOperationalLimits( OperationalLimits.builder() .setMaxOutputValueBytes(123) @@ -53,17 +52,16 @@ public void getConfig() { .setUseSeparateWindmillHeartbeatStreams(false) .build()) .build(); - configManager.setConfig(config); - assertEquals(config, configManager.getConfig()); + globalConfigHandle.setConfig(config); + assertEquals(config, globalConfigHandle.getConfig()); } @Test public void onConfig_configSetAfterRegisteringCallback() throws InterruptedException { CountDownLatch latch = new CountDownLatch(1); - StreamingEnginePipelineConfigManagerImpl configManager = - new StreamingEnginePipelineConfigManagerImpl(); - StreamingEnginePipelineConfig configToSet = - StreamingEnginePipelineConfig.builder() + StreamingGlobalConfigHandleImpl globalConfigHandle = new StreamingGlobalConfigHandleImpl(); + StreamingGlobalConfig configToSet = + StreamingGlobalConfig.builder() .setOperationalLimits( OperationalLimits.builder() .setMaxOutputValueBytes(123) @@ -76,24 +74,23 @@ public void onConfig_configSetAfterRegisteringCallback() throws InterruptedExcep .setUseSeparateWindmillHeartbeatStreams(false) .build()) .build(); - AtomicReference configFromCallback = new AtomicReference<>(); - configManager.onConfig( + AtomicReference configFromCallback = new AtomicReference<>(); + globalConfigHandle.onConfig( config -> { configFromCallback.set(config); latch.countDown(); }); - configManager.setConfig(configToSet); + globalConfigHandle.setConfig(configToSet); assertTrue(latch.await(10, TimeUnit.SECONDS)); - assertEquals(configFromCallback.get(), configManager.getConfig()); + assertEquals(configFromCallback.get(), globalConfigHandle.getConfig()); } @Test public void onConfig_configSetBeforeRegisteringCallback() throws InterruptedException { CountDownLatch latch = new CountDownLatch(1); - StreamingEnginePipelineConfigManagerImpl configManager = - new StreamingEnginePipelineConfigManagerImpl(); - StreamingEnginePipelineConfig configToSet = - StreamingEnginePipelineConfig.builder() + StreamingGlobalConfigHandleImpl globalConfigHandle = new StreamingGlobalConfigHandleImpl(); + StreamingGlobalConfig configToSet = + StreamingGlobalConfig.builder() .setOperationalLimits( OperationalLimits.builder() .setMaxOutputValueBytes(123) @@ -106,26 +103,25 @@ public void onConfig_configSetBeforeRegisteringCallback() throws InterruptedExce .setUseSeparateWindmillHeartbeatStreams(false) .build()) .build(); - AtomicReference configFromCallback = new AtomicReference<>(); - configManager.setConfig(configToSet); - configManager.onConfig( + AtomicReference configFromCallback = new AtomicReference<>(); + globalConfigHandle.setConfig(configToSet); + globalConfigHandle.onConfig( config -> { configFromCallback.set(config); latch.countDown(); }); assertTrue(latch.await(10, TimeUnit.SECONDS)); - assertEquals(configFromCallback.get(), configManager.getConfig()); + assertEquals(configFromCallback.get(), globalConfigHandle.getConfig()); } @Test public void onConfig_shouldNotCallCallbackForIfConfigRemainsSame() throws InterruptedException { CountDownLatch latch = new CountDownLatch(1); AtomicInteger callbackCount = new AtomicInteger(0); - StreamingEnginePipelineConfigManagerImpl configManager = - new StreamingEnginePipelineConfigManagerImpl(); - Supplier configToSet = + StreamingGlobalConfigHandleImpl globalConfigHandle = new StreamingGlobalConfigHandleImpl(); + Supplier configToSet = () -> - StreamingEnginePipelineConfig.builder() + StreamingGlobalConfig.builder() .setOperationalLimits( OperationalLimits.builder() .setMaxOutputValueBytes(123) @@ -138,14 +134,14 @@ public void onConfig_shouldNotCallCallbackForIfConfigRemainsSame() throws Interr .setUseSeparateWindmillHeartbeatStreams(false) .build()) .build(); - configManager.onConfig( + globalConfigHandle.onConfig( config -> { callbackCount.incrementAndGet(); latch.countDown(); }); - configManager.setConfig(configToSet.get()); + globalConfigHandle.setConfig(configToSet.get()); // call setter again with same config - configManager.setConfig(configToSet.get()); + globalConfigHandle.setConfig(configToSet.get()); assertTrue(latch.await(10, TimeUnit.SECONDS)); assertEquals(1, callbackCount.get()); } From c713718dfe6938e39f3c78fe36e6e717bd48de9d Mon Sep 17 00:00:00 2001 From: Arun Pandian Date: Thu, 12 Sep 2024 19:43:50 -0700 Subject: [PATCH 04/13] review comments --- .../worker/StreamingDataflowWorker.java | 2 +- .../worker/StreamingModeExecutionContext.java | 9 +++++---- .../config/FixedGlobalConfigHandle.java | 4 ++-- .../config/StreamingGlobalConfigHandle.java | 12 +++++------- .../StreamingGlobalConfigHandleImpl.java | 18 ++++++------------ .../harness/StreamingWorkerStatusPages.java | 2 +- .../ComputationWorkExecutorFactory.java | 8 ++++---- .../processing/StreamingWorkScheduler.java | 14 +++++++------- .../worker/StreamingDataflowWorkerTest.java | 7 ++++--- .../StreamingModeExecutionContextTest.java | 4 ++-- .../worker/WorkerCustomSourcesTest.java | 8 ++++---- .../config/FixedGlobalConfigHandleTest.java | 12 ++++++------ ...mingEngineComputationConfigFetcherTest.java | 6 +++--- .../StreamingGlobalConfigHandleImplTest.java | 16 ++++++++++------ 14 files changed, 60 insertions(+), 62 deletions(-) diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java index 092e43bf3a6f..c6cc58739f41 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java @@ -434,7 +434,7 @@ public static StreamingDataflowWorker fromOptions(DataflowWorkerHarnessOptions o configFetcher = StreamingEngineComputationConfigFetcher.create( options.getGlobalConfigRefreshPeriod().getMillis(), dataflowServiceClient); - configFetcher.getGlobalConfigHandle().onConfig(dispatcherClient::onJobConfig); + configFetcher.getGlobalConfigHandle().registerConfigObserver(dispatcherClient::onJobConfig); computationStateCache = computationStateCacheFactory.apply(configFetcher); windmillStreamFactory = windmillStreamFactoryBuilder diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContext.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContext.java index 85af41577358..5ff94884e974 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContext.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContext.java @@ -108,7 +108,7 @@ public class StreamingModeExecutionContext extends DataflowExecutionContext stateNameMap; private final WindmillStateCache.ForComputation stateCache; private final ReaderCache readerCache; - private final StreamingGlobalConfigHandle configManager; + private final StreamingGlobalConfigHandle globalConfigHandle; private final boolean throwExceptionOnLargeOutput; private volatile long backlogBytes; @@ -155,7 +155,7 @@ public StreamingModeExecutionContext( MetricsContainerRegistry metricsContainerRegistry, DataflowExecutionStateTracker executionStateTracker, StreamingModeExecutionStateRegistry executionStateRegistry, - StreamingGlobalConfigHandle configManager, + StreamingGlobalConfigHandle globalConfigHandle, long sinkByteLimit, boolean throwExceptionOnLargeOutput) { super( @@ -166,7 +166,7 @@ public StreamingModeExecutionContext( sinkByteLimit); this.computationId = computationId; this.readerCache = readerCache; - this.configManager = configManager; + this.globalConfigHandle = globalConfigHandle; this.sideInputCache = new HashMap<>(); this.stateNameMap = ImmutableMap.copyOf(stateNameMap); this.stateCache = stateCache; @@ -205,7 +205,8 @@ public void start( this.work = work; this.computationKey = WindmillComputationKey.create(computationId, work.getShardedKey()); this.sideInputStateFetcher = sideInputStateFetcher; - this.operationalLimits = configManager.getConfig().operationalLimits(); + // Snapshot the limits for entire bundle processing. + this.operationalLimits = globalConfigHandle.getConfig().operationalLimits(); this.outputBuilder = outputBuilder; this.sideInputCache.clear(); clearSinkFullHint(); diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/FixedGlobalConfigHandle.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/FixedGlobalConfigHandle.java index e0485ae77b71..c244ecb8c7a8 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/FixedGlobalConfigHandle.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/FixedGlobalConfigHandle.java @@ -25,7 +25,7 @@ @Internal @ThreadSafe /* - * StreamingEnginePipelineConfigManager returning a fixed config + * StreamingGlobalConfigHandle returning a fixed config * initialized during construction. Used for Appliance and Tests. */ public class FixedGlobalConfigHandle implements StreamingGlobalConfigHandle { @@ -42,7 +42,7 @@ public StreamingGlobalConfig getConfig() { } @Override - public void onConfig(@Nonnull Consumer callback) { + public void registerConfigObserver(@Nonnull Consumer callback) { callback.accept(config); } } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingGlobalConfigHandle.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingGlobalConfigHandle.java index 646f3e212e31..f0a8989fbe3c 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingGlobalConfigHandle.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingGlobalConfigHandle.java @@ -26,14 +26,12 @@ @ThreadSafe public interface StreamingGlobalConfigHandle { - /* - * Returns the latest StreamingEnginePipelineConfig - */ + /** Returns the latest StreamingGlobalConfig */ StreamingGlobalConfig getConfig(); - /* - * Subscribe to config updates by registering a callback. - * Callback should be called the first time with settings, if any, inline before the method returns. + /** + * Subscribe to config updates by registering a callback. Callback should be called the first time + * with settings, if any, inline before the method returns. */ - void onConfig(@Nonnull Consumer callback); + void registerConfigObserver(@Nonnull Consumer callback); } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingGlobalConfigHandleImpl.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingGlobalConfigHandleImpl.java index 7c9e4ab8f6dc..458f9b1630d9 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingGlobalConfigHandleImpl.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingGlobalConfigHandleImpl.java @@ -36,9 +36,6 @@ public class StreamingGlobalConfigHandleImpl implements StreamingGlobalConfigHan private final CopyOnWriteArrayList> config_callbacks = new CopyOnWriteArrayList<>(); - /* - * Returns the latest StreamingEnginePipelineConfig - */ @Override public StreamingGlobalConfig getConfig() { Preconditions.checkState( @@ -47,25 +44,22 @@ public StreamingGlobalConfig getConfig() { return streamingEngineConfig.get(); } - /* - * Subscribe to config updates by registering a callback. - * Callback will be called the first time with settings, if any, inline before the method returns. - */ @Override - public void onConfig(@Nonnull Consumer callback) { + public void registerConfigObserver(@Nonnull Consumer callback) { StreamingGlobalConfig config; synchronized (this) { config_callbacks.add(callback); config = streamingEngineConfig.get(); } if (config != null) { - callback.accept(config); + // read config from streamingEngineConfig again + // to prevent calling callback with stale config. + // The cached `config` will be stale if setConfig + // ran after the synchronized block. + callback.accept(streamingEngineConfig.get()); } } - /* - * Package private setter for setting config - */ void setConfig(@Nonnull StreamingGlobalConfig config) { Iterator> iterator; synchronized (this) { diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/harness/StreamingWorkerStatusPages.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/harness/StreamingWorkerStatusPages.java index 4654845c83da..b65a5f10658e 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/harness/StreamingWorkerStatusPages.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/harness/StreamingWorkerStatusPages.java @@ -110,7 +110,7 @@ public final class StreamingWorkerStatusPages { this.getDataStatusProvider = getDataStatusProvider; this.workUnitExecutor = workUnitExecutor; this.statusPageDumper = statusPageDumper; - globalConfigHandle.onConfig(seConfig::set); + globalConfigHandle.registerConfigObserver(seConfig::set); } public static StreamingWorkerStatusPages.Builder builder() { diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/processing/ComputationWorkExecutorFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/processing/ComputationWorkExecutorFactory.java index c1a761c8b41a..d5e0b3a24e2a 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/processing/ComputationWorkExecutorFactory.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/processing/ComputationWorkExecutorFactory.java @@ -95,7 +95,7 @@ final class ComputationWorkExecutorFactory { private final long maxSinkBytes; private final IdGenerator idGenerator; - private final StreamingGlobalConfigHandle configManager; + private final StreamingGlobalConfigHandle globalConfigHandle; private final boolean throwExceptionOnLargeOutput; ComputationWorkExecutorFactory( @@ -106,13 +106,13 @@ final class ComputationWorkExecutorFactory { DataflowExecutionStateSampler sampler, CounterSet pendingDeltaCounters, IdGenerator idGenerator, - StreamingGlobalConfigHandle configManager) { + StreamingGlobalConfigHandle globalConfigHandle) { this.options = options; this.mapTaskExecutorFactory = mapTaskExecutorFactory; this.readerCache = readerCache; this.stateCacheFactory = stateCacheFactory; this.idGenerator = idGenerator; - this.configManager = configManager; + this.globalConfigHandle = globalConfigHandle; this.readerRegistry = ReaderRegistry.defaultRegistry(); this.sinkRegistry = SinkRegistry.defaultRegistry(); this.sampler = sampler; @@ -266,7 +266,7 @@ private StreamingModeExecutionContext createExecutionContext( stageInfo.metricsContainerRegistry(), executionStateTracker, stageInfo.executionStateRegistry(), - configManager, + globalConfigHandle, maxSinkBytes, throwExceptionOnLargeOutput); } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/processing/StreamingWorkScheduler.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/processing/StreamingWorkScheduler.java index 3c968f71bf44..641fd119a42d 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/processing/StreamingWorkScheduler.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/processing/StreamingWorkScheduler.java @@ -84,7 +84,7 @@ public final class StreamingWorkScheduler { private final HotKeyLogger hotKeyLogger; private final ConcurrentMap stageInfoMap; private final DataflowExecutionStateSampler sampler; - private final StreamingGlobalConfigHandle configManager; + private final StreamingGlobalConfigHandle globalConfigHandle; public StreamingWorkScheduler( DataflowWorkerHarnessOptions options, @@ -98,7 +98,7 @@ public StreamingWorkScheduler( HotKeyLogger hotKeyLogger, ConcurrentMap stageInfoMap, DataflowExecutionStateSampler sampler, - StreamingGlobalConfigHandle configManager) { + StreamingGlobalConfigHandle globalConfigHandle) { this.options = options; this.clock = clock; this.computationWorkExecutorFactory = computationWorkExecutorFactory; @@ -110,7 +110,7 @@ public StreamingWorkScheduler( this.hotKeyLogger = hotKeyLogger; this.stageInfoMap = stageInfoMap; this.sampler = sampler; - this.configManager = configManager; + this.globalConfigHandle = globalConfigHandle; } public static StreamingWorkScheduler create( @@ -126,7 +126,7 @@ public static StreamingWorkScheduler create( HotKeyLogger hotKeyLogger, DataflowExecutionStateSampler sampler, IdGenerator idGenerator, - StreamingGlobalConfigHandle configManager, + StreamingGlobalConfigHandle globalConfigHandle, ConcurrentMap stageInfoMap) { ComputationWorkExecutorFactory computationWorkExecutorFactory = new ComputationWorkExecutorFactory( @@ -137,7 +137,7 @@ public static StreamingWorkScheduler create( sampler, streamingCounters.pendingDeltaCounters(), idGenerator, - configManager); + globalConfigHandle); return new StreamingWorkScheduler( options, @@ -151,7 +151,7 @@ public static StreamingWorkScheduler create( hotKeyLogger, stageInfoMap, sampler, - configManager); + globalConfigHandle); } private static long computeShuffleBytesRead(Windmill.WorkItem workItem) { @@ -295,7 +295,7 @@ private Windmill.WorkItemCommitRequest validateCommitRequestSize( Windmill.WorkItemCommitRequest commitRequest, String computationId, Windmill.WorkItem workItem) { - long byteLimit = configManager.getConfig().operationalLimits().getMaxWorkItemCommitBytes(); + long byteLimit = globalConfigHandle.getConfig().operationalLimits().getMaxWorkItemCommitBytes(); int commitSize = commitRequest.getSerializedSize(); int estimatedCommitSize = commitSize < 0 ? Integer.MAX_VALUE : commitSize; diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorkerTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorkerTest.java index eafd0d1f6f49..95d3a9f7ce55 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorkerTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorkerTest.java @@ -277,7 +277,8 @@ public Long get() { @Rule public TestRule restoreMDC = new RestoreDataflowLoggingMDC(); @Rule public ErrorCollector errorCollector = new ErrorCollector(); WorkUnitClient mockWorkUnitClient = mock(WorkUnitClient.class); - StreamingGlobalConfigHandleImpl mockConfigManager = mock(StreamingGlobalConfigHandleImpl.class); + StreamingGlobalConfigHandleImpl mockglobalConfigHandle = + mock(StreamingGlobalConfigHandleImpl.class); HotKeyLogger hotKeyLogger = mock(HotKeyLogger.class); private @Nullable ComputationStateCache computationStateCache = null; @@ -839,7 +840,7 @@ private DataflowWorkerHarnessOptions createTestingPipelineOptions(String... args private StreamingDataflowWorker makeWorker( StreamingDataflowWorkerTestParams streamingDataflowWorkerTestParams) { - when(mockConfigManager.getConfig()) + when(mockglobalConfigHandle.getConfig()) .thenReturn(streamingDataflowWorkerTestParams.streamingGlobalConfig()); StreamingDataflowWorker worker = StreamingDataflowWorker.forTesting( @@ -854,7 +855,7 @@ private StreamingDataflowWorker makeWorker( hotKeyLogger, streamingDataflowWorkerTestParams.clock(), streamingDataflowWorkerTestParams.executorSupplier(), - mockConfigManager, + mockglobalConfigHandle, streamingDataflowWorkerTestParams.localRetryTimeoutMs()); this.computationStateCache = worker.getComputationStateCache(); return worker; diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContextTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContextTest.java index f178576c085f..a1d4210f3dbc 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContextTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContextTest.java @@ -110,7 +110,7 @@ public void setUp() { options = PipelineOptionsFactory.as(DataflowWorkerHarnessOptions.class); CounterSet counterSet = new CounterSet(); ConcurrentHashMap stateNameMap = new ConcurrentHashMap<>(); - StreamingGlobalConfigHandle configManager = + StreamingGlobalConfigHandle globalConfigHandle = new FixedGlobalConfigHandle(StreamingGlobalConfig.builder().build()); stateNameMap.put(NameContextsForTests.nameContextForTest().userName(), "testStateFamily"); executionContext = @@ -132,7 +132,7 @@ public void setUp() { PipelineOptionsFactory.create(), "test-work-item-id"), executionStateRegistry, - configManager, + globalConfigHandle, Long.MAX_VALUE, /*throwExceptionOnLargeOutput=*/ false); } diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSourcesTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSourcesTest.java index 6d56ec78f4eb..8ad73a5145bc 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSourcesTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSourcesTest.java @@ -597,7 +597,7 @@ public void testReadUnboundedReader() throws Exception { StreamingModeExecutionStateRegistry executionStateRegistry = new StreamingModeExecutionStateRegistry(); ReaderCache readerCache = new ReaderCache(Duration.standardMinutes(1), Runnable::run); - StreamingGlobalConfigHandle configManager = + StreamingGlobalConfigHandle globalConfigHandle = new FixedGlobalConfigHandle(StreamingGlobalConfig.builder().build()); StreamingModeExecutionContext context = new StreamingModeExecutionContext( @@ -615,7 +615,7 @@ public void testReadUnboundedReader() throws Exception { PipelineOptionsFactory.create(), "test-work-item-id"), executionStateRegistry, - configManager, + globalConfigHandle, Long.MAX_VALUE, /*throwExceptionOnLargeOutput=*/ false); @@ -965,7 +965,7 @@ public void testFailedWorkItemsAbort() throws Exception { CounterSet counterSet = new CounterSet(); StreamingModeExecutionStateRegistry executionStateRegistry = new StreamingModeExecutionStateRegistry(); - StreamingGlobalConfigHandle configManager = + StreamingGlobalConfigHandle globalConfigHandle = new FixedGlobalConfigHandle(StreamingGlobalConfig.builder().build()); StreamingModeExecutionContext context = new StreamingModeExecutionContext( @@ -986,7 +986,7 @@ public void testFailedWorkItemsAbort() throws Exception { PipelineOptionsFactory.create(), "test-work-item-id"), executionStateRegistry, - configManager, + globalConfigHandle, Long.MAX_VALUE, /*throwExceptionOnLargeOutput=*/ false); diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/config/FixedGlobalConfigHandleTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/config/FixedGlobalConfigHandleTest.java index 823e0c42822c..b5cb85a58c12 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/config/FixedGlobalConfigHandleTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/config/FixedGlobalConfigHandleTest.java @@ -50,12 +50,12 @@ public void getConfig() { .setUseSeparateWindmillHeartbeatStreams(false) .build()) .build(); - FixedGlobalConfigHandle configManager = new FixedGlobalConfigHandle(config); - assertEquals(config, configManager.getConfig()); + FixedGlobalConfigHandle globalConfigHandle = new FixedGlobalConfigHandle(config); + assertEquals(config, globalConfigHandle.getConfig()); } @Test - public void onConfig() throws InterruptedException { + public void registerConfigObserver() throws InterruptedException { StreamingGlobalConfig config = StreamingGlobalConfig.builder() .setOperationalLimits( @@ -70,15 +70,15 @@ public void onConfig() throws InterruptedException { .setUseSeparateWindmillHeartbeatStreams(false) .build()) .build(); - FixedGlobalConfigHandle configManager = new FixedGlobalConfigHandle(config); + FixedGlobalConfigHandle globalConfigHandle = new FixedGlobalConfigHandle(config); AtomicReference configFromCallback = new AtomicReference<>(); CountDownLatch latch = new CountDownLatch(1); - configManager.onConfig( + globalConfigHandle.registerConfigObserver( cbConfig -> { configFromCallback.set(cbConfig); latch.countDown(); }); assertTrue(latch.await(10, TimeUnit.SECONDS)); - assertEquals(config, configManager.getConfig()); + assertEquals(configFromCallback.get(), globalConfigHandle.getConfig()); } } diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingEngineComputationConfigFetcherTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingEngineComputationConfigFetcherTest.java index 24fe85aa5471..9fa17588c94d 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingEngineComputationConfigFetcherTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingEngineComputationConfigFetcherTest.java @@ -80,7 +80,7 @@ public void testStart_requiresInitialConfig() throws IOException, InterruptedExc when(mockDataflowServiceClient.getGlobalStreamingConfigWorkItem()) .thenReturn(Optional.of(initialConfig)); StreamingGlobalConfigHandleImpl globalConfigHandle = new StreamingGlobalConfigHandleImpl(); - globalConfigHandle.onConfig( + globalConfigHandle.registerConfigObserver( config -> { try { receivedPipelineConfig.add(config); @@ -131,7 +131,7 @@ public void testStart_startsPeriodicConfigRequests() throws IOException, Interru // StreamingConfigTask. .thenReturn(Optional.of(new WorkItem().setJobId("jobId"))); StreamingGlobalConfigHandleImpl globalConfigHandle = new StreamingGlobalConfigHandleImpl(); - globalConfigHandle.onConfig( + globalConfigHandle.registerConfigObserver( config -> { receivedPipelineConfig.add(config); numExpectedRefreshes.countDown(); @@ -208,7 +208,7 @@ public void testGetComputationConfig() throws IOException { public void testGetComputationConfig_noComputationPresent() throws IOException { Set receivedPipelineConfig = new HashSet<>(); StreamingGlobalConfigHandleImpl globalConfigHandle = new StreamingGlobalConfigHandleImpl(); - globalConfigHandle.onConfig(receivedPipelineConfig::add); + globalConfigHandle.registerConfigObserver(receivedPipelineConfig::add); streamingEngineConfigFetcher = createConfigFetcher(/* waitForInitialConfig= */ false, 0, globalConfigHandle); when(mockDataflowServiceClient.getStreamingConfigWorkItem(anyString())) diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingGlobalConfigHandleImplTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingGlobalConfigHandleImplTest.java index c025dbc476c0..0fae05050205 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingGlobalConfigHandleImplTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingGlobalConfigHandleImplTest.java @@ -35,6 +35,7 @@ @RunWith(JUnit4.class) public class StreamingGlobalConfigHandleImplTest { + @Test public void getConfig() { StreamingGlobalConfigHandleImpl globalConfigHandle = new StreamingGlobalConfigHandleImpl(); @@ -57,7 +58,8 @@ public void getConfig() { } @Test - public void onConfig_configSetAfterRegisteringCallback() throws InterruptedException { + public void registerConfigObserver_configSetAfterRegisteringCallback() + throws InterruptedException { CountDownLatch latch = new CountDownLatch(1); StreamingGlobalConfigHandleImpl globalConfigHandle = new StreamingGlobalConfigHandleImpl(); StreamingGlobalConfig configToSet = @@ -75,7 +77,7 @@ public void onConfig_configSetAfterRegisteringCallback() throws InterruptedExcep .build()) .build(); AtomicReference configFromCallback = new AtomicReference<>(); - globalConfigHandle.onConfig( + globalConfigHandle.registerConfigObserver( config -> { configFromCallback.set(config); latch.countDown(); @@ -86,7 +88,8 @@ public void onConfig_configSetAfterRegisteringCallback() throws InterruptedExcep } @Test - public void onConfig_configSetBeforeRegisteringCallback() throws InterruptedException { + public void registerConfigObserver_configSetBeforeRegisteringCallback() + throws InterruptedException { CountDownLatch latch = new CountDownLatch(1); StreamingGlobalConfigHandleImpl globalConfigHandle = new StreamingGlobalConfigHandleImpl(); StreamingGlobalConfig configToSet = @@ -105,7 +108,7 @@ public void onConfig_configSetBeforeRegisteringCallback() throws InterruptedExce .build(); AtomicReference configFromCallback = new AtomicReference<>(); globalConfigHandle.setConfig(configToSet); - globalConfigHandle.onConfig( + globalConfigHandle.registerConfigObserver( config -> { configFromCallback.set(config); latch.countDown(); @@ -115,7 +118,8 @@ public void onConfig_configSetBeforeRegisteringCallback() throws InterruptedExce } @Test - public void onConfig_shouldNotCallCallbackForIfConfigRemainsSame() throws InterruptedException { + public void registerConfigObserver_shouldNotCallCallbackForIfConfigRemainsSame() + throws InterruptedException { CountDownLatch latch = new CountDownLatch(1); AtomicInteger callbackCount = new AtomicInteger(0); StreamingGlobalConfigHandleImpl globalConfigHandle = new StreamingGlobalConfigHandleImpl(); @@ -134,7 +138,7 @@ public void onConfig_shouldNotCallCallbackForIfConfigRemainsSame() throws Interr .setUseSeparateWindmillHeartbeatStreams(false) .build()) .build(); - globalConfigHandle.onConfig( + globalConfigHandle.registerConfigObserver( config -> { callbackCount.incrementAndGet(); latch.countDown(); From 785999b18c5a2623cff826c0f19959d25ef97bc7 Mon Sep 17 00:00:00 2001 From: Arun Pandian Date: Tue, 17 Sep 2024 01:40:57 -0700 Subject: [PATCH 05/13] review comments --- .../config/StreamingGlobalConfigHandle.java | 2 +- .../StreamingGlobalConfigHandleImpl.java | 41 +++++++++++-------- 2 files changed, 24 insertions(+), 19 deletions(-) diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingGlobalConfigHandle.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingGlobalConfigHandle.java index f0a8989fbe3c..abc60c0c3ce4 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingGlobalConfigHandle.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingGlobalConfigHandle.java @@ -31,7 +31,7 @@ public interface StreamingGlobalConfigHandle { /** * Subscribe to config updates by registering a callback. Callback should be called the first time - * with settings, if any, inline before the method returns. + * with settings, if any. */ void registerConfigObserver(@Nonnull Consumer callback); } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingGlobalConfigHandleImpl.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingGlobalConfigHandleImpl.java index 458f9b1630d9..3aa5b580db87 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingGlobalConfigHandleImpl.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingGlobalConfigHandleImpl.java @@ -18,11 +18,15 @@ package org.apache.beam.runners.dataflow.worker.streaming.config; import com.google.common.base.Preconditions; +import java.util.ArrayList; import java.util.Iterator; -import java.util.concurrent.CopyOnWriteArrayList; +import java.util.List; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Consumer; import javax.annotation.Nonnull; +import javax.annotation.concurrent.GuardedBy; import javax.annotation.concurrent.ThreadSafe; import org.apache.beam.sdk.annotations.Internal; @@ -33,8 +37,13 @@ public class StreamingGlobalConfigHandleImpl implements StreamingGlobalConfigHan private final AtomicReference streamingEngineConfig = new AtomicReference<>(); - private final CopyOnWriteArrayList> config_callbacks = - new CopyOnWriteArrayList<>(); + @GuardedBy("this") + private final List> config_callbacks = new ArrayList<>(); + + // Using a single threaded executor to call callbacks in the scheduled order. + private final ExecutorService singleThreadExecutor = + Executors.newSingleThreadExecutor( + r -> new Thread(r, "StreamingGlobalConfigHandleImpl Executor")); @Override public StreamingGlobalConfig getConfig() { @@ -46,17 +55,12 @@ public StreamingGlobalConfig getConfig() { @Override public void registerConfigObserver(@Nonnull Consumer callback) { - StreamingGlobalConfig config; synchronized (this) { config_callbacks.add(callback); - config = streamingEngineConfig.get(); - } - if (config != null) { - // read config from streamingEngineConfig again - // to prevent calling callback with stale config. - // The cached `config` will be stale if setConfig - // ran after the synchronized block. - callback.accept(streamingEngineConfig.get()); + // If the config is already set, schedule a callback + if (streamingEngineConfig.get() != null) { + scheduleConfigCallback(callback); + } } } @@ -67,12 +71,13 @@ void setConfig(@Nonnull StreamingGlobalConfig config) { return; } streamingEngineConfig.set(config); - // iterator of CopyOnWriteArrayList provides - // snapshot semantics - iterator = config_callbacks.iterator(); - } - while (iterator.hasNext()) { - iterator.next().accept(config); + for (Consumer callback : config_callbacks) { + scheduleConfigCallback(callback); + } } } + + private void scheduleConfigCallback(Consumer callback) { + singleThreadExecutor.submit(() -> callback.accept(streamingEngineConfig.get())); + } } From de82664dad430d3644ee04f0d41067b7cf2576a7 Mon Sep 17 00:00:00 2001 From: Arun Pandian Date: Tue, 17 Sep 2024 01:54:35 -0700 Subject: [PATCH 06/13] review comments --- .../config/StreamingGlobalConfigHandleImpl.java | 2 +- .../streaming/harness/StreamingWorkerStatusPages.java | 9 +++++---- .../config/StreamingGlobalConfigHandleImplTest.java | 1 + 3 files changed, 7 insertions(+), 5 deletions(-) diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingGlobalConfigHandleImpl.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingGlobalConfigHandleImpl.java index 3aa5b580db87..209f925df552 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingGlobalConfigHandleImpl.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingGlobalConfigHandleImpl.java @@ -17,7 +17,7 @@ */ package org.apache.beam.runners.dataflow.worker.streaming.config; -import com.google.common.base.Preconditions; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.common.base.Preconditions; import java.util.ArrayList; import java.util.Iterator; import java.util.List; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/harness/StreamingWorkerStatusPages.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/harness/StreamingWorkerStatusPages.java index b65a5f10658e..711ca81b8cad 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/harness/StreamingWorkerStatusPages.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/harness/StreamingWorkerStatusPages.java @@ -80,7 +80,7 @@ public final class StreamingWorkerStatusPages { private final DebugCapture.@Nullable Manager debugCapture; private final @Nullable ChannelzServlet channelzServlet; - private final AtomicReference seConfig = new AtomicReference<>(); + private final AtomicReference globalConfig = new AtomicReference<>(); StreamingWorkerStatusPages( Supplier clock, @@ -110,7 +110,7 @@ public final class StreamingWorkerStatusPages { this.getDataStatusProvider = getDataStatusProvider; this.workUnitExecutor = workUnitExecutor; this.statusPageDumper = statusPageDumper; - globalConfigHandle.registerConfigObserver(seConfig::set); + globalConfigHandle.registerConfigObserver(globalConfig::set); } public static StreamingWorkerStatusPages.Builder builder() { @@ -161,11 +161,12 @@ private void addStreamingEngineStatusPages() { "jobSettings", "User Worker Job Settings", writer -> { - if (seConfig.get() == null) { + StreamingGlobalConfig config = globalConfig.get(); + if (config == null) { writer.println("Job Settings not loaded."); return; } - writer.println(seConfig.get().userWorkerJobSettings().toString()); + writer.println(config.userWorkerJobSettings().toString()); }); } diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingGlobalConfigHandleImplTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingGlobalConfigHandleImplTest.java index 0fae05050205..aaab58003e66 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingGlobalConfigHandleImplTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingGlobalConfigHandleImplTest.java @@ -147,6 +147,7 @@ public void registerConfigObserver_shouldNotCallCallbackForIfConfigRemainsSame() // call setter again with same config globalConfigHandle.setConfig(configToSet.get()); assertTrue(latch.await(10, TimeUnit.SECONDS)); + Thread.sleep(TimeUnit.SECONDS.toMillis(10)); assertEquals(1, callbackCount.get()); } } From 808df344487addd3d30c4caf25f133aede0658e9 Mon Sep 17 00:00:00 2001 From: Arun Pandian Date: Tue, 17 Sep 2024 01:56:55 -0700 Subject: [PATCH 07/13] review comments --- .../beam/runners/dataflow/worker/StreamingDataflowWorker.java | 2 +- .../streaming/config/StreamingGlobalConfigHandleImpl.java | 2 +- .../worker/streaming/harness/StreamingWorkerStatusPages.java | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java index c6cc58739f41..bc9fd8228c70 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java @@ -295,7 +295,7 @@ private StreamingDataflowWorker( .setCurrentActiveCommitBytes(workCommitter::currentActiveCommitBytes) .setGetDataStatusProvider(getDataClient::printHtml) .setWorkUnitExecutor(workUnitExecutor) - .setglobalConfigHandle(configFetcher.getGlobalConfigHandle()) + .setGlobalConfigHandle(configFetcher.getGlobalConfigHandle()) .build(); Windmill.GetWorkRequest request = diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingGlobalConfigHandleImpl.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingGlobalConfigHandleImpl.java index 209f925df552..9c38f9f389fa 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingGlobalConfigHandleImpl.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingGlobalConfigHandleImpl.java @@ -17,7 +17,6 @@ */ package org.apache.beam.runners.dataflow.worker.streaming.config; -import org.apache.beam.vendor.grpc.v1p60p1.com.google.common.base.Preconditions; import java.util.ArrayList; import java.util.Iterator; import java.util.List; @@ -29,6 +28,7 @@ import javax.annotation.concurrent.GuardedBy; import javax.annotation.concurrent.ThreadSafe; import org.apache.beam.sdk.annotations.Internal; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.common.base.Preconditions; @Internal @ThreadSafe diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/harness/StreamingWorkerStatusPages.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/harness/StreamingWorkerStatusPages.java index 711ca81b8cad..e2fa2245a3f1 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/harness/StreamingWorkerStatusPages.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/harness/StreamingWorkerStatusPages.java @@ -274,7 +274,7 @@ public interface Builder { Builder setStatusPageDumper(ScheduledExecutorService statusPageDumper); - Builder setglobalConfigHandle(StreamingGlobalConfigHandle globalConfigHandle); + Builder setGlobalConfigHandle(StreamingGlobalConfigHandle globalConfigHandle); StreamingWorkerStatusPages build(); } From 0e6b1dcd3e50a26216222a681cd26bc99b174831 Mon Sep 17 00:00:00 2001 From: Arun Pandian Date: Tue, 17 Sep 2024 02:20:38 -0700 Subject: [PATCH 08/13] review comments --- .../config/StreamingGlobalConfigHandleImpl.java | 14 +++++++++++++- 1 file changed, 13 insertions(+), 1 deletion(-) diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingGlobalConfigHandleImpl.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingGlobalConfigHandleImpl.java index 9c38f9f389fa..be0faa03b718 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingGlobalConfigHandleImpl.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingGlobalConfigHandleImpl.java @@ -22,6 +22,7 @@ import java.util.List; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; +import java.util.concurrent.Future; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Consumer; import javax.annotation.Nonnull; @@ -29,10 +30,13 @@ import javax.annotation.concurrent.ThreadSafe; import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.vendor.grpc.v1p60p1.com.google.common.base.Preconditions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; @Internal @ThreadSafe public class StreamingGlobalConfigHandleImpl implements StreamingGlobalConfigHandle { + private static final Logger LOG = LoggerFactory.getLogger(StreamingGlobalConfigHandleImpl.class); private final AtomicReference streamingEngineConfig = new AtomicReference<>(); @@ -78,6 +82,14 @@ void setConfig(@Nonnull StreamingGlobalConfig config) { } private void scheduleConfigCallback(Consumer callback) { - singleThreadExecutor.submit(() -> callback.accept(streamingEngineConfig.get())); + Future unusedFuture = + singleThreadExecutor.submit( + () -> { + try { + callback.accept(streamingEngineConfig.get()); + } catch (Exception e) { + LOG.error("Exception from StreamingGlobalConfig callback", e); + } + }); } } From 6dc6a2f525290e07b4aed6f1f83feacc80b8c44a Mon Sep 17 00:00:00 2001 From: Arun Pandian Date: Tue, 17 Sep 2024 18:27:43 -0700 Subject: [PATCH 09/13] Sequence and run config callbacks --- .../StreamingGlobalConfigHandleImpl.java | 89 ++++++++++--------- 1 file changed, 49 insertions(+), 40 deletions(-) diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingGlobalConfigHandleImpl.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingGlobalConfigHandleImpl.java index be0faa03b718..fbd699aee839 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingGlobalConfigHandleImpl.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingGlobalConfigHandleImpl.java @@ -17,12 +17,8 @@ */ package org.apache.beam.runners.dataflow.worker.streaming.config; -import java.util.ArrayList; -import java.util.Iterator; -import java.util.List; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.Future; +import java.util.concurrent.CopyOnWriteArrayList; +import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Consumer; import javax.annotation.Nonnull; @@ -30,24 +26,15 @@ import javax.annotation.concurrent.ThreadSafe; import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.vendor.grpc.v1p60p1.com.google.common.base.Preconditions; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; @Internal @ThreadSafe public class StreamingGlobalConfigHandleImpl implements StreamingGlobalConfigHandle { - private static final Logger LOG = LoggerFactory.getLogger(StreamingGlobalConfigHandleImpl.class); private final AtomicReference streamingEngineConfig = new AtomicReference<>(); - @GuardedBy("this") - private final List> config_callbacks = new ArrayList<>(); - - // Using a single threaded executor to call callbacks in the scheduled order. - private final ExecutorService singleThreadExecutor = - Executors.newSingleThreadExecutor( - r -> new Thread(r, "StreamingGlobalConfigHandleImpl Executor")); + private final CopyOnWriteArrayList configCallbacks = new CopyOnWriteArrayList<>(); @Override public StreamingGlobalConfig getConfig() { @@ -59,37 +46,59 @@ public StreamingGlobalConfig getConfig() { @Override public void registerConfigObserver(@Nonnull Consumer callback) { - synchronized (this) { - config_callbacks.add(callback); - // If the config is already set, schedule a callback - if (streamingEngineConfig.get() != null) { - scheduleConfigCallback(callback); - } + ConfigCallback configCallback = new ConfigCallback(callback); + configCallbacks.add(configCallback); + if (streamingEngineConfig.get() != null) { + configCallback.run(); } } void setConfig(@Nonnull StreamingGlobalConfig config) { - Iterator> iterator; - synchronized (this) { - if (config.equals(streamingEngineConfig.get())) { + if (config.equals(streamingEngineConfig.get())) { + return; + } + streamingEngineConfig.set(config); + for (ConfigCallback configCallback : configCallbacks) { + configCallback.run(); + } + } + + + private class ConfigCallback { + + private final AtomicInteger queuedOrRunning = new AtomicInteger(0); + private final Consumer configConsumer; + + private ConfigCallback(Consumer configConsumer) { + this.configConsumer = configConsumer; + } + + /** + * Runs the passed in callback with the latest config. Overlapping `run()` calls will be + * collapsed into one. If the callback is already running a new call will be scheduled to run + * after the current execution completes, on the same thread which ran the previous run. + */ + private void run() { + // If the callback is already running, + // Increment queued and return. The thread running + // the callback will run it again with the latest config. + if (queuedOrRunning.incrementAndGet() > 1) { return; } - streamingEngineConfig.set(config); - for (Consumer callback : config_callbacks) { - scheduleConfigCallback(callback); + // Else run the callback + while (true) { + configConsumer.accept(StreamingGlobalConfigHandleImpl.this.streamingEngineConfig.get()); + if (queuedOrRunning.updateAndGet(queuedOrRunning -> { + if (queuedOrRunning == 1) { + // If there are no queued requests stop processing. + return 0; + } + // Else, clear queue, set 1 running and run the callback + return 1; + }) == 0) { + break; + } } } } - - private void scheduleConfigCallback(Consumer callback) { - Future unusedFuture = - singleThreadExecutor.submit( - () -> { - try { - callback.accept(streamingEngineConfig.get()); - } catch (Exception e) { - LOG.error("Exception from StreamingGlobalConfig callback", e); - } - }); - } } From 069acca7562be657a6cd9950afd32d8b28eb9445 Mon Sep 17 00:00:00 2001 From: Arun Pandian Date: Tue, 17 Sep 2024 18:44:12 -0700 Subject: [PATCH 10/13] add test for concurrent setConfig --- .../config/StreamingGlobalConfigHandle.java | 2 +- .../StreamingGlobalConfigHandleImpl.java | 20 +++++----- .../StreamingGlobalConfigHandleImplTest.java | 39 +++++++++++++++++++ 3 files changed, 50 insertions(+), 11 deletions(-) diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingGlobalConfigHandle.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingGlobalConfigHandle.java index abc60c0c3ce4..6f75ba887473 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingGlobalConfigHandle.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingGlobalConfigHandle.java @@ -31,7 +31,7 @@ public interface StreamingGlobalConfigHandle { /** * Subscribe to config updates by registering a callback. Callback should be called the first time - * with settings, if any. + * with settings, if any. The callback could execute inline before the method returns. */ void registerConfigObserver(@Nonnull Consumer callback); } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingGlobalConfigHandleImpl.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingGlobalConfigHandleImpl.java index fbd699aee839..6896de2ab3a0 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingGlobalConfigHandleImpl.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingGlobalConfigHandleImpl.java @@ -22,7 +22,6 @@ import java.util.concurrent.atomic.AtomicReference; import java.util.function.Consumer; import javax.annotation.Nonnull; -import javax.annotation.concurrent.GuardedBy; import javax.annotation.concurrent.ThreadSafe; import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.vendor.grpc.v1p60p1.com.google.common.base.Preconditions; @@ -63,7 +62,6 @@ void setConfig(@Nonnull StreamingGlobalConfig config) { } } - private class ConfigCallback { private final AtomicInteger queuedOrRunning = new AtomicInteger(0); @@ -88,14 +86,16 @@ private void run() { // Else run the callback while (true) { configConsumer.accept(StreamingGlobalConfigHandleImpl.this.streamingEngineConfig.get()); - if (queuedOrRunning.updateAndGet(queuedOrRunning -> { - if (queuedOrRunning == 1) { - // If there are no queued requests stop processing. - return 0; - } - // Else, clear queue, set 1 running and run the callback - return 1; - }) == 0) { + if (queuedOrRunning.updateAndGet( + queuedOrRunning -> { + if (queuedOrRunning == 1) { + // If there are no queued requests stop processing. + return 0; + } + // Else, clear queue, set 1 running and run the callback + return 1; + }) + == 0) { break; } } diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingGlobalConfigHandleImplTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingGlobalConfigHandleImplTest.java index aaab58003e66..42a2e9fa9bda 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingGlobalConfigHandleImplTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingGlobalConfigHandleImplTest.java @@ -20,6 +20,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; +import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; @@ -150,4 +151,42 @@ public void registerConfigObserver_shouldNotCallCallbackForIfConfigRemainsSame() Thread.sleep(TimeUnit.SECONDS.toMillis(10)); assertEquals(1, callbackCount.get()); } + + @Test + public void registerConfigObserver_updateConfigWhenCallbackIsRunning() + throws InterruptedException { + CountDownLatch latch = new CountDownLatch(2); + StreamingGlobalConfigHandleImpl globalConfigHandle = new StreamingGlobalConfigHandleImpl(); + StreamingGlobalConfig initialConfig = + StreamingGlobalConfig.builder() + .setOperationalLimits(OperationalLimits.builder().setMaxOutputValueBytes(4569).build()) + .build(); + StreamingGlobalConfig updatedConfig = + StreamingGlobalConfig.builder() + .setOperationalLimits( + OperationalLimits.builder() + .setMaxOutputValueBytes(123) + .setMaxOutputKeyBytes(324) + .setMaxWorkItemCommitBytes(456) + .build()) + .setWindmillServiceEndpoints(ImmutableSet.of(HostAndPort.fromHost("windmillHost"))) + .setUserWorkerJobSettings( + UserWorkerRunnerV1Settings.newBuilder() + .setUseSeparateWindmillHeartbeatStreams(false) + .build()) + .build(); + CopyOnWriteArrayList configsFromCallback = new CopyOnWriteArrayList<>(); + globalConfigHandle.registerConfigObserver( + config -> { + configsFromCallback.add(config); + if (globalConfigHandle.getConfig().equals(config)) { + globalConfigHandle.setConfig(updatedConfig); + } + latch.countDown(); + }); + globalConfigHandle.setConfig(initialConfig); + assertTrue(latch.await(10, TimeUnit.SECONDS)); + assertEquals(configsFromCallback.get(0), initialConfig); + assertEquals(configsFromCallback.get(1), updatedConfig); + } } From 4e6eae71fc31cb9fe33a2b2807e2af6d43495e0e Mon Sep 17 00:00:00 2001 From: Arun Pandian Date: Tue, 17 Sep 2024 19:04:10 -0700 Subject: [PATCH 11/13] improve tests --- .../StreamingGlobalConfigHandleImplTest.java | 30 ++++++++++++++----- 1 file changed, 22 insertions(+), 8 deletions(-) diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingGlobalConfigHandleImplTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingGlobalConfigHandleImplTest.java index 42a2e9fa9bda..387b12c306cf 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingGlobalConfigHandleImplTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingGlobalConfigHandleImplTest.java @@ -61,7 +61,7 @@ public void getConfig() { @Test public void registerConfigObserver_configSetAfterRegisteringCallback() throws InterruptedException { - CountDownLatch latch = new CountDownLatch(1); + CountDownLatch latch = new CountDownLatch(2); StreamingGlobalConfigHandleImpl globalConfigHandle = new StreamingGlobalConfigHandleImpl(); StreamingGlobalConfig configToSet = StreamingGlobalConfig.builder() @@ -77,21 +77,28 @@ public void registerConfigObserver_configSetAfterRegisteringCallback() .setUseSeparateWindmillHeartbeatStreams(false) .build()) .build(); - AtomicReference configFromCallback = new AtomicReference<>(); + AtomicReference configFromCallback1 = new AtomicReference<>(); + AtomicReference configFromCallback2 = new AtomicReference<>(); + globalConfigHandle.registerConfigObserver( + config -> { + configFromCallback1.set(config); + latch.countDown(); + }); globalConfigHandle.registerConfigObserver( config -> { - configFromCallback.set(config); + configFromCallback2.set(config); latch.countDown(); }); globalConfigHandle.setConfig(configToSet); assertTrue(latch.await(10, TimeUnit.SECONDS)); - assertEquals(configFromCallback.get(), globalConfigHandle.getConfig()); + assertEquals(configFromCallback1.get(), globalConfigHandle.getConfig()); + assertEquals(configFromCallback2.get(), globalConfigHandle.getConfig()); } @Test public void registerConfigObserver_configSetBeforeRegisteringCallback() throws InterruptedException { - CountDownLatch latch = new CountDownLatch(1); + CountDownLatch latch = new CountDownLatch(2); StreamingGlobalConfigHandleImpl globalConfigHandle = new StreamingGlobalConfigHandleImpl(); StreamingGlobalConfig configToSet = StreamingGlobalConfig.builder() @@ -107,15 +114,22 @@ public void registerConfigObserver_configSetBeforeRegisteringCallback() .setUseSeparateWindmillHeartbeatStreams(false) .build()) .build(); - AtomicReference configFromCallback = new AtomicReference<>(); + AtomicReference configFromCallback1 = new AtomicReference<>(); + AtomicReference configFromCallback2 = new AtomicReference<>(); globalConfigHandle.setConfig(configToSet); globalConfigHandle.registerConfigObserver( config -> { - configFromCallback.set(config); + configFromCallback1.set(config); + latch.countDown(); + }); + globalConfigHandle.registerConfigObserver( + config -> { + configFromCallback2.set(config); latch.countDown(); }); assertTrue(latch.await(10, TimeUnit.SECONDS)); - assertEquals(configFromCallback.get(), globalConfigHandle.getConfig()); + assertEquals(configFromCallback1.get(), globalConfigHandle.getConfig()); + assertEquals(configFromCallback2.get(), globalConfigHandle.getConfig()); } @Test From ce1f07e0525127569b0f7b6b8bdf78d5ebfca825 Mon Sep 17 00:00:00 2001 From: Arun Pandian Date: Tue, 17 Sep 2024 21:45:18 -0700 Subject: [PATCH 12/13] catch exceptions from callback --- .../StreamingGlobalConfigHandleImpl.java | 10 ++- .../StreamingGlobalConfigHandleImplTest.java | 70 +++++++++++++++++++ 2 files changed, 79 insertions(+), 1 deletion(-) diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingGlobalConfigHandleImpl.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingGlobalConfigHandleImpl.java index 6896de2ab3a0..aa92ea319fe8 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingGlobalConfigHandleImpl.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingGlobalConfigHandleImpl.java @@ -25,11 +25,15 @@ import javax.annotation.concurrent.ThreadSafe; import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.vendor.grpc.v1p60p1.com.google.common.base.Preconditions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; @Internal @ThreadSafe public class StreamingGlobalConfigHandleImpl implements StreamingGlobalConfigHandle { + private static final Logger LOG = LoggerFactory.getLogger(StreamingGlobalConfigHandleImpl.class); + private final AtomicReference streamingEngineConfig = new AtomicReference<>(); @@ -85,7 +89,11 @@ private void run() { } // Else run the callback while (true) { - configConsumer.accept(StreamingGlobalConfigHandleImpl.this.streamingEngineConfig.get()); + try { + configConsumer.accept(StreamingGlobalConfigHandleImpl.this.streamingEngineConfig.get()); + } catch (Exception e) { + LOG.error("Exception running GlobalConfig callback", e); + } if (queuedOrRunning.updateAndGet( queuedOrRunning -> { if (queuedOrRunning == 1) { diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingGlobalConfigHandleImplTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingGlobalConfigHandleImplTest.java index 387b12c306cf..6a8ec6e4b1ee 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingGlobalConfigHandleImplTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingGlobalConfigHandleImplTest.java @@ -132,6 +132,76 @@ public void registerConfigObserver_configSetBeforeRegisteringCallback() assertEquals(configFromCallback2.get(), globalConfigHandle.getConfig()); } + @Test + public void registerConfigObserver_configSetBeforeRegisteringCallback_callbackThrowsException() + throws InterruptedException { + CountDownLatch latch = new CountDownLatch(2); + StreamingGlobalConfigHandleImpl globalConfigHandle = new StreamingGlobalConfigHandleImpl(); + StreamingGlobalConfig configToSet = + StreamingGlobalConfig.builder() + .setOperationalLimits( + OperationalLimits.builder() + .setMaxOutputValueBytes(123) + .setMaxOutputKeyBytes(324) + .setMaxWorkItemCommitBytes(456) + .build()) + .setWindmillServiceEndpoints(ImmutableSet.of(HostAndPort.fromHost("windmillHost"))) + .setUserWorkerJobSettings( + UserWorkerRunnerV1Settings.newBuilder() + .setUseSeparateWindmillHeartbeatStreams(false) + .build()) + .build(); + AtomicReference configFromCallback = new AtomicReference<>(); + globalConfigHandle.setConfig(configToSet); + globalConfigHandle.registerConfigObserver( + config -> { + latch.countDown(); + throw new RuntimeException(); + }); + globalConfigHandle.registerConfigObserver( + config -> { + configFromCallback.set(config); + latch.countDown(); + }); + assertTrue(latch.await(10, TimeUnit.SECONDS)); + assertEquals(configFromCallback.get(), configToSet); + } + + @Test + public void registerConfigObserver_configSetAfterRegisteringCallback_callbackThrowsException() + throws InterruptedException { + CountDownLatch latch = new CountDownLatch(2); + StreamingGlobalConfigHandleImpl globalConfigHandle = new StreamingGlobalConfigHandleImpl(); + StreamingGlobalConfig configToSet = + StreamingGlobalConfig.builder() + .setOperationalLimits( + OperationalLimits.builder() + .setMaxOutputValueBytes(123) + .setMaxOutputKeyBytes(324) + .setMaxWorkItemCommitBytes(456) + .build()) + .setWindmillServiceEndpoints(ImmutableSet.of(HostAndPort.fromHost("windmillHost"))) + .setUserWorkerJobSettings( + UserWorkerRunnerV1Settings.newBuilder() + .setUseSeparateWindmillHeartbeatStreams(false) + .build()) + .build(); + AtomicReference configFromCallback = new AtomicReference<>(); + globalConfigHandle.registerConfigObserver( + config -> { + latch.countDown(); + throw new RuntimeException(); + }); + globalConfigHandle.registerConfigObserver( + config -> { + configFromCallback.set(config); + latch.countDown(); + }); + globalConfigHandle.setConfig(configToSet); + assertTrue(latch.await(10, TimeUnit.SECONDS)); + assertEquals(configFromCallback.get(), configToSet); + } + @Test public void registerConfigObserver_shouldNotCallCallbackForIfConfigRemainsSame() throws InterruptedException { From c32695e4fb133a2267866d605896f1430627de6f Mon Sep 17 00:00:00 2001 From: Arun Pandian Date: Wed, 18 Sep 2024 03:23:41 -0700 Subject: [PATCH 13/13] review comments --- .../worker/StreamingDataflowWorker.java | 14 ++++++++++---- .../StreamingGlobalConfigHandleImpl.java | 1 + .../harness/StreamingWorkerStatusPages.java | 2 +- .../client/grpc/GrpcDispatcherClient.java | 1 + .../worker/StreamingDataflowWorkerTest.java | 6 +++--- .../StreamingGlobalConfigHandleImplTest.java | 19 ++++++++++++++++++- 6 files changed, 34 insertions(+), 9 deletions(-) diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java index bc9fd8228c70..0dedd4f34fd6 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java @@ -504,10 +504,16 @@ static StreamingDataflowWorker forTesting( executorSupplier) : new StreamingApplianceComputationConfigFetcher( windmillServer::getConfig, globalConfigHandle); - StreamingGlobalConfig config = configFetcher.getGlobalConfigHandle().getConfig(); - if (!config.windmillServiceEndpoints().isEmpty()) { - windmillServer.setWindmillServiceEndpoints(config.windmillServiceEndpoints()); - } + configFetcher + .getGlobalConfigHandle() + .registerConfigObserver( + config -> { + if (config.windmillServiceEndpoints().isEmpty()) { + LOG.warn("Received empty windmill service endpoints"); + return; + } + windmillServer.setWindmillServiceEndpoints(config.windmillServiceEndpoints()); + }); ConcurrentMap stateNameMap = new ConcurrentHashMap<>(prePopulatedStateNameMappings); ComputationStateCache computationStateCache = diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingGlobalConfigHandleImpl.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingGlobalConfigHandleImpl.java index aa92ea319fe8..9ed5c9fcf396 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingGlobalConfigHandleImpl.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingGlobalConfigHandleImpl.java @@ -40,6 +40,7 @@ public class StreamingGlobalConfigHandleImpl implements StreamingGlobalConfigHan private final CopyOnWriteArrayList configCallbacks = new CopyOnWriteArrayList<>(); @Override + @Nonnull public StreamingGlobalConfig getConfig() { Preconditions.checkState( streamingEngineConfig.get() != null, diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/harness/StreamingWorkerStatusPages.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/harness/StreamingWorkerStatusPages.java index e2fa2245a3f1..6981312eff1d 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/harness/StreamingWorkerStatusPages.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/harness/StreamingWorkerStatusPages.java @@ -161,7 +161,7 @@ private void addStreamingEngineStatusPages() { "jobSettings", "User Worker Job Settings", writer -> { - StreamingGlobalConfig config = globalConfig.get(); + @Nullable StreamingGlobalConfig config = globalConfig.get(); if (config == null) { writer.println("Job Settings not loaded."); return; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcDispatcherClient.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcDispatcherClient.java index 4a264b84db9c..412608ea3981 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcDispatcherClient.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcDispatcherClient.java @@ -150,6 +150,7 @@ public boolean hasInitializedEndpoints() { public void onJobConfig(StreamingGlobalConfig config) { if (config.windmillServiceEndpoints().isEmpty()) { + LOG.warn("Dispatcher client received empty windmill service endpoints from global config"); return; } consumeWindmillDispatcherEndpoints(config.windmillServiceEndpoints()); diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorkerTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorkerTest.java index 95d3a9f7ce55..dadf02171235 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorkerTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorkerTest.java @@ -277,7 +277,7 @@ public Long get() { @Rule public TestRule restoreMDC = new RestoreDataflowLoggingMDC(); @Rule public ErrorCollector errorCollector = new ErrorCollector(); WorkUnitClient mockWorkUnitClient = mock(WorkUnitClient.class); - StreamingGlobalConfigHandleImpl mockglobalConfigHandle = + StreamingGlobalConfigHandleImpl mockGlobalConfigHandle = mock(StreamingGlobalConfigHandleImpl.class); HotKeyLogger hotKeyLogger = mock(HotKeyLogger.class); @@ -840,7 +840,7 @@ private DataflowWorkerHarnessOptions createTestingPipelineOptions(String... args private StreamingDataflowWorker makeWorker( StreamingDataflowWorkerTestParams streamingDataflowWorkerTestParams) { - when(mockglobalConfigHandle.getConfig()) + when(mockGlobalConfigHandle.getConfig()) .thenReturn(streamingDataflowWorkerTestParams.streamingGlobalConfig()); StreamingDataflowWorker worker = StreamingDataflowWorker.forTesting( @@ -855,7 +855,7 @@ private StreamingDataflowWorker makeWorker( hotKeyLogger, streamingDataflowWorkerTestParams.clock(), streamingDataflowWorkerTestParams.executorSupplier(), - mockglobalConfigHandle, + mockGlobalConfigHandle, streamingDataflowWorkerTestParams.localRetryTimeoutMs()); this.computationStateCache = worker.getComputationStateCache(); return worker; diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingGlobalConfigHandleImplTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingGlobalConfigHandleImplTest.java index 6a8ec6e4b1ee..059f60731a7d 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingGlobalConfigHandleImplTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingGlobalConfigHandleImplTest.java @@ -56,6 +56,23 @@ public void getConfig() { .build(); globalConfigHandle.setConfig(config); assertEquals(config, globalConfigHandle.getConfig()); + + StreamingGlobalConfig updatedConfig = + StreamingGlobalConfig.builder() + .setOperationalLimits( + OperationalLimits.builder() + .setMaxOutputValueBytes(324) + .setMaxOutputKeyBytes(456) + .setMaxWorkItemCommitBytes(123) + .build()) + .setWindmillServiceEndpoints(ImmutableSet.of(HostAndPort.fromHost("windmillHost1"))) + .setUserWorkerJobSettings( + UserWorkerRunnerV1Settings.newBuilder() + .setUseSeparateWindmillHeartbeatStreams(true) + .build()) + .build(); + globalConfigHandle.setConfig(updatedConfig); + assertEquals(updatedConfig, globalConfigHandle.getConfig()); } @Test @@ -263,7 +280,7 @@ public void registerConfigObserver_updateConfigWhenCallbackIsRunning() globalConfigHandle.registerConfigObserver( config -> { configsFromCallback.add(config); - if (globalConfigHandle.getConfig().equals(config)) { + if (config.equals(initialConfig)) { globalConfigHandle.setConfig(updatedConfig); } latch.countDown();