From e990fb3e05df983a1805a4404e3eb060e2f28ffb Mon Sep 17 00:00:00 2001 From: Martin Trieu Date: Fri, 31 May 2024 19:33:30 -0700 Subject: [PATCH 01/13] integrate direct path/fanout worker harness components with StreamingDataflowWorker --- .../DataflowStreamingPipelineOptions.java | 19 +- .../MetricTrackingWindmillServerStub.java | 100 +++++- .../worker/StreamingDataflowWorker.java | 305 ++++++++++++------ .../worker/WorkItemCancelledException.java | 11 +- .../worker/streaming/ActiveWorkState.java | 18 ++ .../worker/streaming/ComputationState.java | 5 + .../dataflow/worker/streaming/StageInfo.java | 13 +- .../dataflow/worker/streaming/Work.java | 24 +- .../harness/StreamingWorkerStatusPages.java | 1 + .../StreamingWorkerStatusReporter.java | 80 ++--- .../windmill/WindmillServiceAddress.java | 20 +- .../client/AbstractWindmillStream.java | 18 +- .../windmill/client/WindmillStream.java | 2 + .../WindmillStreamCancelledException.java | 39 +++ .../windmill/client/grpc/ChannelzServlet.java | 27 +- .../client/grpc/GrpcDirectGetWorkStream.java | 54 ++-- .../client/grpc/GrpcGetDataStream.java | 68 +++- .../grpc/GrpcGetDataStreamRequests.java | 9 +- .../grpc/GrpcWindmillStreamFactory.java | 98 +++++- .../client/grpc/StreamingEngineClient.java | 128 ++++++-- .../client/grpc/WindmillStreamSender.java | 21 +- .../grpc/stubs/WindmillChannelFactory.java | 39 +-- .../budget/EvenGetWorkBudgetDistributor.java | 2 +- .../work/refresh/ActiveWorkRefreshers.java | 50 --- .../DispatchedActiveWorkRefresher.java | 46 ++- .../refresh/FanOutActiveWorkRefresher.java | 136 ++++++++ .../work/refresh/HeartbeatRequests.java | 113 +++++++ .../dataflow/worker/FakeWindmillServer.java | 15 + .../worker/StreamingDataflowWorkerTest.java | 10 +- .../StreamingWorkerStatusReporterTest.java | 13 +- .../client/WindmillStreamPoolTest.java | 5 + .../StreamingEngineWorkCommitterTest.java | 11 +- .../client/grpc/ChannelzServletTest.java | 6 +- .../grpc/StreamingEngineClientTest.java | 16 +- .../client/grpc/WindmillStreamSenderTest.java | 12 +- .../EvenGetWorkBudgetDistributorTest.java | 4 +- .../DispatchedActiveWorkRefresherTest.java | 2 +- .../FanOutActiveWorkRefresherTest.java | 221 +++++++++++++ .../work/refresh/HeartbeatRequestsTest.java | 289 +++++++++++++++++ 39 files changed, 1656 insertions(+), 394 deletions(-) create mode 100644 runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/WindmillStreamCancelledException.java delete mode 100644 runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/ActiveWorkRefreshers.java create mode 100644 runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/FanOutActiveWorkRefresher.java create mode 100644 runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/HeartbeatRequests.java create mode 100644 runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/FanOutActiveWorkRefresherTest.java create mode 100644 runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/HeartbeatRequestsTest.java diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowStreamingPipelineOptions.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowStreamingPipelineOptions.java index a761d38de1ab0..8fa93f3ca0e0e 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowStreamingPipelineOptions.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowStreamingPipelineOptions.java @@ -20,6 +20,7 @@ import org.apache.beam.sdk.options.Default; import org.apache.beam.sdk.options.DefaultValueFactory; import org.apache.beam.sdk.options.Description; +import org.apache.beam.sdk.options.ExperimentalOptions; import org.apache.beam.sdk.options.Hidden; import org.apache.beam.sdk.options.PipelineOptions; import org.joda.time.Duration; @@ -221,13 +222,11 @@ public interface DataflowStreamingPipelineOptions extends PipelineOptions { void setWindmillServiceStreamMaxBackoffMillis(int value); - @Description( - "If true, Dataflow streaming pipeline will be running in direct path mode." - + " VMs must have IPv6 enabled for this to work.") - @Default.Boolean(false) - boolean getIsWindmillServiceDirectPathEnabled(); + @Description("Enables direct path mode for streaming engine.") + @Default.InstanceFactory(EnableWindmillServiceDirectPathFactory.class) + boolean isEnableWindmillServiceDirectPath(); - void setIsWindmillServiceDirectPathEnabled(boolean isWindmillServiceDirectPathEnabled); + void setEnableWindmillServiceDirectPath(boolean value); /** * Factory for creating local Windmill address. Reads from system propery 'windmill.hostport' for @@ -302,4 +301,12 @@ public Integer create(PipelineOptions options) { return streamingOptions.isEnableStreamingEngine() ? Integer.MAX_VALUE : 1; } } + + /** EnableStreamingEngine defaults to false unless one of the two experiments is set. */ + class EnableWindmillServiceDirectPathFactory implements DefaultValueFactory { + @Override + public Boolean create(PipelineOptions options) { + return ExperimentalOptions.hasExperiment(options, "enable_windmill_service_direct_path"); + } + } } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/MetricTrackingWindmillServerStub.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/MetricTrackingWindmillServerStub.java index d808d4f4ab589..a6297d019f8e9 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/MetricTrackingWindmillServerStub.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/MetricTrackingWindmillServerStub.java @@ -23,7 +23,10 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; import java.util.concurrent.atomic.AtomicInteger; import javax.annotation.concurrent.GuardedBy; import org.apache.beam.runners.dataflow.worker.util.MemoryMonitor; @@ -31,11 +34,15 @@ import org.apache.beam.runners.dataflow.worker.windmill.Windmill.HeartbeatRequest; import org.apache.beam.runners.dataflow.worker.windmill.WindmillServerStub; import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStream.GetDataStream; +import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStreamCancelledException; import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStreamPool; import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.SettableFuture; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.ThreadFactoryBuilder; import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.Duration; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Wrapper around a {@link WindmillServerStub} that tracks metrics for the number of in-flight @@ -49,6 +56,8 @@ "nullness" // TODO(https://github.com/apache/beam/issues/20497) }) public class MetricTrackingWindmillServerStub { + private static final Logger LOG = LoggerFactory.getLogger(MetricTrackingWindmillServerStub.class); + private static final String FAN_OUT_REFRESH_WORK_EXECUTOR = "FanOutActiveWorkRefreshExecutor"; private static final int MAX_READS_PER_BATCH = 60; private static final int MAX_ACTIVE_READS = 10; @@ -59,8 +68,8 @@ public class MetricTrackingWindmillServerStub { private final WindmillServerStub server; private final MemoryMonitor gcThrashingMonitor; private final boolean useStreamingRequests; - private final WindmillStreamPool getDataStreamPool; + private final ExecutorService fanOutActiveWorkRefreshExecutor; // This may be the same instance as getDataStreamPool based upon options. private final WindmillStreamPool heartbeatStreamPool; @@ -106,6 +115,9 @@ public static Builder builder(WindmillServerStub server, MemoryMonitor gcThrashi this.server = server; this.gcThrashingMonitor = gcThrashingMonitor; this.useStreamingRequests = useStreamingRequests; + this.fanOutActiveWorkRefreshExecutor = + Executors.newCachedThreadPool( + new ThreadFactoryBuilder().setNameFormat(FAN_OUT_REFRESH_WORK_EXECUTOR).build()); if (useStreamingRequests) { getDataStreamPool = WindmillStreamPool.create( @@ -254,6 +266,27 @@ public Windmill.KeyedGetDataResponse getStateData( } } + public Windmill.KeyedGetDataResponse getStateData( + GetDataStream getDataStream, String computation, Windmill.KeyedGetDataRequest request) { + gcThrashingMonitor.waitForResources("GetStateData"); + activeStateReads.getAndIncrement(); + if (getDataStream.isClosed()) { + throw new WorkItemCancelledException(request.getShardingKey()); + } + + try { + return getDataStream.requestKeyedData(computation, request); + } catch (Exception e) { + if (WindmillStreamCancelledException.isWindmillStreamCancelledException(e)) { + LOG.error("Tried to fetch keyed data from a closed stream. Work has been cancelled", e); + throw new WorkItemCancelledException(request.getShardingKey()); + } + throw new RuntimeException(e); + } finally { + activeStateReads.getAndDecrement(); + } + } + public Windmill.GlobalData getSideInputData(Windmill.GlobalDataRequest request) { gcThrashingMonitor.waitForResources("GetSideInputData"); activeSideInputs.getAndIncrement(); @@ -278,6 +311,19 @@ public Windmill.GlobalData getSideInputData(Windmill.GlobalDataRequest request) } } + public Windmill.GlobalData getSideInputData( + GetDataStream getDataStream, Windmill.GlobalDataRequest request) { + gcThrashingMonitor.waitForResources("GetSideInputData"); + activeSideInputs.getAndIncrement(); + try { + return getDataStream.requestGlobalData(request); + } catch (Exception e) { + throw new RuntimeException("Failed to get side input: ", e); + } finally { + activeSideInputs.getAndDecrement(); + } + } + /** Tells windmill processing is ongoing for the given keys. */ public void refreshActiveWork(Map> heartbeats) { if (heartbeats.isEmpty()) { @@ -319,6 +365,58 @@ public void refreshActiveWork(Map> heartbeats) { } } + /** + * Attempts to refresh active work, fanning out to each {@link GetDataStream} in parallel. + * + * @implNote Skips closed {@link GetDataStream}(s). + */ + public void refreshActiveWorkWithFanOut( + Map>> heartbeats) { + if (heartbeats.isEmpty()) { + return; + } + try { + List> fanOutRefreshActiveWork = new ArrayList<>(); + for (Map.Entry>> heartbeat : + heartbeats.entrySet()) { + GetDataStream stream = heartbeat.getKey(); + Map> heartbeatRequests = heartbeat.getValue(); + if (stream.isClosed()) { + LOG.warn( + "Trying to refresh work on stream={} after work has moved off of worker." + + " heartbeats={}", + stream, + heartbeatRequests); + } else { + fanOutRefreshActiveWork.add(sendHeartbeatOnStreamFuture(heartbeat)); + } + } + + // Don't block until we kick off all the refresh active work RPCs. + @SuppressWarnings("rawtypes") + CompletableFuture parallelFanOutRefreshActiveWork = + CompletableFuture.allOf(fanOutRefreshActiveWork.toArray(new CompletableFuture[0])); + parallelFanOutRefreshActiveWork.join(); + } finally { + activeHeartbeats.set(0); + } + } + + private CompletableFuture sendHeartbeatOnStreamFuture( + Map.Entry>> heartbeat) { + return CompletableFuture.runAsync( + () -> { + GetDataStream stream = heartbeat.getKey(); + Map> heartbeatRequests = heartbeat.getValue(); + activeHeartbeats.getAndUpdate(existing -> existing + heartbeat.getValue().size()); + stream.refreshActiveWork(heartbeatRequests); + // Active heartbeats should never drop below 0. + activeHeartbeats.getAndUpdate( + existing -> Math.max(existing - heartbeat.getValue().size(), 0)); + }, + fanOutActiveWorkRefreshExecutor); + } + public void printHtml(PrintWriter writer) { writer.println("Active Fetches:"); writer.println(" Side Inputs: " + activeSideInputs.get()); 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 fc1be2cd13727..4f41f85921cdc 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 @@ -18,7 +18,6 @@ package org.apache.beam.runners.dataflow.worker; import static org.apache.beam.runners.dataflow.worker.windmill.client.grpc.stubs.WindmillChannelFactory.remoteChannel; -import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; import com.google.api.services.dataflow.model.CounterUpdate; import com.google.api.services.dataflow.model.MapTask; @@ -39,6 +38,7 @@ import java.util.function.Consumer; import java.util.function.Function; import java.util.function.Supplier; +import javax.annotation.Nullable; import org.apache.beam.runners.core.metrics.MetricsLogger; import org.apache.beam.runners.dataflow.DataflowRunner; import org.apache.beam.runners.dataflow.options.DataflowWorkerHarnessOptions; @@ -66,6 +66,7 @@ import org.apache.beam.runners.dataflow.worker.windmill.WindmillServerStub; import org.apache.beam.runners.dataflow.worker.windmill.WindmillServiceAddress; import org.apache.beam.runners.dataflow.worker.windmill.appliance.JniWindmillApplianceServer; +import org.apache.beam.runners.dataflow.worker.windmill.client.CloseableStream; import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStream.GetWorkStream; import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStreamPool; import org.apache.beam.runners.dataflow.worker.windmill.client.commits.CompleteCommit; @@ -76,24 +77,27 @@ import org.apache.beam.runners.dataflow.worker.windmill.client.grpc.GrpcDispatcherClient; import org.apache.beam.runners.dataflow.worker.windmill.client.grpc.GrpcWindmillServer; import org.apache.beam.runners.dataflow.worker.windmill.client.grpc.GrpcWindmillStreamFactory; +import org.apache.beam.runners.dataflow.worker.windmill.client.grpc.StreamingEngineClient; import org.apache.beam.runners.dataflow.worker.windmill.client.grpc.stubs.ChannelCache; import org.apache.beam.runners.dataflow.worker.windmill.client.grpc.stubs.ChannelCachingRemoteStubFactory; import org.apache.beam.runners.dataflow.worker.windmill.client.grpc.stubs.ChannelCachingStubFactory; import org.apache.beam.runners.dataflow.worker.windmill.client.grpc.stubs.IsolationChannel; import org.apache.beam.runners.dataflow.worker.windmill.state.WindmillStateCache; +import org.apache.beam.runners.dataflow.worker.windmill.work.budget.GetWorkBudget; +import org.apache.beam.runners.dataflow.worker.windmill.work.budget.GetWorkBudgetDistributors; import org.apache.beam.runners.dataflow.worker.windmill.work.processing.StreamingWorkScheduler; import org.apache.beam.runners.dataflow.worker.windmill.work.processing.failures.FailureTracker; import org.apache.beam.runners.dataflow.worker.windmill.work.processing.failures.StreamingApplianceFailureTracker; import org.apache.beam.runners.dataflow.worker.windmill.work.processing.failures.StreamingEngineFailureTracker; import org.apache.beam.runners.dataflow.worker.windmill.work.processing.failures.WorkFailureProcessor; import org.apache.beam.runners.dataflow.worker.windmill.work.refresh.ActiveWorkRefresher; -import org.apache.beam.runners.dataflow.worker.windmill.work.refresh.ActiveWorkRefreshers; +import org.apache.beam.runners.dataflow.worker.windmill.work.refresh.DispatchedActiveWorkRefresher; +import org.apache.beam.runners.dataflow.worker.windmill.work.refresh.FanOutActiveWorkRefresher; import org.apache.beam.sdk.fn.IdGenerator; import org.apache.beam.sdk.fn.IdGenerators; import org.apache.beam.sdk.fn.JvmInitializers; import org.apache.beam.sdk.io.FileSystems; import org.apache.beam.sdk.io.gcp.bigquery.BigQuerySinkMetrics; -import org.apache.beam.sdk.metrics.MetricName; import org.apache.beam.sdk.metrics.MetricsEnvironment; import org.apache.beam.sdk.util.construction.CoderTranslation; import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.ManagedChannel; @@ -114,13 +118,6 @@ }) public class StreamingDataflowWorker { - // TODO(https://github.com/apache/beam/issues/19632): Update throttling counters to use generic - // throttling-msecs metric. - public static final MetricName BIGQUERY_STREAMING_INSERT_THROTTLE_TIME = - MetricName.named( - "org.apache.beam.sdk.io.gcp.bigquery.BigQueryServicesImpl$DatasetServiceImpl", - "throttling-msecs"); - /** * Sinks are marked 'full' in {@link StreamingModeExecutionContext} once the amount of data sinked * (across all the sinks, if there are more than one) reaches this limit. This serves as hint for @@ -129,11 +126,13 @@ public class StreamingDataflowWorker { */ public static final int MAX_SINK_BYTES = 10_000_000; - // Maximum number of threads for processing. Currently, each thread processes one key at a time. + // Maximum number of threads for processing. Currently, each thread processes one key at a time. static final int MAX_PROCESSING_THREADS = 300; static final long THREAD_EXPIRATION_TIME_SEC = 60; static final int GET_WORK_STREAM_TIMEOUT_MINUTES = 3; static final Duration COMMIT_STREAM_TIMEOUT = Duration.standardMinutes(1); + private static final String BEAM_FN_API_EXPERIMENT = "beam_fn_api"; + private static final String ENABLE_IPV6_EXPERIMENT = "enable_private_ipv6_google_access"; private static final Logger LOG = LoggerFactory.getLogger(StreamingDataflowWorker.class); /** The idGenerator to generate unique id globally. */ @@ -168,9 +167,11 @@ public class StreamingDataflowWorker { private final StreamingWorkerStatusReporter workerStatusReporter; private final StreamingCounters streamingCounters; private final StreamingWorkScheduler streamingWorkScheduler; + private @Nullable StreamingEngineClient streamingEngineClient; private StreamingDataflowWorker( WindmillServerStub windmillServer, + WorkUnitClient dataflowServiceClient, long clientId, ComputationConfig.Fetcher configFetcher, ComputationStateCache computationStateCache, @@ -180,13 +181,14 @@ private StreamingDataflowWorker( DataflowWorkerHarnessOptions options, HotKeyLogger hotKeyLogger, Supplier clock, - StreamingWorkerStatusReporter workerStatusReporter, FailureTracker failureTracker, WorkFailureProcessor workFailureProcessor, StreamingCounters streamingCounters, MemoryMonitor memoryMonitor, AtomicInteger maxWorkItemCommitBytes, + ChannelCachingStubFactory stubFactory, GrpcWindmillStreamFactory windmillStreamFactory, + GrpcDispatcherClient dispatcherClient, Function executorSupplier, ConcurrentMap stageInfoMap) { this.configFetcher = configFetcher; @@ -248,22 +250,105 @@ private StreamingDataflowWorker( // Register standard file systems. FileSystems.setDefaultPipelineOptions(options); + WorkerStatusPages workerStatusPages = + WorkerStatusPages.create(DEFAULT_STATUS_PORT, memoryMonitor); + + this.streamingCounters = streamingCounters; + this.memoryMonitor = memoryMonitor; + + this.streamingEngineClient = null; + this.streamingWorkScheduler = + StreamingWorkScheduler.create( + options, + clock, + readerCache, + mapTaskExecutorFactory, + workUnitExecutor, + stateCache::forComputation, + request -> + streamingEngineClient != null + ? metricTrackingWindmillServer.getSideInputData( + streamingEngineClient.getGlobalDataStream(request.getDataId().getTag()), + request) + : metricTrackingWindmillServer.getSideInputData(request), + failureTracker, + workFailureProcessor, + streamingCounters, + hotKeyLogger, + sampler, + maxWorkItemCommitBytes, + ID_GENERATOR, + stageInfoMap); + int stuckCommitDurationMillis = windmillServiceEnabled && options.getStuckCommitDurationMillis() > 0 ? options.getStuckCommitDurationMillis() : 0; - this.activeWorkRefresher = - ActiveWorkRefreshers.createDispatchedActiveWorkRefresher( - clock, - options.getActiveWorkRefreshPeriodMillis(), - stuckCommitDurationMillis, - computationStateCache::getAllPresentComputations, - sampler, - metricTrackingWindmillServer::refreshActiveWork, - executorSupplier.apply("RefreshWork")); + if (isDirectPathPipeline(options)) { + this.streamingEngineClient = + StreamingEngineClient.create( + JobHeader.newBuilder() + .setJobId(options.getJobId()) + .setProjectId(options.getProject()) + .setWorkerId(options.getWorkerId()) + .setClientId(clientId) + .build(), + GetWorkBudget.builder() + .setItems(chooseMaximumBundlesOutstanding(options)) + .setBytes(MAX_GET_WORK_FETCH_BYTES) + .build(), + windmillStreamFactory, + (workItem, + watermarks, + processingContext, + ackWorkItemQueued, + getWorkStreamLatencies) -> + computationStateCache + .get(processingContext.computationId()) + .ifPresent( + computationState -> { + streamingWorkScheduler.scheduleWork( + computationState, + workItem, + watermarks, + processingContext, + getWorkStreamLatencies); + ackWorkItemQueued.accept(workItem); + }), + stubFactory, + GetWorkBudgetDistributors.distributeEvenly( + computationStateCache::totalCurrentActiveGetWorkBudget), + dispatcherClient, + commitWorkStream -> + StreamingEngineWorkCommitter.create( + () -> CloseableStream.create(commitWorkStream, () -> {}), + Math.max(options.getWindmillServiceCommitThreads(), 1), + this::onCompleteCommit), + getDataStream -> + (computationId, request) -> + metricTrackingWindmillServer.getStateData( + getDataStream, computationId, request)); + this.activeWorkRefresher = + FanOutActiveWorkRefresher.create( + clock, + options.getActiveWorkRefreshPeriodMillis(), + stuckCommitDurationMillis, + computationStateCache::getAllPresentComputations, + sampler, + metricTrackingWindmillServer::refreshActiveWorkWithFanOut); + } else { + this.activeWorkRefresher = + DispatchedActiveWorkRefresher.create( + clock, + options.getActiveWorkRefreshPeriodMillis(), + stuckCommitDurationMillis, + computationStateCache::getAllPresentComputations, + sampler, + metricTrackingWindmillServer::refreshActiveWork, + executorSupplier.apply("RefreshWork")); + this.streamingEngineClient = null; + } - WorkerStatusPages workerStatusPages = - WorkerStatusPages.create(DEFAULT_STATUS_PORT, memoryMonitor); StreamingWorkerStatusPages.Builder statusPagesBuilder = StreamingWorkerStatusPages.builder() .setClock(clock) @@ -272,7 +357,10 @@ private StreamingDataflowWorker( .setStatusPages(workerStatusPages) .setStateCache(stateCache) .setComputationStateCache(computationStateCache) - .setCurrentActiveCommitBytes(workCommitter::currentActiveCommitBytes) + .setCurrentActiveCommitBytes( + streamingEngineClient != null + ? streamingEngineClient::currentActiveCommitBytes + : workCommitter::currentActiveCommitBytes) .setGetDataStatusProvider(metricTrackingWindmillServer::printHtml) .setWorkUnitExecutor(workUnitExecutor); @@ -281,34 +369,36 @@ private StreamingDataflowWorker( ? statusPagesBuilder .setDebugCapture( new DebugCapture.Manager(options, workerStatusPages.getDebugCapturePages())) - .setChannelzServlet(new ChannelzServlet(CHANNELZ_PATH, options, windmillServer)) + .setChannelzServlet( + new ChannelzServlet( + CHANNELZ_PATH, + options, + streamingEngineClient != null + ? streamingEngineClient::currentWindmillEndpoints + : windmillServer::getWindmillServiceEndpoints)) .setWindmillStreamFactory(windmillStreamFactory) .build() : statusPagesBuilder.build(); - this.workerStatusReporter = workerStatusReporter; - this.streamingCounters = streamingCounters; - this.memoryMonitor = memoryMonitor; - - this.streamingWorkScheduler = - StreamingWorkScheduler.create( - options, - clock, - readerCache, - mapTaskExecutorFactory, - workUnitExecutor, - stateCache::forComputation, - metricTrackingWindmillServer::getSideInputData, + this.workerStatusReporter = + StreamingWorkerStatusReporter.create( + dataflowServiceClient, + streamingEngineClient != null + ? streamingEngineClient::getAndResetThrottleTimes + : windmillServer::getAndResetThrottleTime, + stageInfoMap::values, failureTracker, - workFailureProcessor, streamingCounters, - hotKeyLogger, - sampler, - maxWorkItemCommitBytes, - ID_GENERATOR, - stageInfoMap); + memoryMonitor, + workUnitExecutor, + executorSupplier, + options.getWindmillHarnessUpdateReportingPeriod().getMillis(), + options.getPerWorkerMetricsUpdateReportingPeriodMillis()); - LOG.debug("windmillServiceEnabled: {}", windmillServiceEnabled); + LOG.debug( + "windmillServiceEnabled: {}; isDirectPathEnabled: {}", + windmillServiceEnabled, + options.isEnableWindmillServiceDirectPath()); LOG.debug("WindmillServiceEndpoint: {}", options.getWindmillServiceEndpoint()); LOG.debug("WindmillServicePort: {}", options.getWindmillServicePort()); LOG.debug("LocalWindmillHostport: {}", options.getLocalWindmillHostport()); @@ -331,6 +421,8 @@ public static StreamingDataflowWorker fromOptions(DataflowWorkerHarnessOptions o new ThreadFactoryBuilder().setNameFormat(threadName).build()); GrpcWindmillStreamFactory.Builder windmillStreamFactoryBuilder = createGrpcwindmillStreamFactoryBuilder(options, clientId); + ChannelCachingStubFactory stubFactory = createStubFactory(options); + GrpcDispatcherClient dispatcherClient = GrpcDispatcherClient.create(stubFactory); ConfigFetcherComputationStateCacheAndWindmillClient configFetcherComputationStateCacheAndWindmillClient = @@ -367,20 +459,10 @@ public static StreamingDataflowWorker fromOptions(DataflowWorkerHarnessOptions o failureTracker, () -> Optional.ofNullable(memoryMonitor.tryToDumpHeap()), clock); - StreamingWorkerStatusReporter workerStatusReporter = - StreamingWorkerStatusReporter.create( - dataflowServiceClient, - windmillServer::getAndResetThrottleTime, - stageInfo::values, - failureTracker, - streamingCounters, - memoryMonitor, - workExecutor, - options.getWindmillHarnessUpdateReportingPeriod().getMillis(), - options.getPerWorkerMetricsUpdateReportingPeriodMillis()); return new StreamingDataflowWorker( windmillServer, + dataflowServiceClient, clientId, configFetcherComputationStateCacheAndWindmillClient.configFetcher(), computationStateCache, @@ -390,13 +472,14 @@ public static StreamingDataflowWorker fromOptions(DataflowWorkerHarnessOptions o options, new HotKeyLogger(), clock, - workerStatusReporter, failureTracker, workFailureProcessor, streamingCounters, memoryMonitor, maxWorkItemCommitBytes, + stubFactory, configFetcherComputationStateCacheAndWindmillClient.windmillStreamFactory(), + dispatcherClient, executorSupplier, stageInfo); } @@ -463,13 +546,14 @@ public static StreamingDataflowWorker fromOptions(DataflowWorkerHarnessOptions o @VisibleForTesting static StreamingDataflowWorker forTesting( + ChannelCachingStubFactory stubFactory, + GrpcDispatcherClient dispatcherClient, Map prePopulatedStateNameMappings, WindmillServerStub windmillServer, List mapTasks, DataflowMapTaskExecutorFactory mapTaskExecutorFactory, WorkUnitClient workUnitClient, DataflowWorkerHarnessOptions options, - boolean publishCounters, HotKeyLogger hotKeyLogger, Supplier clock, Function executorSupplier, @@ -523,19 +607,6 @@ static StreamingDataflowWorker forTesting( () -> Optional.ofNullable(memoryMonitor.tryToDumpHeap()), clock, localRetryTimeoutMs); - StreamingWorkerStatusReporter workerStatusReporter = - StreamingWorkerStatusReporter.forTesting( - publishCounters, - workUnitClient, - windmillServer::getAndResetThrottleTime, - stageInfo::values, - failureTracker, - streamingCounters, - memoryMonitor, - workExecutor, - executorSupplier, - options.getWindmillHarnessUpdateReportingPeriod().getMillis(), - options.getPerWorkerMetricsUpdateReportingPeriodMillis()); GrpcWindmillStreamFactory.Builder windmillStreamFactory = createGrpcwindmillStreamFactoryBuilder(options, 1) @@ -544,6 +615,7 @@ static StreamingDataflowWorker forTesting( return new StreamingDataflowWorker( windmillServer, + workUnitClient, 1L, configFetcher, computationStateCache, @@ -553,18 +625,19 @@ static StreamingDataflowWorker forTesting( options, hotKeyLogger, clock, - workerStatusReporter, failureTracker, workFailureProcessor, streamingCounters, memoryMonitor, maxWorkItemCommitBytes, + stubFactory, options.isEnableStreamingEngine() ? windmillStreamFactory .setHealthCheckIntervalMillis( options.getWindmillServiceStreamingRpcHealthCheckPeriodMs()) .build() : windmillStreamFactory.build(), + dispatcherClient, executorSupplier, stageInfo); } @@ -624,16 +697,7 @@ public static void main(String[] args) throws Exception { DataflowWorkerHarnessHelper.initializeGlobalStateAndPipelineOptions( StreamingDataflowWorker.class, DataflowWorkerHarnessOptions.class); DataflowWorkerHarnessHelper.configureLogging(options); - checkArgument( - options.isStreaming(), - "%s instantiated with options indicating batch use", - StreamingDataflowWorker.class.getName()); - - checkArgument( - !DataflowRunner.hasExperiment(options, "beam_fn_api"), - "%s cannot be main() class with beam_fn_api enabled", - StreamingDataflowWorker.class.getSimpleName()); - + validateWorkerOptions(options); CoderTranslation.verifyModelCodersRegistered(); LOG.debug("Creating StreamingDataflowWorker from options: {}", options); @@ -649,8 +713,37 @@ public static void main(String[] args) throws Exception { } JvmInitializers.runBeforeProcessing(options); - worker.startStatusPages(); - worker.start(); + try { + worker.startStatusPages(); + worker.start(); + } catch (Throwable e) { + LOG.error("Harness shutting down due to uncaught exception.", e); + worker.stop(); + } + } + + private static void validateWorkerOptions(DataflowWorkerHarnessOptions options) { + Preconditions.checkArgument( + options.isStreaming(), + "%s instantiated with options indicating batch use", + StreamingDataflowWorker.class.getName()); + + Preconditions.checkArgument( + !DataflowRunner.hasExperiment(options, BEAM_FN_API_EXPERIMENT), + "%s cannot be main() class with beam_fn_api enabled", + StreamingDataflowWorker.class.getSimpleName()); + } + + private static boolean isDirectPathPipeline(DataflowWorkerHarnessOptions options) { + boolean isIpV6Enabled = options.getDataflowServiceOptions().contains(ENABLE_IPV6_EXPERIMENT); + if (options.isEnableWindmillServiceDirectPath() && !isIpV6Enabled) { + LOG.warn( + "DirectPath is currently only supported with IPv6 networking stack. Defaulting to CloudPath."); + } + + return options.isEnableStreamingEngine() + && options.isEnableWindmillServiceDirectPath() + && isIpV6Enabled; } private static ChannelCachingStubFactory createStubFactory( @@ -702,24 +795,24 @@ private static void enableBigQueryMetrics() { BigQuerySinkMetrics.setSupportStreamingInsertsMetrics(true); } - @VisibleForTesting - final void reportPeriodicWorkerUpdatesForTest() { - workerStatusReporter.reportPeriodicWorkerUpdates(); - } - - private int chooseMaximumNumberOfThreads() { + private static int chooseMaximumNumberOfThreads(DataflowWorkerHarnessOptions options) { if (options.getNumberOfWorkerHarnessThreads() != 0) { return options.getNumberOfWorkerHarnessThreads(); } return MAX_PROCESSING_THREADS; } - private int chooseMaximumBundlesOutstanding() { + private static int chooseMaximumBundlesOutstanding(DataflowWorkerHarnessOptions options) { int maxBundles = options.getMaxBundlesFromWindmillOutstanding(); if (maxBundles > 0) { return maxBundles; } - return chooseMaximumNumberOfThreads() + 100; + return chooseMaximumNumberOfThreads(options) + 100; + } + + @VisibleForTesting + final void reportPeriodicWorkerUpdatesForTest() { + workerStatusReporter.reportPeriodicWorkerUpdates(); } @VisibleForTesting @@ -740,14 +833,17 @@ ComputationStateCache getComputationStateCache() { @SuppressWarnings("FutureReturnValueIgnored") public void start() { running.set(true); - configFetcher.start(); - - memoryMonitorThread.start(); - dispatchThread.start(); sampler.start(); + memoryMonitorThread.start(); + + if (streamingEngineClient != null) { + streamingEngineClient.start(); + } else { + dispatchThread.start(); + workCommitter.start(); + } - workCommitter.start(); workerStatusReporter.start(); activeWorkRefresher.start(); } @@ -765,10 +861,15 @@ void stop() { activeWorkRefresher.stop(); statusPages.stop(); running.set(false); - dispatchThread.interrupt(); - dispatchThread.join(); - workCommitter.stop(); + if (streamingEngineClient != null) { + streamingEngineClient.finish(); + } else { + dispatchThread.interrupt(); + dispatchThread.join(); + workCommitter.stop(); + } + memoryMonitor.stop(); memoryMonitorThread.join(); workUnitExecutor.shutdown(); @@ -835,7 +936,7 @@ void streamingDispatchLoop() { windmillServer.getWorkStream( Windmill.GetWorkRequest.newBuilder() .setClientId(clientId) - .setMaxItems(chooseMaximumBundlesOutstanding()) + .setMaxItems(chooseMaximumBundlesOutstanding(options)) .setMaxBytes(MAX_GET_WORK_FETCH_BYTES) .build(), (String computation, @@ -897,7 +998,7 @@ private Windmill.GetWorkResponse getWork() { return windmillServer.getWork( Windmill.GetWorkRequest.newBuilder() .setClientId(clientId) - .setMaxItems(chooseMaximumBundlesOutstanding()) + .setMaxItems(chooseMaximumBundlesOutstanding(options)) .setMaxBytes(MAX_GET_WORK_FETCH_BYTES) .build()); } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WorkItemCancelledException.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WorkItemCancelledException.java index 934977fe0985e..8b33aa1823d47 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WorkItemCancelledException.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WorkItemCancelledException.java @@ -17,17 +17,16 @@ */ package org.apache.beam.runners.dataflow.worker; +import javax.annotation.Nullable; + /** Indicates that the work item was cancelled and should not be retried. */ -@SuppressWarnings({ - "nullness" // TODO(https://github.com/apache/beam/issues/20497) -}) public class WorkItemCancelledException extends RuntimeException { - public WorkItemCancelledException(long sharding_key) { - super("Work item cancelled for key " + sharding_key); + public WorkItemCancelledException(long shardingKey) { + super("Work item cancelled for key " + shardingKey); } /** Returns whether an exception was caused by a {@link WorkItemCancelledException}. */ - public static boolean isWorkItemCancelledException(Throwable t) { + public static boolean isWorkItemCancelledException(@Nullable Throwable t) { while (t != null) { if (t instanceof WorkItemCancelledException) { return true; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/ActiveWorkState.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/ActiveWorkState.java index 3e226514d57ec..7a90b2c76f5b9 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/ActiveWorkState.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/ActiveWorkState.java @@ -31,6 +31,7 @@ import java.util.Queue; import java.util.concurrent.atomic.AtomicReference; import java.util.function.BiConsumer; +import java.util.stream.Collectors; import java.util.stream.Stream; import javax.annotation.Nullable; import javax.annotation.concurrent.GuardedBy; @@ -45,6 +46,7 @@ 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.collect.ImmutableList; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableListMultimap; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Multimap; import org.joda.time.Duration; @@ -331,6 +333,22 @@ synchronized ImmutableList getKeyHeartbeats( .collect(toImmutableList()); } + synchronized ImmutableListMultimap getReadOnlyActiveWork() { + // Do not return a reference to the underlying workQueue as iterations over it will cause a + // ConcurrentModificationException as it is not a thread-safe data structure. + ImmutableListMultimap.Builder readOnlyActiveWork = + ImmutableListMultimap.builder(); + for (Entry> keyedWorkQueues : activeWork.entrySet()) { + readOnlyActiveWork.putAll( + keyedWorkQueues.getKey(), + keyedWorkQueues.getValue().stream() + .map(ExecutableWork::work) + .collect(Collectors.toList())); + } + + return readOnlyActiveWork.build(); + } + /** * Returns the current aggregate {@link GetWorkBudget} that is active on the user worker. Active * means that the work is received from Windmill, being processed or queued to be processed in diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/ComputationState.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/ComputationState.java index 434e784847997..594c629b5fb73 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/ComputationState.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/ComputationState.java @@ -31,6 +31,7 @@ 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.collect.ImmutableList; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableListMultimap; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Multimap; import org.joda.time.Instant; @@ -139,6 +140,10 @@ public void invalidateStuckCommits(Instant stuckCommitDeadline) { stuckCommitDeadline, this::completeWorkAndScheduleNextWorkForKey); } + public ImmutableListMultimap currentActiveWorkReadOnly() { + return activeWorkState.getReadOnlyActiveWork(); + } + private void execute(ExecutableWork executableWork) { executor.execute(executableWork, executableWork.work().getWorkItem().getSerializedSize()); } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/StageInfo.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/StageInfo.java index 8f14ea26a4619..1ce48c4c88bee 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/StageInfo.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/StageInfo.java @@ -28,7 +28,6 @@ import java.util.List; import org.apache.beam.runners.dataflow.worker.DataflowSystemMetrics; import org.apache.beam.runners.dataflow.worker.MetricsContainerRegistry; -import org.apache.beam.runners.dataflow.worker.StreamingDataflowWorker; import org.apache.beam.runners.dataflow.worker.StreamingModeExecutionContext.StreamingModeExecutionStateRegistry; import org.apache.beam.runners.dataflow.worker.StreamingStepMetricsContainer; import org.apache.beam.runners.dataflow.worker.counters.Counter; @@ -36,11 +35,19 @@ import org.apache.beam.runners.dataflow.worker.counters.DataflowCounterUpdateExtractor; import org.apache.beam.runners.dataflow.worker.counters.NameContext; import org.apache.beam.sdk.io.gcp.bigquery.BigQuerySinkMetrics; +import org.apache.beam.sdk.metrics.MetricName; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; /** Contains a few of the stage specific fields. E.g. metrics container registry, counters etc. */ @AutoValue public abstract class StageInfo { + // TODO(https://github.com/apache/beam/issues/19632): Update throttling counters to use generic + // throttling-msecs metric. + private static final MetricName BIGQUERY_STREAMING_INSERT_THROTTLE_TIME = + MetricName.named( + "org.apache.beam.sdk.io.gcp.bigquery.BigQueryServicesImpl$DatasetServiceImpl", + "throttling-msecs"); + public static StageInfo create(String stageName, String systemName) { NameContext nameContext = NameContext.newBuilder(stageName).setSystemName(systemName).build(); CounterSet deltaCounters = new CounterSet(); @@ -101,10 +108,10 @@ private void translateKnownStepCounters(CounterUpdate stepCounterUpdate) { stepCounterUpdate.getStructuredNameAndMetadata().getName(); if ((THROTTLING_MSECS_METRIC_NAME.getNamespace().equals(structuredName.getOriginNamespace()) && THROTTLING_MSECS_METRIC_NAME.getName().equals(structuredName.getName())) - || (StreamingDataflowWorker.BIGQUERY_STREAMING_INSERT_THROTTLE_TIME + || (BIGQUERY_STREAMING_INSERT_THROTTLE_TIME .getNamespace() .equals(structuredName.getOriginNamespace()) - && StreamingDataflowWorker.BIGQUERY_STREAMING_INSERT_THROTTLE_TIME + && BIGQUERY_STREAMING_INSERT_THROTTLE_TIME .getName() .equals(structuredName.getName()))) { long msecs = DataflowCounterUpdateExtractor.splitIntToLong(stepCounterUpdate.getInteger()); diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/Work.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/Work.java index ed3f2671b40c0..6182f295d4b66 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/Work.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/Work.java @@ -31,6 +31,7 @@ import java.util.function.Consumer; import java.util.function.Function; import java.util.function.Supplier; +import javax.annotation.Nullable; import javax.annotation.concurrent.NotThreadSafe; import org.apache.beam.repackaged.core.org.apache.commons.lang3.tuple.Pair; import org.apache.beam.runners.dataflow.worker.ActiveMessageMetadata; @@ -43,6 +44,7 @@ import org.apache.beam.runners.dataflow.worker.windmill.Windmill.LatencyAttribution.ActiveLatencyBreakdown.Distribution; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.WorkItem; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.WorkItemCommitRequest; +import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStream.GetDataStream; import org.apache.beam.runners.dataflow.worker.windmill.client.commits.Commit; import org.apache.beam.runners.dataflow.worker.windmill.client.commits.WorkCommitter; import org.apache.beam.runners.dataflow.worker.windmill.state.WindmillStateReader; @@ -107,7 +109,15 @@ public static ProcessingContext createProcessingContext( String computationId, BiFunction getKeyedDataFn, Consumer workCommitter) { - return ProcessingContext.create(computationId, getKeyedDataFn, workCommitter); + return ProcessingContext.create(computationId, getKeyedDataFn, workCommitter, null); + } + + public static ProcessingContext createFanOutProcessingContext( + String computationId, + BiFunction getKeyedDataFn, + Consumer workCommitter, + GetDataStream getDataStream) { + return ProcessingContext.create(computationId, getKeyedDataFn, workCommitter, getDataStream); } private static LatencyAttribution.Builder createLatencyAttributionWithActiveLatencyBreakdown( @@ -171,6 +181,10 @@ public State getState() { return currentState.state(); } + public @Nullable GetDataStream getDataStream() { + return processingContext.getDataStream(); + } + public void setState(State state) { Instant now = clock.get(); totalDurationPerState.compute( @@ -315,11 +329,13 @@ public abstract static class ProcessingContext { private static ProcessingContext create( String computationId, BiFunction getKeyedDataFn, - Consumer workCommitter) { + Consumer workCommitter, + @Nullable GetDataStream getDataStream) { return new AutoValue_Work_ProcessingContext( computationId, request -> Optional.ofNullable(getKeyedDataFn.apply(computationId, request)), - workCommitter); + workCommitter, + getDataStream); } /** Computation that the {@link Work} belongs to. */ @@ -334,5 +350,7 @@ private static ProcessingContext create( * {@link WorkItem}. */ public abstract Consumer workCommitter(); + + public abstract @Nullable GetDataStream getDataStream(); } } 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 d305e25af7e50..1788479856b44 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 @@ -135,6 +135,7 @@ public void start(DataflowWorkerHarnessOptions options) { statusPages.addStatusDataProvider( "exception", "Last Exception", new LastExceptionDataProvider()); statusPages.addStatusDataProvider("cache", "State Cache", stateCache); + statusPages.addStatusDataProvider("activeGetWorkBudget", "Active GetWork Budget", writer -> {}); if (isStreamingEngine()) { addStreamingEngineStatusPages(); diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/harness/StreamingWorkerStatusReporter.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/harness/StreamingWorkerStatusReporter.java index ba77d8e1ce265..2e8190afc8dd6 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/harness/StreamingWorkerStatusReporter.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/harness/StreamingWorkerStatusReporter.java @@ -34,7 +34,6 @@ import java.util.Iterator; import java.util.List; import java.util.Optional; -import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; @@ -57,7 +56,6 @@ import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ListMultimap; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.MultimapBuilder; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.math.LongMath; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.ThreadFactoryBuilder; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -74,7 +72,6 @@ public final class StreamingWorkerStatusReporter { private static final String WORKER_MESSAGE_REPORTER_THREAD = "ReportWorkerMessage"; private static final String GLOBAL_WORKER_UPDATE_REPORTER_THREAD = "GlobalWorkerUpdates"; - private final boolean publishCounters; private final int initialMaxThreadCount; private final int initialMaxBundlesOutstanding; private final WorkUnitClient dataflowServiceClient; @@ -98,7 +95,6 @@ public final class StreamingWorkerStatusReporter { private final AtomicLong workerMessagesIndex; private StreamingWorkerStatusReporter( - boolean publishCounters, WorkUnitClient dataflowServiceClient, Supplier windmillQuotaThrottleTime, Supplier> allStageInfo, @@ -109,7 +105,6 @@ private StreamingWorkerStatusReporter( Function executorFactory, long windmillHarnessUpdateReportingPeriodMillis, long perWorkerMetricsUpdateReportingPeriodMillis) { - this.publishCounters = publishCounters; this.dataflowServiceClient = dataflowServiceClient; this.windmillQuotaThrottleTime = windmillQuotaThrottleTime; this.allStageInfo = allStageInfo; @@ -132,34 +127,6 @@ private StreamingWorkerStatusReporter( } public static StreamingWorkerStatusReporter create( - WorkUnitClient workUnitClient, - Supplier windmillQuotaThrottleTime, - Supplier> allStageInfo, - FailureTracker failureTracker, - StreamingCounters streamingCounters, - MemoryMonitor memoryMonitor, - BoundedQueueExecutor workExecutor, - long windmillHarnessUpdateReportingPeriodMillis, - long perWorkerMetricsUpdateReportingPeriodMillis) { - return new StreamingWorkerStatusReporter( - /* publishCounters= */ true, - workUnitClient, - windmillQuotaThrottleTime, - allStageInfo, - failureTracker, - streamingCounters, - memoryMonitor, - workExecutor, - threadName -> - Executors.newSingleThreadScheduledExecutor( - new ThreadFactoryBuilder().setNameFormat(threadName).build()), - windmillHarnessUpdateReportingPeriodMillis, - perWorkerMetricsUpdateReportingPeriodMillis); - } - - @VisibleForTesting - public static StreamingWorkerStatusReporter forTesting( - boolean publishCounters, WorkUnitClient workUnitClient, Supplier windmillQuotaThrottleTime, Supplier> allStageInfo, @@ -171,7 +138,6 @@ public static StreamingWorkerStatusReporter forTesting( long windmillHarnessUpdateReportingPeriodMillis, long perWorkerMetricsUpdateReportingPeriodMillis) { return new StreamingWorkerStatusReporter( - publishCounters, workUnitClient, windmillQuotaThrottleTime, allStageInfo, @@ -228,6 +194,22 @@ private static void shutdownExecutor(ScheduledExecutorService executor) { } } + // Calculates the PerWorkerMetrics reporting frequency, ensuring alignment with the + // WorkerMessages RPC schedule. The desired reporting period + // (perWorkerMetricsUpdateReportingPeriodMillis) is adjusted to the nearest multiple + // of the RPC interval (windmillHarnessUpdateReportingPeriodMillis). + private static long getPerWorkerMetricsUpdateFrequency( + long windmillHarnessUpdateReportingPeriodMillis, + long perWorkerMetricsUpdateReportingPeriodMillis) { + if (windmillHarnessUpdateReportingPeriodMillis == 0) { + return 0; + } + return LongMath.divide( + perWorkerMetricsUpdateReportingPeriodMillis, + windmillHarnessUpdateReportingPeriodMillis, + RoundingMode.CEILING); + } + @SuppressWarnings("FutureReturnValueIgnored") public void start() { reportHarnessStartup(); @@ -276,22 +258,6 @@ private void reportHarnessStartup() { } } - // Calculates the PerWorkerMetrics reporting frequency, ensuring alignment with the - // WorkerMessages RPC schedule. The desired reporting period - // (perWorkerMetricsUpdateReportingPeriodMillis) is adjusted to the nearest multiple - // of the RPC interval (windmillHarnessUpdateReportingPeriodMillis). - private static long getPerWorkerMetricsUpdateFrequency( - long windmillHarnessUpdateReportingPeriodMillis, - long perWorkerMetricsUpdateReportingPeriodMillis) { - if (windmillHarnessUpdateReportingPeriodMillis == 0) { - return 0; - } - return LongMath.divide( - perWorkerMetricsUpdateReportingPeriodMillis, - windmillHarnessUpdateReportingPeriodMillis, - RoundingMode.CEILING); - } - /** Sends counter updates to Dataflow backend. */ private void sendWorkerUpdatesToDataflowService( CounterSet deltaCounters, CounterSet cumulativeCounters) throws IOException { @@ -303,13 +269,11 @@ private void sendWorkerUpdatesToDataflowService( List counterUpdates = new ArrayList<>(COUNTER_UPDATES_SIZE); - if (publishCounters) { - allStageInfo.get().forEach(s -> counterUpdates.addAll(s.extractCounterUpdates())); - counterUpdates.addAll( - cumulativeCounters.extractUpdates(false, DataflowCounterUpdateExtractor.INSTANCE)); - counterUpdates.addAll( - deltaCounters.extractModifiedDeltaUpdates(DataflowCounterUpdateExtractor.INSTANCE)); - } + allStageInfo.get().forEach(s -> counterUpdates.addAll(s.extractCounterUpdates())); + counterUpdates.addAll( + cumulativeCounters.extractUpdates(false, DataflowCounterUpdateExtractor.INSTANCE)); + counterUpdates.addAll( + deltaCounters.extractModifiedDeltaUpdates(DataflowCounterUpdateExtractor.INSTANCE)); // Handle duplicate counters from different stages. Store all the counters in a multimap and // send the counters that appear multiple times in separate RPCs. Same logical counter could @@ -368,7 +332,7 @@ private List createWorkerMessage() { List workerMessages = new ArrayList<>(2); workerMessages.add(createWorkerMessageForStreamingScalingReport()); - createWorkerMessageForPerWorkerMetrics().ifPresent(metrics -> workerMessages.add(metrics)); + createWorkerMessageForPerWorkerMetrics().ifPresent(workerMessages::add); return workerMessages; } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/WindmillServiceAddress.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/WindmillServiceAddress.java index 90f93b072673d..3c9226be34a5d 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/WindmillServiceAddress.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/WindmillServiceAddress.java @@ -19,38 +19,30 @@ import com.google.auto.value.AutoOneOf; import com.google.auto.value.AutoValue; -import java.net.Inet6Address; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.net.HostAndPort; /** Used to create channels to communicate with Streaming Engine via gRpc. */ @AutoOneOf(WindmillServiceAddress.Kind.class) public abstract class WindmillServiceAddress { - public static WindmillServiceAddress create(Inet6Address ipv6Address) { - return AutoOneOf_WindmillServiceAddress.ipv6(ipv6Address); - } public static WindmillServiceAddress create(HostAndPort gcpServiceAddress) { return AutoOneOf_WindmillServiceAddress.gcpServiceAddress(gcpServiceAddress); } - public abstract Kind getKind(); - - public abstract Inet6Address ipv6(); - - public abstract HostAndPort gcpServiceAddress(); - - public abstract AuthenticatedGcpServiceAddress authenticatedGcpServiceAddress(); - public static WindmillServiceAddress create( AuthenticatedGcpServiceAddress authenticatedGcpServiceAddress) { return AutoOneOf_WindmillServiceAddress.authenticatedGcpServiceAddress( authenticatedGcpServiceAddress); } + public abstract Kind getKind(); + + public abstract HostAndPort gcpServiceAddress(); + + public abstract AuthenticatedGcpServiceAddress authenticatedGcpServiceAddress(); + public enum Kind { - IPV6, GCP_SERVICE_ADDRESS, - // TODO(m-trieu): Use for direct connections when ALTS is enabled. AUTHENTICATED_GCP_SERVICE_ADDRESS } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/AbstractWindmillStream.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/AbstractWindmillStream.java index 028a5c2e1d4b6..f958cbaa94ce0 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/AbstractWindmillStream.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/AbstractWindmillStream.java @@ -176,6 +176,7 @@ protected final void startStream() { onNewStream(); if (clientClosed.get()) { close(); + streamRegistry.remove(this); } return; } @@ -238,7 +239,7 @@ public final void appendSummaryHtml(PrintWriter writer) { protected abstract void appendSpecificHtml(PrintWriter writer); @Override - public final synchronized void close() { + public synchronized void close() { // Synchronization of close and onCompleted necessary for correct retry logic in onNewStream. clientClosed.set(true); requestObserver().onCompleted(); @@ -255,6 +256,16 @@ public final Instant startTime() { return new Instant(startTimeMs.get()); } + @Override + public boolean isClosed() { + return streamClosed.get() || clientClosed.get(); + } + + private void setLastError(String error) { + lastError.set(error); + lastErrorTime.set(DateTime.now()); + } + private class ResponseObserver implements StreamObserver { @Override @@ -337,9 +348,4 @@ private void onStreamFinished(@Nullable Throwable t) { executor.execute(AbstractWindmillStream.this::startStream); } } - - private void setLastError(String error) { - lastError.set(error); - lastErrorTime.set(DateTime.now()); - } } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/WindmillStream.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/WindmillStream.java index d044e9300790b..1355837acc0dd 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/WindmillStream.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/WindmillStream.java @@ -41,6 +41,8 @@ public interface WindmillStream { /** Returns when the stream was opened. */ Instant startTime(); + boolean isClosed(); + /** Handle representing a stream of GetWork responses. */ @ThreadSafe interface GetWorkStream extends WindmillStream { diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/WindmillStreamCancelledException.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/WindmillStreamCancelledException.java new file mode 100644 index 0000000000000..194206b21b856 --- /dev/null +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/WindmillStreamCancelledException.java @@ -0,0 +1,39 @@ +/* + * 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.windmill.client; + +import org.apache.beam.sdk.annotations.Internal; + +@Internal +public final class WindmillStreamCancelledException extends RuntimeException { + + public WindmillStreamCancelledException(String message) { + super(message); + } + + /** Returns whether an exception was caused by a {@link WindmillStreamCancelledException}. */ + public static boolean isWindmillStreamCancelledException(Throwable t) { + while (t != null) { + if (t instanceof WindmillStreamCancelledException) { + return true; + } + t = t.getCause(); + } + return false; + } +} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/ChannelzServlet.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/ChannelzServlet.java index e0f823d79ade5..adfb380d21647 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/ChannelzServlet.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/ChannelzServlet.java @@ -23,6 +23,7 @@ import java.util.HashSet; import java.util.List; import java.util.Set; +import java.util.function.Supplier; import java.util.stream.Collectors; import javax.annotation.Nullable; import javax.servlet.ServletException; @@ -31,7 +32,6 @@ import org.apache.beam.runners.dataflow.options.DataflowStreamingPipelineOptions; import org.apache.beam.runners.dataflow.worker.status.BaseStatusServlet; import org.apache.beam.runners.dataflow.worker.status.DebugCapture; -import org.apache.beam.runners.dataflow.worker.windmill.WindmillServerStub; import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.channelz.v1.*; import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.protobuf.services.ChannelzService; @@ -47,16 +47,16 @@ public class ChannelzServlet extends BaseStatusServlet implements DebugCapture.C private static final int MAX_TOP_CHANNELS_TO_RETURN = 500; private final ChannelzService channelzService; - private final WindmillServerStub windmillServerStub; + private final Supplier> currentWindmillEndpoints; private final boolean showOnlyWindmillServiceChannels; public ChannelzServlet( String path, DataflowStreamingPipelineOptions options, - WindmillServerStub windmillServerStub) { + Supplier> currentWindmillEndpoints) { super(path); channelzService = ChannelzService.newInstance(MAX_TOP_CHANNELS_TO_RETURN); - this.windmillServerStub = windmillServerStub; + this.currentWindmillEndpoints = currentWindmillEndpoints; showOnlyWindmillServiceChannels = options.getChannelzShowOnlyWindmillServiceChannels(); } @@ -81,14 +81,6 @@ public void captureData(PrintWriter writer) { writer.println(""); } - // channelz proto says there won't be cycles in the ref graph. - // we track visited ids to be defensive and prevent any accidental cycles. - private static class VisitedSets { - - Set channels = new HashSet<>(); - Set subchannels = new HashSet<>(); - } - private void appendTopChannels(PrintWriter writer) { SettableFuture future = SettableFuture.create(); // IDEA: If there are more than MAX_TOP_CHANNELS_TO_RETURN top channels @@ -127,8 +119,7 @@ private void appendTopChannels(PrintWriter writer) { } private List filterWindmillChannels(List channels) { - ImmutableSet windmillServiceEndpoints = - windmillServerStub.getWindmillServiceEndpoints(); + ImmutableSet windmillServiceEndpoints = currentWindmillEndpoints.get(); Set windmillServiceHosts = windmillServiceEndpoints.stream().map(HostAndPort::getHost).collect(Collectors.toSet()); List windmillChannels = new ArrayList<>(); @@ -291,4 +282,12 @@ public void onCompleted() { } }; } + + // channelz proto says there won't be cycles in the ref graph. + // we track visited ids to be defensive and prevent any accidental cycles. + private static class VisitedSets { + + Set channels = new HashSet<>(); + Set subchannels = new HashSet<>(); + } } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcDirectGetWorkStream.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcDirectGetWorkStream.java index 6f4b5b7b33fb7..f4477d6db4c9a 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcDirectGetWorkStream.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcDirectGetWorkStream.java @@ -24,6 +24,7 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.atomic.AtomicReference; +import java.util.function.BiFunction; import java.util.function.Function; import java.util.function.Supplier; import javax.annotation.Nullable; @@ -33,6 +34,8 @@ import org.apache.beam.runners.dataflow.worker.windmill.Windmill; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.ComputationWorkItemMetadata; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.GetWorkRequest; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill.KeyedGetDataRequest; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill.KeyedGetDataResponse; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.StreamingGetWorkRequest; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.StreamingGetWorkResponseChunk; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.WorkItem; @@ -82,6 +85,9 @@ public final class GrpcDirectGetWorkStream private final ThrottleTimer getWorkThrottleTimer; private final Supplier getDataStream; private final Supplier workCommitter; + private final Function< + GetDataStream, BiFunction> + keyedGetDataFn; /** * Map of stream IDs to their buffers. Used to aggregate streaming gRPC response chunks as they @@ -103,6 +109,8 @@ private GrpcDirectGetWorkStream( ThrottleTimer getWorkThrottleTimer, Supplier getDataStream, Supplier workCommitter, + Function> + keyedGetDataFn, WorkItemScheduler workItemScheduler) { super( startGetWorkRpcFn, backoff, streamObserverFactory, streamRegistry, logEveryNStreamFailures); @@ -114,6 +122,7 @@ private GrpcDirectGetWorkStream( // stream. this.getDataStream = Suppliers.memoize(getDataStream::get); this.workCommitter = Suppliers.memoize(workCommitter::get); + this.keyedGetDataFn = keyedGetDataFn; this.inFlightBudget = new AtomicReference<>(GetWorkBudget.noBudget()); this.nextBudgetAdjustment = new AtomicReference<>(GetWorkBudget.noBudget()); this.pendingResponseBudget = new AtomicReference<>(GetWorkBudget.noBudget()); @@ -132,6 +141,8 @@ public static GrpcDirectGetWorkStream create( ThrottleTimer getWorkThrottleTimer, Supplier getDataStream, Supplier workCommitter, + Function> + keyedGetDataFn, WorkItemScheduler workItemScheduler) { GrpcDirectGetWorkStream getWorkStream = new GrpcDirectGetWorkStream( @@ -144,6 +155,7 @@ public static GrpcDirectGetWorkStream create( getWorkThrottleTimer, getDataStream, workCommitter, + keyedGetDataFn, workItemScheduler); getWorkStream.startStream(); return getWorkStream; @@ -187,22 +199,24 @@ private void sendRequestExtension() { @Override protected synchronized void onNewStream() { workItemBuffers.clear(); - // Add the current in-flight budget to the next adjustment. Only positive values are allowed - // here - // with negatives defaulting to 0, since GetWorkBudgets cannot be created with negative values. - GetWorkBudget budgetAdjustment = nextBudgetAdjustment.get().apply(inFlightBudget.get()); - inFlightBudget.set(budgetAdjustment); - send( - StreamingGetWorkRequest.newBuilder() - .setRequest( - request - .toBuilder() - .setMaxBytes(budgetAdjustment.bytes()) - .setMaxItems(budgetAdjustment.items())) - .build()); - - // We just sent the budget, reset it. - nextBudgetAdjustment.set(GetWorkBudget.noBudget()); + if (!isClosed()) { + // Add the current in-flight budget to the next adjustment. Only positive values are allowed + // here with negatives defaulting to 0, since GetWorkBudgets cannot be created with negative + // values. + GetWorkBudget budgetAdjustment = nextBudgetAdjustment.get().apply(inFlightBudget.get()); + inFlightBudget.set(budgetAdjustment); + send( + StreamingGetWorkRequest.newBuilder() + .setRequest( + request + .toBuilder() + .setMaxBytes(budgetAdjustment.bytes()) + .setMaxItems(budgetAdjustment.items())) + .build()); + + // We just sent the budget, reset it. + nextBudgetAdjustment.set(GetWorkBudget.noBudget()); + } } @Override @@ -326,8 +340,12 @@ private void runAndReset() { } private Work.ProcessingContext createProcessingContext(String computationId) { - return Work.createProcessingContext( - computationId, getDataStream.get()::requestKeyedData, workCommitter.get()::commit); + return Work.createFanOutProcessingContext( + computationId, + (computation, request) -> + keyedGetDataFn.apply(getDataStream.get()).apply(computation, request), + workCommitter.get()::commit, + getDataStream.get()); } } } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetDataStream.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetDataStream.java index feb15c2ac83cb..08e1e790ed237 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetDataStream.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetDataStream.java @@ -34,6 +34,7 @@ import java.util.concurrent.atomic.AtomicLong; import java.util.function.Consumer; import java.util.function.Function; +import java.util.stream.Collectors; import org.apache.beam.runners.dataflow.worker.windmill.Windmill; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.ComputationGetDataRequest; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.ComputationHeartbeatRequest; @@ -48,6 +49,7 @@ import org.apache.beam.runners.dataflow.worker.windmill.Windmill.StreamingGetDataResponse; import org.apache.beam.runners.dataflow.worker.windmill.client.AbstractWindmillStream; import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStream.GetDataStream; +import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStreamCancelledException; import org.apache.beam.runners.dataflow.worker.windmill.client.grpc.GrpcGetDataStreamRequests.QueuedBatch; import org.apache.beam.runners.dataflow.worker.windmill.client.grpc.GrpcGetDataStreamRequests.QueuedRequest; import org.apache.beam.runners.dataflow.worker.windmill.client.grpc.observers.StreamObserverFactory; @@ -129,8 +131,43 @@ public static GrpcGetDataStream create( return getDataStream; } + private static String createStreamCancelledErrorMessage(QueuedBatch batch) { + return batch.requests().stream() + .map( + request -> { + switch (request.getDataRequest().getKind()) { + case GLOBAL: + return "GetSideInput=" + request.getDataRequest().global(); + case COMPUTATION: + return request.getDataRequest().computation().getRequestsList().stream() + .map( + keyedRequest -> + "KeyedGetState=[" + + "key=" + + keyedRequest.getKey() + + "shardingKey=" + + keyedRequest.getShardingKey() + + "cacheToken=" + + keyedRequest.getCacheToken() + + "workToken" + + keyedRequest.getWorkToken() + + "]") + .collect(Collectors.joining()); + default: + // Will never happen switch is exhaustive. + throw new IllegalStateException(); + } + }) + .collect(Collectors.joining(",")); + } + @Override protected synchronized void onNewStream() { + // Stream has been explicitly closed. + if (isClosed()) { + return; + } + send(StreamingGetDataRequest.newBuilder().setHeader(jobHeader).build()); if (clientClosed.get()) { // We rely on close only occurring after all methods on the stream have returned. @@ -263,6 +300,17 @@ public void sendHealthCheck() { } } + @Override + public synchronized void close() { + super.close(); + for (AppendableInputStream responseStream : pending.values()) { + responseStream.cancel(); + } + // Stream has been explicitly closed. + pending.clear(); + batches.clear(); + } + @Override public void appendSpecificHtml(PrintWriter writer) { writer.format( @@ -295,10 +343,8 @@ private ResponseT issueRequest(QueuedRequest request, ParseFn withDefaultDeadline(stub).getWorkStream(responseObserver), request, grpcBackOff.get(), - newStreamObserverFactory(), + newBufferringStreamObserverFactory(), streamRegistry, logEveryNStreamFailures, getWorkThrottleTimer, @@ -195,26 +199,44 @@ public GetWorkStream createDirectGetWorkStream( ThrottleTimer getWorkThrottleTimer, Supplier getDataStream, Supplier workCommitter, + Function< + GetDataStream, + BiFunction> + keyedGetDataFn, WorkItemScheduler workItemScheduler) { return GrpcDirectGetWorkStream.create( - responseObserver -> withDefaultDeadline(stub).getWorkStream(responseObserver), + stub::getWorkStream, request, grpcBackOff.get(), - newStreamObserverFactory(), + newSimpleStreamObserverFactory(), streamRegistry, logEveryNStreamFailures, getWorkThrottleTimer, getDataStream, workCommitter, + keyedGetDataFn, workItemScheduler); } + public GetDataStream createDirectGetDataStream( + CloudWindmillServiceV1Alpha1Stub stub, + ThrottleTimer getDataThrottleTimer, + boolean sendKeyedGetDataRequests, + Consumer> processHeartbeatResponses) { + return createGetDataStream( + () -> stub, + getDataThrottleTimer, + sendKeyedGetDataRequests, + processHeartbeatResponses, + newSimpleStreamObserverFactory()); + } + public GetDataStream createGetDataStream( CloudWindmillServiceV1Alpha1Stub stub, ThrottleTimer getDataThrottleTimer) { return GrpcGetDataStream.create( - responseObserver -> withDefaultDeadline(stub).getDataStream(responseObserver), + responseObserver -> stub.get().getDataStream(responseObserver), grpcBackOff.get(), - newStreamObserverFactory(), + streamObserverFactory, streamRegistry, logEveryNStreamFailures, getDataThrottleTimer, @@ -227,10 +249,26 @@ public GetDataStream createGetDataStream( public CommitWorkStream createCommitWorkStream( CloudWindmillServiceV1Alpha1Stub stub, ThrottleTimer commitWorkThrottleTimer) { + return createCommitWorkStream( + () -> withDefaultDeadline(stub), + commitWorkThrottleTimer, + newBufferringStreamObserverFactory()); + } + + public CommitWorkStream createDirectCommitWorkStream( + CloudWindmillServiceV1Alpha1Stub stub, ThrottleTimer commitWorkThrottleTimer) { + return createCommitWorkStream( + () -> stub, commitWorkThrottleTimer, newSimpleStreamObserverFactory()); + } + + private CommitWorkStream createCommitWorkStream( + Supplier stub, + ThrottleTimer commitWorkThrottleTimer, + StreamObserverFactory streamObserverFactory) { return GrpcCommitWorkStream.create( - responseObserver -> withDefaultDeadline(stub).commitWorkStream(responseObserver), + responseObserver -> stub.get().commitWorkStream(responseObserver), grpcBackOff.get(), - newStreamObserverFactory(), + streamObserverFactory, streamRegistry, logEveryNStreamFailures, commitWorkThrottleTimer, @@ -246,7 +284,7 @@ public GetWorkerMetadataStream createGetWorkerMetadataStream( return GrpcGetWorkerMetadataStream.create( responseObserver -> withDefaultDeadline(stub).getWorkerMetadata(responseObserver), grpcBackOff.get(), - newStreamObserverFactory(), + newBufferringStreamObserverFactory(), streamRegistry, logEveryNStreamFailures, jobHeader, @@ -255,11 +293,53 @@ public GetWorkerMetadataStream createGetWorkerMetadataStream( onNewWindmillEndpoints); } - private StreamObserverFactory newStreamObserverFactory() { + private StreamObserverFactory newBufferringStreamObserverFactory() { return StreamObserverFactory.direct( DEFAULT_STREAM_RPC_DEADLINE_SECONDS * 2, windmillMessagesBetweenIsReadyChecks); } + /** + * Simple {@link StreamObserverFactory} that does not buffer or provide extra functionality for + * request observers. + * + * @implNote Used to create stream observers for direct path streams that do not share any + * underlying resources between threads. + */ + private StreamObserverFactory newSimpleStreamObserverFactory() { + return new StreamObserverFactory() { + @Override + public StreamObserver from( + Function, StreamObserver> clientFactory, + StreamObserver responseObserver) { + return clientFactory.apply(responseObserver); + } + }; + } + + /** + * Schedules streaming RPC health checks to run on a background daemon thread, which will be + * cleaned up when the JVM shutdown. + */ + public void scheduleHealthChecks(int healthCheckInterval) { + if (healthCheckInterval < 0) { + return; + } + + new Timer("WindmillHealthCheckTimer") + .schedule( + new TimerTask() { + @Override + public void run() { + Instant reportThreshold = Instant.now().minus(Duration.millis(healthCheckInterval)); + for (AbstractWindmillStream stream : streamRegistry) { + stream.maybeSendHealthCheck(reportThreshold); + } + } + }, + 0, + healthCheckInterval); + } + @Override public void appendSummaryHtml(PrintWriter writer) { writer.write("Active Streams:
"); diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/StreamingEngineClient.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/StreamingEngineClient.java index 4760062c5754a..d1a2d15379f5e 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/StreamingEngineClient.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/StreamingEngineClient.java @@ -26,17 +26,25 @@ import java.util.Optional; import java.util.Queue; import java.util.Random; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionStage; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; +import java.util.function.BiFunction; +import java.util.function.Consumer; import java.util.function.Function; import java.util.function.Supplier; +import java.util.stream.Collectors; import javax.annotation.CheckReturnValue; import javax.annotation.concurrent.ThreadSafe; +import org.apache.beam.repackaged.core.org.apache.commons.lang3.tuple.Pair; import org.apache.beam.runners.dataflow.worker.windmill.CloudWindmillServiceV1Alpha1Grpc.CloudWindmillServiceV1Alpha1Stub; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.GetWorkRequest; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.JobHeader; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill.KeyedGetDataRequest; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill.KeyedGetDataResponse; import org.apache.beam.runners.dataflow.worker.windmill.WindmillConnection; import org.apache.beam.runners.dataflow.worker.windmill.WindmillEndpoints; import org.apache.beam.runners.dataflow.worker.windmill.WindmillEndpoints.Endpoint; @@ -52,6 +60,7 @@ import org.apache.beam.runners.dataflow.worker.windmill.work.budget.GetWorkBudgetDistributor; import org.apache.beam.runners.dataflow.worker.windmill.work.budget.GetWorkBudgetRefresher; import org.apache.beam.sdk.annotations.Internal; +import org.apache.beam.sdk.util.MoreFutures; 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.Suppliers; @@ -76,6 +85,8 @@ public final class StreamingEngineClient { private static final Logger LOG = LoggerFactory.getLogger(StreamingEngineClient.class); private static final String PUBLISH_NEW_WORKER_METADATA_THREAD = "PublishNewWorkerMetadataThread"; private static final String CONSUME_NEW_WORKER_METADATA_THREAD = "ConsumeNewWorkerMetadataThread"; + private static final String STREAM_MANAGER_THREAD = "WindmillStreamManager"; + private final JobHeader jobHeader; private final GrpcWindmillStreamFactory streamFactory; private final WorkItemScheduler workItemScheduler; @@ -91,6 +102,10 @@ public final class StreamingEngineClient { private final Supplier getWorkerMetadataStream; private final Queue newWindmillEndpoints; private final Function workCommitterFactory; + private final Function< + GetDataStream, BiFunction> + keyedGetDataFn; + private final ExecutorService windmillStreamManager; /** Writes are guarded by synchronization, reads are lock free. */ private final AtomicReference connections; @@ -107,8 +122,11 @@ private StreamingEngineClient( GetWorkBudgetDistributor getWorkBudgetDistributor, GrpcDispatcherClient dispatcherClient, long clientId, - Function workCommitterFactory) { + Function workCommitterFactory, + Function> + keyedGetDataFn) { this.jobHeader = jobHeader; + this.keyedGetDataFn = keyedGetDataFn; this.started = false; this.streamFactory = streamFactory; this.workItemScheduler = workItemScheduler; @@ -143,6 +161,9 @@ private StreamingEngineClient( newWorkerMetadataPublisher.submit( () -> newWindmillEndpoints.add(endpoints)))); this.workCommitterFactory = workCommitterFactory; + this.windmillStreamManager = + Executors.newCachedThreadPool( + new ThreadFactoryBuilder().setNameFormat(STREAM_MANAGER_THREAD).build()); } private static ExecutorService singleThreadedExecutorServiceOf(String threadName) { @@ -171,7 +192,9 @@ public static StreamingEngineClient create( ChannelCachingStubFactory channelCachingStubFactory, GetWorkBudgetDistributor getWorkBudgetDistributor, GrpcDispatcherClient dispatcherClient, - Function workCommitterFactory) { + Function workCommitterFactory, + Function> + keyedGetDataFn) { return new StreamingEngineClient( jobHeader, totalGetWorkBudget, @@ -181,7 +204,8 @@ public static StreamingEngineClient create( getWorkBudgetDistributor, dispatcherClient, /* clientId= */ new Random().nextLong(), - workCommitterFactory); + workCommitterFactory, + keyedGetDataFn); } @VisibleForTesting @@ -194,7 +218,9 @@ static StreamingEngineClient forTesting( GetWorkBudgetDistributor getWorkBudgetDistributor, GrpcDispatcherClient dispatcherClient, long clientId, - Function workCommitterFactory) { + Function workCommitterFactory, + Function> + keyedGetDataFn) { StreamingEngineClient streamingEngineClient = new StreamingEngineClient( jobHeader, @@ -205,7 +231,8 @@ static StreamingEngineClient forTesting( getWorkBudgetDistributor, dispatcherClient, clientId, - workCommitterFactory); + workCommitterFactory, + keyedGetDataFn); streamingEngineClient.start(); return streamingEngineClient; } @@ -225,9 +252,6 @@ public ImmutableSet currentWindmillEndpoints() { .map(Endpoint::directEndpoint) .filter(Optional::isPresent) .map(Optional::get) - .filter( - windmillServiceAddress -> - windmillServiceAddress.getKind() != WindmillServiceAddress.Kind.IPV6) .map( windmillServiceAddress -> windmillServiceAddress.getKind() == WindmillServiceAddress.Kind.GCP_SERVICE_ADDRESS @@ -280,15 +304,18 @@ private synchronized void consumeWindmillWorkerEndpoints(WindmillEndpoints newWi ImmutableMap newWindmillConnections = createNewWindmillConnections(newWindmillEndpoints.windmillEndpoints()); + CompletableFuture closeStaleStreams = closeStaleStreams(newWindmillConnections.values()); + StreamingEngineConnectionState newConnectionsState = StreamingEngineConnectionState.builder() .setWindmillConnections(newWindmillConnections) - .setWindmillStreams( - closeStaleStreamsAndCreateNewStreams(newWindmillConnections.values())) + .setWindmillStreams(createAndStartNewStreams(newWindmillConnections.values()).join()) .setGlobalDataStreams( createNewGlobalDataStreams(newWindmillEndpoints.globalDataEndpoints())) .build(); + closeStaleStreams.join(); + LOG.info( "Setting new connections: {}. Previous connections: {}.", newConnectionsState, @@ -298,6 +325,60 @@ private synchronized void consumeWindmillWorkerEndpoints(WindmillEndpoints newWi getWorkBudgetRefresher.requestBudgetRefresh(); } + /** Close the streams that are no longer valid in parallel. */ + private synchronized CompletableFuture closeStaleStreams( + Collection newWindmillConnections) { + ImmutableMap currentStreams = + connections.get().windmillStreams(); + return CompletableFuture.allOf( + currentStreams.entrySet().stream() + .filter( + connectionAndStream -> + !newWindmillConnections.contains(connectionAndStream.getKey())) + .map( + entry -> + CompletableFuture.runAsync( + () -> { + entry.getValue().closeAllStreams(); + entry + .getKey() + .directEndpoint() + .ifPresent(channelCachingStubFactory::remove); + }, + windmillStreamManager)) + .toArray(CompletableFuture[]::new)); + } + + private synchronized CompletableFuture> + createAndStartNewStreams(Collection newWindmillConnections) { + ImmutableMap currentStreams = + connections.get().windmillStreams(); + CompletionStage>> + connectionAndSenderFuture = + MoreFutures.allAsList( + newWindmillConnections.stream() + .map( + connection -> + MoreFutures.supplyAsync( + () -> + Pair.of( + connection, + Optional.ofNullable(currentStreams.get(connection)) + .orElseGet( + () -> + createAndStartWindmillStreamSenderFor( + connection))), + windmillStreamManager)) + .collect(Collectors.toList())); + + return connectionAndSenderFuture + .thenApply( + connectionsAndSenders -> + connectionsAndSenders.stream() + .collect(toImmutableMap(Pair::getLeft, Pair::getRight))) + .toCompletableFuture(); + } + /** Add up all the throttle times of all streams including GetWorkerMetadataStream. */ public long getAndResetThrottleTimes() { return connections.get().windmillStreams().values().stream() @@ -331,30 +412,6 @@ private synchronized ImmutableMap createNewWindmil endpoint, WindmillConnection.from(endpoint, this::createWindmillStub)))); } - private synchronized ImmutableMap - closeStaleStreamsAndCreateNewStreams(Collection newWindmillConnections) { - ImmutableMap currentStreams = - connections.get().windmillStreams(); - - // Close the streams that are no longer valid. - currentStreams.entrySet().stream() - .filter( - connectionAndStream -> !newWindmillConnections.contains(connectionAndStream.getKey())) - .forEach( - entry -> { - entry.getValue().closeAllStreams(); - entry.getKey().directEndpoint().ifPresent(channelCachingStubFactory::remove); - }); - - return newWindmillConnections.stream() - .collect( - toImmutableMap( - Function.identity(), - newConnection -> - Optional.ofNullable(currentStreams.get(newConnection)) - .orElseGet(() -> createAndStartWindmillStreamSenderFor(newConnection)))); - } - private ImmutableMap> createNewGlobalDataStreams( ImmutableMap newGlobalDataEndpoints) { ImmutableMap> currentGlobalDataStreams = @@ -400,7 +457,8 @@ private WindmillStreamSender createAndStartWindmillStreamSenderFor( GetWorkBudget.noBudget(), streamFactory, workItemScheduler, - workCommitterFactory); + workCommitterFactory, + keyedGetDataFn); windmillStreamSender.startStreams(); return windmillStreamSender; } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/WindmillStreamSender.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/WindmillStreamSender.java index e9f008eb522eb..9ed78560dfb81 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/WindmillStreamSender.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/WindmillStreamSender.java @@ -19,10 +19,12 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; +import java.util.function.BiFunction; import java.util.function.Function; import java.util.function.Supplier; import javax.annotation.concurrent.ThreadSafe; import org.apache.beam.runners.dataflow.worker.windmill.CloudWindmillServiceV1Alpha1Grpc.CloudWindmillServiceV1Alpha1Stub; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.GetWorkRequest; import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStream.CommitWorkStream; import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStream.GetDataStream; @@ -70,7 +72,11 @@ private WindmillStreamSender( AtomicReference getWorkBudget, GrpcWindmillStreamFactory streamingEngineStreamFactory, WorkItemScheduler workItemScheduler, - Function workCommitterFactory) { + Function workCommitterFactory, + Function< + GetDataStream, + BiFunction> + keyedGetDataFn) { this.started = new AtomicBoolean(false); this.getWorkBudget = getWorkBudget; this.streamingEngineThrottleTimers = StreamingEngineThrottleTimers.create(); @@ -84,10 +90,11 @@ private WindmillStreamSender( () -> streamingEngineStreamFactory.createGetDataStream( stub, streamingEngineThrottleTimers.getDataThrottleTimer())); + this.commitWorkStream = Suppliers.memoize( () -> - streamingEngineStreamFactory.createCommitWorkStream( + streamingEngineStreamFactory.createDirectCommitWorkStream( stub, streamingEngineThrottleTimers.commitWorkThrottleTimer())); this.workCommitter = Suppliers.memoize(() -> workCommitterFactory.apply(commitWorkStream.get())); @@ -100,6 +107,7 @@ private WindmillStreamSender( streamingEngineThrottleTimers.getWorkThrottleTimer(), getDataStream, workCommitter, + keyedGetDataFn, workItemScheduler)); } @@ -109,14 +117,19 @@ public static WindmillStreamSender create( GetWorkBudget getWorkBudget, GrpcWindmillStreamFactory streamingEngineStreamFactory, WorkItemScheduler workItemScheduler, - Function workCommitterFactory) { + Function workCommitterFactory, + Function< + GetDataStream, + BiFunction> + keyedGetDataFn) { return new WindmillStreamSender( stub, getWorkRequest, new AtomicReference<>(getWorkBudget), streamingEngineStreamFactory, workItemScheduler, - workCommitterFactory); + workCommitterFactory, + keyedGetDataFn); } private static GetWorkRequest withRequestBudget(GetWorkRequest request, GetWorkBudget budget) { diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/stubs/WindmillChannelFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/stubs/WindmillChannelFactory.java index 9aec29a3ba4d7..be565ece0ba7c 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/stubs/WindmillChannelFactory.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/stubs/WindmillChannelFactory.java @@ -36,7 +36,6 @@ /** Utility class used to create different RPC Channels. */ public final class WindmillChannelFactory { public static final String LOCALHOST = "localhost"; - private static final int DEFAULT_GRPC_PORT = 443; private static final int MAX_REMOTE_TRACE_EVENTS = 100; private WindmillChannelFactory() {} @@ -55,31 +54,24 @@ public static Channel localhostChannel(int port) { public static ManagedChannel remoteChannel( WindmillServiceAddress windmillServiceAddress, int windmillServiceRpcChannelTimeoutSec) { switch (windmillServiceAddress.getKind()) { - case IPV6: - return remoteChannel(windmillServiceAddress.ipv6(), windmillServiceRpcChannelTimeoutSec); case GCP_SERVICE_ADDRESS: return remoteChannel( windmillServiceAddress.gcpServiceAddress(), windmillServiceRpcChannelTimeoutSec); - // switch is exhaustive will never happen. case AUTHENTICATED_GCP_SERVICE_ADDRESS: - return remoteDirectChannel( - windmillServiceAddress.authenticatedGcpServiceAddress(), - windmillServiceRpcChannelTimeoutSec); + return remoteDirectChannel(windmillServiceAddress.authenticatedGcpServiceAddress()); + // switch is exhaustive will never happen. default: throw new UnsupportedOperationException( - "Only IPV6, GCP_SERVICE_ADDRESS, AUTHENTICATED_GCP_SERVICE_ADDRESS are supported WindmillServiceAddresses."); + "Only GCP_SERVICE_ADDRESS or AUTHENTICATED_GCP_SERVICE_ADDRESS are supported WindmillServiceAddress Kinds."); } } static ManagedChannel remoteDirectChannel( - AuthenticatedGcpServiceAddress authenticatedGcpServiceAddress, - int windmillServiceRpcChannelTimeoutSec) { - return withDefaultChannelOptions( - AltsChannelBuilder.forAddress( - authenticatedGcpServiceAddress.gcpServiceAddress().getHost(), - authenticatedGcpServiceAddress.gcpServiceAddress().getPort()) - .overrideAuthority(authenticatedGcpServiceAddress.authenticatingService()), - windmillServiceRpcChannelTimeoutSec) + AuthenticatedGcpServiceAddress authenticatedGcpServiceAddress) { + return AltsChannelBuilder.forAddress( + authenticatedGcpServiceAddress.gcpServiceAddress().getHost(), + authenticatedGcpServiceAddress.gcpServiceAddress().getPort()) + .overrideAuthority(authenticatedGcpServiceAddress.authenticatingService()) .build(); } @@ -105,17 +97,6 @@ public static Channel remoteChannel( } } - public static ManagedChannel remoteChannel( - Inet6Address directEndpoint, int windmillServiceRpcChannelTimeoutSec) { - try { - return createRemoteChannel( - NettyChannelBuilder.forAddress(new InetSocketAddress(directEndpoint, DEFAULT_GRPC_PORT)), - windmillServiceRpcChannelTimeoutSec); - } catch (SSLException sslException) { - throw new WindmillChannelCreationException(directEndpoint.toString(), sslException); - } - } - @SuppressWarnings("nullness") private static ManagedChannel createRemoteChannel( NettyChannelBuilder channelBuilder, int windmillServiceRpcChannelTimeoutSec) @@ -144,7 +125,7 @@ private static > T withDefaultChannelOpti .maxInboundMetadataSize(1024 * 1024); } - public static class WindmillChannelCreationException extends IllegalStateException { + private static class WindmillChannelCreationException extends RuntimeException { private WindmillChannelCreationException(HostAndPort endpoint, SSLException sourceException) { super( String.format( @@ -153,7 +134,7 @@ private WindmillChannelCreationException(HostAndPort endpoint, SSLException sour sourceException); } - WindmillChannelCreationException(String directEndpoint, Throwable sourceException) { + private WindmillChannelCreationException(String directEndpoint, Throwable sourceException) { super( String.format( "Exception thrown when trying to create channel to endpoint={%s}", directEndpoint), diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/budget/EvenGetWorkBudgetDistributor.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/budget/EvenGetWorkBudgetDistributor.java index 3a17222d3e6bd..27714b9311314 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/budget/EvenGetWorkBudgetDistributor.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/budget/EvenGetWorkBudgetDistributor.java @@ -80,7 +80,7 @@ public void distributeBudget( private ImmutableMap computeDesiredBudgets( ImmutableCollection streams, GetWorkBudget totalGetWorkBudget) { GetWorkBudget activeWorkBudget = activeWorkBudgetSupplier.get(); - LOG.info("Current active work budget: {}", activeWorkBudget); + LOG.debug("Current active work budget: {}", activeWorkBudget); // TODO: Fix possibly non-deterministic handing out of budgets. // Rounding up here will drift upwards over the lifetime of the streams. GetWorkBudget budgetPerStream = diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/ActiveWorkRefreshers.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/ActiveWorkRefreshers.java deleted file mode 100644 index 5a59a7f1ae01d..0000000000000 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/ActiveWorkRefreshers.java +++ /dev/null @@ -1,50 +0,0 @@ -/* - * 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.windmill.work.refresh; - -import java.util.Collection; -import java.util.List; -import java.util.Map; -import java.util.concurrent.ScheduledExecutorService; -import java.util.function.Consumer; -import java.util.function.Supplier; -import org.apache.beam.runners.dataflow.worker.DataflowExecutionStateSampler; -import org.apache.beam.runners.dataflow.worker.streaming.ComputationState; -import org.apache.beam.runners.dataflow.worker.windmill.Windmill.HeartbeatRequest; -import org.joda.time.Instant; - -/** Utility class for {@link ActiveWorkRefresher}. */ -public final class ActiveWorkRefreshers { - public static ActiveWorkRefresher createDispatchedActiveWorkRefresher( - Supplier clock, - int activeWorkRefreshPeriodMillis, - int stuckCommitDurationMillis, - Supplier> computations, - DataflowExecutionStateSampler sampler, - Consumer>> activeWorkRefresherFn, - ScheduledExecutorService scheduledExecutorService) { - return new DispatchedActiveWorkRefresher( - clock, - activeWorkRefreshPeriodMillis, - stuckCommitDurationMillis, - computations, - sampler, - activeWorkRefresherFn, - scheduledExecutorService); - } -} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/DispatchedActiveWorkRefresher.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/DispatchedActiveWorkRefresher.java index f81233498fe32..5a9d214a7ecab 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/DispatchedActiveWorkRefresher.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/DispatchedActiveWorkRefresher.java @@ -17,24 +17,30 @@ */ package org.apache.beam.runners.dataflow.worker.windmill.work.refresh; +import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap.toImmutableMap; + import java.util.Collection; -import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.concurrent.ScheduledExecutorService; import java.util.function.Consumer; import java.util.function.Supplier; +import javax.annotation.concurrent.ThreadSafe; import org.apache.beam.runners.dataflow.worker.DataflowExecutionStateSampler; import org.apache.beam.runners.dataflow.worker.streaming.ComputationState; import org.apache.beam.runners.dataflow.worker.windmill.Windmill; +import org.apache.beam.sdk.annotations.Internal; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.joda.time.Duration; import org.joda.time.Instant; -final class DispatchedActiveWorkRefresher extends ActiveWorkRefresher { +@Internal +@ThreadSafe +public final class DispatchedActiveWorkRefresher extends ActiveWorkRefresher { private final Consumer>> activeWorkRefresherFn; - DispatchedActiveWorkRefresher( + private DispatchedActiveWorkRefresher( Supplier clock, int activeWorkRefreshPeriodMillis, int stuckCommitDurationMillis, @@ -52,16 +58,38 @@ final class DispatchedActiveWorkRefresher extends ActiveWorkRefresher { this.activeWorkRefresherFn = activeWorkRefresherFn; } + public static DispatchedActiveWorkRefresher create( + Supplier clock, + int activeWorkRefreshPeriodMillis, + int stuckCommitDurationMillis, + Supplier> computations, + DataflowExecutionStateSampler sampler, + Consumer>> activeWorkRefresherFn, + ScheduledExecutorService scheduledExecutorService) { + return new DispatchedActiveWorkRefresher( + clock, + activeWorkRefreshPeriodMillis, + stuckCommitDurationMillis, + computations, + sampler, + activeWorkRefresherFn, + scheduledExecutorService); + } + @Override protected void refreshActiveWork() { - Map> heartbeats = new HashMap<>(); Instant refreshDeadline = clock.get().minus(Duration.millis(activeWorkRefreshPeriodMillis)); - for (ComputationState computationState : computations.get()) { - heartbeats.put( - computationState.getComputationId(), - computationState.getKeyHeartbeats(refreshDeadline, sampler)); - } + ImmutableMap> heartbeats = + computations.get().stream() + .collect( + toImmutableMap( + ComputationState::getComputationId, + computationState -> + HeartbeatRequests.getRefreshableKeyHeartbeats( + computationState.currentActiveWorkReadOnly(), + refreshDeadline, + sampler))); activeWorkRefresherFn.accept(heartbeats); } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/FanOutActiveWorkRefresher.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/FanOutActiveWorkRefresher.java new file mode 100644 index 0000000000000..42f1aeeae7141 --- /dev/null +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/FanOutActiveWorkRefresher.java @@ -0,0 +1,136 @@ +/* + * 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.windmill.work.refresh; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.function.Consumer; +import java.util.function.Supplier; +import javax.annotation.concurrent.ThreadSafe; +import org.apache.beam.runners.dataflow.worker.DataflowExecutionStateSampler; +import org.apache.beam.runners.dataflow.worker.streaming.ComputationState; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill.HeartbeatRequest; +import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStream.GetDataStream; +import org.apache.beam.sdk.annotations.Internal; +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.collect.ImmutableListMultimap; +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; + +/** + * {@link ActiveWorkRefresher} implementation that fans out refreshes active work to multiple {@link + * GetDataStream}. + */ +@Internal +@ThreadSafe +public class FanOutActiveWorkRefresher extends ActiveWorkRefresher { + + private static final String REFRESH_ACTIVE_WORK_EXECUTOR = "RefreshWorkWithFanOut"; + private final Consumer>>> + refreshActiveWorkFn; + + private FanOutActiveWorkRefresher( + Supplier clock, + int activeWorkRefreshPeriodMillis, + int stuckCommitDurationMillis, + Supplier> computations, + DataflowExecutionStateSampler sampler, + Consumer>>> refreshActiveWorkFn, + ScheduledExecutorService activeWorkRefreshExecutor) { + super( + clock, + activeWorkRefreshPeriodMillis, + stuckCommitDurationMillis, + computations, + sampler, + activeWorkRefreshExecutor); + this.refreshActiveWorkFn = refreshActiveWorkFn; + } + + public static ActiveWorkRefresher create( + Supplier clock, + int activeWorkRefreshPeriodMillis, + int stuckCommitDurationMillis, + Supplier> computations, + DataflowExecutionStateSampler sampler, + Consumer>>> refreshActiveWorkFn) { + return new FanOutActiveWorkRefresher( + clock, + activeWorkRefreshPeriodMillis, + stuckCommitDurationMillis, + computations, + sampler, + refreshActiveWorkFn, + Executors.newSingleThreadScheduledExecutor( + new ThreadFactoryBuilder().setNameFormat(REFRESH_ACTIVE_WORK_EXECUTOR).build())); + } + + @VisibleForTesting + static FanOutActiveWorkRefresher forTesting( + Supplier clock, + int activeWorkRefreshPeriodMillis, + int stuckCommitDurationMillis, + Supplier> computations, + DataflowExecutionStateSampler sampler, + Consumer>>> refreshActiveWorkFn, + ScheduledExecutorService activeWorkRefreshExecutor) { + return new FanOutActiveWorkRefresher( + clock, + activeWorkRefreshPeriodMillis, + stuckCommitDurationMillis, + computations, + sampler, + refreshActiveWorkFn, + activeWorkRefreshExecutor); + } + + @Override + protected void refreshActiveWork() { + Instant refreshDeadline = clock.get().minus(Duration.millis(activeWorkRefreshPeriodMillis)); + + Map>> fannedOutHeartbeatRequests = + new HashMap<>(); + for (ComputationState computationState : computations.get()) { + String computationId = computationState.getComputationId(); + + // Get heartbeat requests for computation's current active work, aggregated by GetDataStream + // to correctly fan-out the heartbeat requests. + ImmutableListMultimap heartbeats = + HeartbeatRequests.getRefreshableDirectKeyHeartbeats( + computationState.currentActiveWorkReadOnly(), refreshDeadline, sampler); + // Aggregate the heartbeats across computations by GetDataStream for correct fan out. + for (Map.Entry> heartbeatsPerStream : + heartbeats.asMap().entrySet()) { + Map> existingHeartbeats = + fannedOutHeartbeatRequests.computeIfAbsent( + heartbeatsPerStream.getKey(), ignored -> new HashMap<>()); + List existingHeartbeatsForComputation = + existingHeartbeats.computeIfAbsent(computationId, ignored -> new ArrayList<>()); + existingHeartbeatsForComputation.addAll(heartbeatsPerStream.getValue()); + } + } + + refreshActiveWorkFn.accept(fannedOutHeartbeatRequests); + } +} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/HeartbeatRequests.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/HeartbeatRequests.java new file mode 100644 index 0000000000000..9b79d587b0cbd --- /dev/null +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/HeartbeatRequests.java @@ -0,0 +1,113 @@ +/* + * 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.windmill.work.refresh; + +import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList.toImmutableList; +import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableListMultimap.toImmutableListMultimap; + +import java.util.Collection; +import java.util.Map; +import java.util.stream.Stream; +import org.apache.beam.repackaged.core.org.apache.commons.lang3.tuple.Pair; +import org.apache.beam.runners.dataflow.worker.DataflowExecutionStateSampler; +import org.apache.beam.runners.dataflow.worker.streaming.ShardedKey; +import org.apache.beam.runners.dataflow.worker.streaming.Work; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill.HeartbeatRequest; +import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStream.GetDataStream; +import org.apache.beam.sdk.annotations.Internal; +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.collect.ImmutableList; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableListMultimap; +import org.joda.time.Instant; + +/** Helper factory class for creating heartbeat requests. */ +@Internal +public final class HeartbeatRequests { + + private HeartbeatRequests() {} + + static ImmutableList getRefreshableKeyHeartbeats( + ImmutableListMultimap activeWork, + Instant refreshDeadline, + DataflowExecutionStateSampler sampler) { + return activeWork.asMap().entrySet().stream() + .flatMap(entry -> toHeartbeatRequestStream(entry, refreshDeadline, sampler)) + .collect(toImmutableList()); + } + + static ImmutableListMultimap getRefreshableDirectKeyHeartbeats( + ImmutableListMultimap activeWork, + Instant refreshDeadline, + DataflowExecutionStateSampler sampler) { + return activeWork.asMap().entrySet().stream() + .flatMap(e -> toDirectHeartbeatRequest(e, refreshDeadline, sampler)) + .collect(toImmutableListMultimap(Pair::getKey, Pair::getValue)); + } + + private static Stream> toDirectHeartbeatRequest( + Map.Entry> shardedKeyAndWorkQueue, + Instant refreshDeadline, + DataflowExecutionStateSampler sampler) { + ShardedKey shardedKey = shardedKeyAndWorkQueue.getKey(); + Collection workQueue = shardedKeyAndWorkQueue.getValue(); + return getRefreshableWork(workQueue, refreshDeadline) + // If the stream was explicitly closed fail the Work as the backend worker is invalid. + .peek(HeartbeatRequests::failWorkForClosedStream) + // Don't send heartbeats for queued work we already know is failed. + .filter(work -> !work.isFailed()) + .map( + work -> + Pair.of( + Preconditions.checkNotNull(work.getDataStream()), + createHeartbeatRequest(shardedKey, work, sampler))); + } + + private static Stream toHeartbeatRequestStream( + Map.Entry> shardedKeyAndWorkQueue, + Instant refreshDeadline, + DataflowExecutionStateSampler sampler) { + ShardedKey shardedKey = shardedKeyAndWorkQueue.getKey(); + Collection workQueue = shardedKeyAndWorkQueue.getValue(); + + return getRefreshableWork(workQueue, refreshDeadline) + // Don't send heartbeats for queued work we already know is failed. + .filter(work -> !work.isFailed()) + .map(work -> createHeartbeatRequest(shardedKey, work, sampler)); + } + + private static HeartbeatRequest createHeartbeatRequest( + ShardedKey shardedKey, Work work, DataflowExecutionStateSampler sampler) { + return HeartbeatRequest.newBuilder() + .setShardingKey(shardedKey.shardingKey()) + .setWorkToken(work.getWorkItem().getWorkToken()) + .setCacheToken(work.getWorkItem().getCacheToken()) + .addAllLatencyAttribution(work.getLatencyAttributions(/* isHeartbeat= */ true, sampler)) + .build(); + } + + private static Stream getRefreshableWork( + Collection workQueue, Instant refreshDeadline) { + return workQueue.stream().filter(work -> work.getStartTime().isBefore(refreshDeadline)); + } + + private static void failWorkForClosedStream(Work work) { + if (Preconditions.checkNotNull(work.getDataStream()).isClosed()) { + work.setFailed(); + } + } +} diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/FakeWindmillServer.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/FakeWindmillServer.java index 127d46b7caf61..2e98748d228f8 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/FakeWindmillServer.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/FakeWindmillServer.java @@ -287,6 +287,11 @@ public boolean awaitTermination(int time, TimeUnit unit) throws InterruptedExcep public Instant startTime() { return startTime; } + + @Override + public boolean isClosed() { + return done.getCount() == 0; + } }; } @@ -359,6 +364,11 @@ public boolean awaitTermination(int time, TimeUnit unit) { public Instant startTime() { return startTime; } + + @Override + public boolean isClosed() { + return false; + } }; } @@ -442,6 +452,11 @@ public boolean awaitTermination(int time, TimeUnit unit) { public Instant startTime() { return startTime; } + + @Override + public boolean isClosed() { + return false; + } }; } 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 e3aee23e511e3..9b39a86055af3 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 @@ -126,6 +126,9 @@ import org.apache.beam.runners.dataflow.worker.windmill.Windmill.Timer.Type; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.WatermarkHold; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.WorkItemCommitRequest; +import org.apache.beam.runners.dataflow.worker.windmill.client.grpc.GrpcDispatcherClient; +import org.apache.beam.runners.dataflow.worker.windmill.client.grpc.stubs.WindmillChannelFactory; +import org.apache.beam.runners.dataflow.worker.windmill.testing.FakeWindmillStubFactory; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.Coder.Context; import org.apache.beam.sdk.coders.CollectionCoder; @@ -835,8 +838,14 @@ private DataflowWorkerHarnessOptions createTestingPipelineOptions(String... args private StreamingDataflowWorker makeWorker( StreamingDataflowWorkerTestParams streamingDataflowWorkerTestParams) { + FakeWindmillStubFactory stubFactory = + new FakeWindmillStubFactory( + () -> WindmillChannelFactory.inProcessChannel("StreamingDataflowWorkerTest")); + GrpcDispatcherClient dispatcherClient = GrpcDispatcherClient.create(stubFactory); StreamingDataflowWorker worker = StreamingDataflowWorker.forTesting( + stubFactory, + dispatcherClient, streamingDataflowWorkerTestParams.stateNameMappings(), server, Collections.singletonList( @@ -844,7 +853,6 @@ private StreamingDataflowWorker makeWorker( IntrinsicMapTaskExecutorFactory.defaultFactory(), mockWorkUnitClient, streamingDataflowWorkerTestParams.options(), - streamingDataflowWorkerTestParams.publishCounters(), hotKeyLogger, streamingDataflowWorkerTestParams.clock(), streamingDataflowWorkerTestParams.executorSupplier(), diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/harness/StreamingWorkerStatusReporterTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/harness/StreamingWorkerStatusReporterTest.java index 7e65a495638f0..3d32af6a33b89 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/harness/StreamingWorkerStatusReporterTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/harness/StreamingWorkerStatusReporterTest.java @@ -59,11 +59,10 @@ public void setUp() { @Test public void testOverrideMaximumThreadCount() throws Exception { StreamingWorkerStatusReporter reporter = - StreamingWorkerStatusReporter.forTesting( - true, + StreamingWorkerStatusReporter.create( mockWorkUnitClient, () -> DEFAULT_WINDMILL_QUOTA_THROTTLE_TIME, - () -> Collections.emptyList(), + Collections::emptyList, mockFailureTracker, StreamingCounters.create(), mockMemoryMonitor, @@ -85,11 +84,10 @@ public void testOverrideMaximumThreadCount() throws Exception { @Test public void testHandleEmptyWorkerMessageResponse() throws Exception { StreamingWorkerStatusReporter reporter = - StreamingWorkerStatusReporter.forTesting( - true, + StreamingWorkerStatusReporter.create( mockWorkUnitClient, () -> DEFAULT_WINDMILL_QUOTA_THROTTLE_TIME, - () -> Collections.emptyList(), + Collections::emptyList, mockFailureTracker, StreamingCounters.create(), mockMemoryMonitor, @@ -97,9 +95,8 @@ public void testHandleEmptyWorkerMessageResponse() throws Exception { (threadName) -> Executors.newSingleThreadScheduledExecutor(), DEFAULT_HARNESS_REPORTING_PERIOD, DEFAULT_PER_WORKER_METRICS_PERIOD); - WorkerMessageResponse workerMessageResponse = new WorkerMessageResponse(); when(mockWorkUnitClient.reportWorkerMessage(any())) - .thenReturn(Collections.singletonList(workerMessageResponse)); + .thenReturn(Collections.singletonList(new WorkerMessageResponse())); reporter.reportPeriodicWorkerMessage(); verify(mockExecutor, Mockito.times(0)).setMaximumPoolSize(anyInt(), anyInt()); } diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/WindmillStreamPoolTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/WindmillStreamPoolTest.java index a2f5e71d04c3f..01f7391dd67f3 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/WindmillStreamPoolTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/WindmillStreamPoolTest.java @@ -250,5 +250,10 @@ public boolean awaitTermination(int time, TimeUnit unit) { public Instant startTime() { return startTime; } + + @Override + public boolean isClosed() { + return closed; + } } } diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/commits/StreamingEngineWorkCommitterTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/commits/StreamingEngineWorkCommitterTest.java index d53690938aef7..9ed8eacc3c69d 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/commits/StreamingEngineWorkCommitterTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/commits/StreamingEngineWorkCommitterTest.java @@ -256,6 +256,8 @@ public void testStop_drainsCommitQueue() { Supplier fakeCommitWorkStream = () -> new CommitWorkStream() { + private boolean closed = false; + @Override public RequestBatcher batcher() { return new RequestBatcher() { @@ -273,7 +275,9 @@ public void flush() {} } @Override - public void close() {} + public void close() { + closed = true; + } @Override public boolean awaitTermination(int time, TimeUnit unit) { @@ -284,6 +288,11 @@ public boolean awaitTermination(int time, TimeUnit unit) { public Instant startTime() { return Instant.now(); } + + @Override + public boolean isClosed() { + return closed; + } }; commitWorkStreamFactory = diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/ChannelzServletTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/ChannelzServletTest.java index 96c675169a7d2..d234cf424767b 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/ChannelzServletTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/ChannelzServletTest.java @@ -56,7 +56,8 @@ public void testRendersAllChannels() throws UnsupportedEncodingException { fakeWindmillServer.setWindmillServiceEndpoints( ImmutableSet.of(HostAndPort.fromHost(windmill1), HostAndPort.fromHost(windmill2))); options.setChannelzShowOnlyWindmillServiceChannels(false); - ChannelzServlet channelzServlet = new ChannelzServlet("/channelz", options, fakeWindmillServer); + ChannelzServlet channelzServlet = + new ChannelzServlet("/channelz", options, fakeWindmillServer::getWindmillServiceEndpoints); StringWriter stringWriter = new StringWriter(); PrintWriter writer = new PrintWriter(stringWriter); channelzServlet.captureData(writer); @@ -88,7 +89,8 @@ public void testRendersOnlyWindmillChannels() throws UnsupportedEncodingExceptio fakeWindmillServer.setWindmillServiceEndpoints( ImmutableSet.of(HostAndPort.fromHost(windmill1), HostAndPort.fromHost(windmill2))); options.setChannelzShowOnlyWindmillServiceChannels(true); - ChannelzServlet channelzServlet = new ChannelzServlet("/channelz", options, fakeWindmillServer); + ChannelzServlet channelzServlet = + new ChannelzServlet("/channelz", options, fakeWindmillServer::getWindmillServiceEndpoints); StringWriter stringWriter = new StringWriter(); PrintWriter writer = new PrintWriter(stringWriter); channelzServlet.captureData(writer); diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/StreamingEngineClientTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/StreamingEngineClientTest.java index bc3afaff1b385..23f4d3edaf6d4 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/StreamingEngineClientTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/StreamingEngineClientTest.java @@ -41,6 +41,7 @@ import java.util.stream.Collectors; import javax.annotation.Nullable; import org.apache.beam.runners.dataflow.worker.windmill.CloudWindmillMetadataServiceV1Alpha1Grpc; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.GetWorkRequest; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.JobHeader; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.WorkerMetadataRequest; @@ -97,7 +98,6 @@ public class StreamingEngineClientTest { .build(); @Rule public final GrpcCleanupRule grpcCleanup = new GrpcCleanupRule(); - @Rule public transient Timeout globalTimeout = Timeout.seconds(600); private final MutableHandlerRegistry serviceRegistry = new MutableHandlerRegistry(); private final GrpcWindmillStreamFactory streamFactory = spy(GrpcWindmillStreamFactory.of(JOB_HEADER).build()); @@ -109,7 +109,7 @@ public class StreamingEngineClientTest { private final GrpcDispatcherClient dispatcherClient = GrpcDispatcherClient.forTesting( stubFactory, new ArrayList<>(), new ArrayList<>(), new HashSet<>()); - + @Rule public transient Timeout globalTimeout = Timeout.seconds(600); private Server fakeStreamingEngineServer; private CountDownLatch getWorkerMetadataReady; private GetWorkerMetadataTestStub fakeGetWorkerMetadataStub; @@ -181,7 +181,9 @@ private StreamingEngineClient newStreamingEngineClient( getWorkBudgetDistributor, dispatcherClient, CLIENT_ID, - ignored -> mock(WorkCommitter.class)); + ignored -> mock(WorkCommitter.class), + getDataStream -> + (computationId, request) -> Windmill.KeyedGetDataResponse.getDefaultInstance()); } @Test @@ -235,7 +237,13 @@ public void testStreamsStartCorrectly() throws InterruptedException { verify(streamFactory, times(2)) .createDirectGetWorkStream( - any(), eq(getWorkRequest(0, 0)), any(), any(), any(), eq(noOpProcessWorkItemFn())); + any(), + eq(getWorkRequest(0, 0)), + any(), + any(), + any(), + any(), + eq(noOpProcessWorkItemFn())); verify(streamFactory, times(2)).createGetDataStream(any(), any()); verify(streamFactory, times(2)).createCommitWorkStream(any(), any()); diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/WindmillStreamSenderTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/WindmillStreamSenderTest.java index 162c69509ae15..8eb62297dbc1a 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/WindmillStreamSenderTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/WindmillStreamSenderTest.java @@ -28,6 +28,7 @@ import org.apache.beam.runners.dataflow.worker.windmill.CloudWindmillServiceV1Alpha1Grpc; import org.apache.beam.runners.dataflow.worker.windmill.CloudWindmillServiceV1Alpha1Grpc.CloudWindmillServiceV1Alpha1Stub; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.GetWorkRequest; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.JobHeader; import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStream.CommitWorkStream; @@ -105,6 +106,7 @@ public void testStartStream_startsAllStreams() { any(ThrottleTimer.class), any(), any(), + any(), eq(workItemScheduler)); verify(streamFactory).createGetDataStream(eq(stub), any(ThrottleTimer.class)); @@ -136,6 +138,7 @@ public void testStartStream_onlyStartsStreamsOnce() { any(ThrottleTimer.class), any(), any(), + any(), eq(workItemScheduler)); verify(streamFactory, times(1)).createGetDataStream(eq(stub), any(ThrottleTimer.class)); @@ -170,6 +173,7 @@ public void testStartStream_onlyStartsStreamsOnceConcurrent() throws Interrupted any(ThrottleTimer.class), any(), any(), + any(), eq(workItemScheduler)); verify(streamFactory, times(1)).createGetDataStream(eq(stub), any(ThrottleTimer.class)); @@ -203,12 +207,12 @@ public void testCloseAllStreams_closesAllStreams() { any(ThrottleTimer.class), any(), any(), + any(), eq(workItemScheduler))) .thenReturn(mockGetWorkStream); - when(mockStreamFactory.createGetDataStream(eq(stub), any(ThrottleTimer.class))) .thenReturn(mockGetDataStream); - when(mockStreamFactory.createCommitWorkStream(eq(stub), any(ThrottleTimer.class))) + when(mockStreamFactory.createDirectCommitWorkStream(eq(stub), any(ThrottleTimer.class))) .thenReturn(mockCommitWorkStream); WindmillStreamSender windmillStreamSender = @@ -236,6 +240,8 @@ private WindmillStreamSender newWindmillStreamSender( budget, streamFactory, workItemScheduler, - ignored -> mock(WorkCommitter.class)); + ignored -> mock(WorkCommitter.class), + getDataStream -> + (computationId, request) -> Windmill.KeyedGetDataResponse.getDefaultInstance()); } } diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/work/budget/EvenGetWorkBudgetDistributorTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/work/budget/EvenGetWorkBudgetDistributorTest.java index 83ae8aa22ce3e..021d2651e66dc 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/work/budget/EvenGetWorkBudgetDistributorTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/work/budget/EvenGetWorkBudgetDistributorTest.java @@ -259,6 +259,8 @@ private WindmillStreamSender createWindmillStreamSender(GetWorkBudget getWorkBud .build()) .build(), (workItem, watermarks, processingContext, ackWorkItemQueued, getWorkStreamLatencies) -> {}, - ignored -> mock(WorkCommitter.class)); + ignored -> mock(WorkCommitter.class), + getDataStream -> + (computationId, request) -> Windmill.KeyedGetDataResponse.getDefaultInstance()); } } diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/DispatchedActiveWorkRefresherTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/DispatchedActiveWorkRefresherTest.java index 175c8421ff8e0..01a46a12e3eb1 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/DispatchedActiveWorkRefresherTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/DispatchedActiveWorkRefresherTest.java @@ -98,7 +98,7 @@ private ActiveWorkRefresher createActiveWorkRefresher( int stuckCommitDurationMillis, Supplier> computations, Consumer>> activeWorkRefresherFn) { - return new DispatchedActiveWorkRefresher( + return DispatchedActiveWorkRefresher.create( clock, activeWorkRefreshPeriodMillis, stuckCommitDurationMillis, diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/FanOutActiveWorkRefresherTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/FanOutActiveWorkRefresherTest.java new file mode 100644 index 0000000000000..9a34760d860b1 --- /dev/null +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/FanOutActiveWorkRefresherTest.java @@ -0,0 +1,221 @@ +/* + * 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.windmill.work.refresh; + +import static com.google.common.truth.Truth.assertThat; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +import com.google.api.services.dataflow.model.MapTask; +import com.google.common.truth.Correspondence; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; +import java.util.function.Consumer; +import java.util.function.Supplier; +import java.util.stream.Collectors; +import org.apache.beam.runners.dataflow.worker.DataflowExecutionStateSampler; +import org.apache.beam.runners.dataflow.worker.streaming.ComputationState; +import org.apache.beam.runners.dataflow.worker.streaming.ExecutableWork; +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.util.BoundedQueueExecutor; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill; +import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStream; +import org.apache.beam.runners.dataflow.worker.windmill.state.WindmillStateCache; +import org.apache.beam.runners.direct.Clock; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; +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.util.concurrent.ThreadFactoryBuilder; +import org.joda.time.Duration; +import org.joda.time.Instant; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +@RunWith(JUnit4.class) +public class FanOutActiveWorkRefresherTest { + private static final Supplier A_LONG_TIME_AGO = + () -> Instant.parse("1998-09-04T00:00:00Z"); + private static final String COMPUTATION_ID_PREFIX = "ComputationId-"; + + private static BoundedQueueExecutor workExecutor() { + return new BoundedQueueExecutor( + 1, + 60, + TimeUnit.SECONDS, + 1, + 10000000, + new ThreadFactoryBuilder().setNameFormat("DataflowWorkUnits-%d").setDaemon(true).build()); + } + + private static ComputationState createComputationState(int computationIdSuffix) { + return createComputationState( + computationIdSuffix, mock(WindmillStateCache.ForComputation.class)); + } + + private static ComputationState createComputationState( + int computationIdSuffix, WindmillStateCache.ForComputation stateCache) { + return new ComputationState( + COMPUTATION_ID_PREFIX + computationIdSuffix, + new MapTask().setStageName("stageName").setSystemName("systemName"), + workExecutor(), + new HashMap<>(), + stateCache); + } + + private ActiveWorkRefresher createActiveWorkRefresher( + Supplier clock, + int activeWorkRefreshPeriodMillis, + int stuckCommitDurationMillis, + Supplier> computations, + Consumer>>> + activeWorkRefresherFn) { + return FanOutActiveWorkRefresher.forTesting( + clock, + activeWorkRefreshPeriodMillis, + stuckCommitDurationMillis, + computations, + DataflowExecutionStateSampler.instance(), + activeWorkRefresherFn, + Executors.newSingleThreadScheduledExecutor()); + } + + private ExecutableWork createOldWork(int workIds, Consumer processWork) { + ShardedKey shardedKey = ShardedKey.create(ByteString.EMPTY, workIds); + return ExecutableWork.create( + Work.create( + Windmill.WorkItem.newBuilder() + .setWorkToken(workIds) + .setCacheToken(workIds) + .setKey(shardedKey.key()) + .setShardingKey(shardedKey.shardingKey()) + .build(), + Watermarks.builder().setInputDataWatermark(Instant.EPOCH).build(), + Work.createFanOutProcessingContext( + "computation", + (computationId, request) -> Windmill.KeyedGetDataResponse.getDefaultInstance(), + ignored -> {}, + mock(WindmillStream.GetDataStream.class)), + A_LONG_TIME_AGO, + ImmutableList.of()), + processWork); + } + + @Test + public void testActiveWorkRefresh() throws InterruptedException { + int activeWorkRefreshPeriodMillis = 100; + + // Block work processing to queue up the work. + CountDownLatch workIsProcessed = new CountDownLatch(1); + Consumer processWork = + ignored -> { + try { + workIsProcessed.await(); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + }; + + List computations = new ArrayList<>(); + Map> computationsAndWork = new HashMap<>(); + for (int i = 0; i < 5; i++) { + ComputationState computationState = createComputationState(i); + ExecutableWork fakeWork = createOldWork(i, processWork); + computationState.activateWork(fakeWork); + + computations.add(computationState); + List activeWorkForComputation = + computationsAndWork.computeIfAbsent( + computationState.getComputationId(), ignored -> new ArrayList<>()); + activeWorkForComputation.add(fakeWork); + when(fakeWork.work().getDataStream().isClosed()).thenReturn(false); + } + + Map>> + fanoutExpectedHeartbeats = new HashMap<>(); + CountDownLatch heartbeatsSent = new CountDownLatch(1); + TestClock fakeClock = new TestClock(Instant.now()); + ActiveWorkRefresher activeWorkRefresher = + createActiveWorkRefresher( + fakeClock::now, + activeWorkRefreshPeriodMillis, + 0, + () -> computations, + heartbeats -> { + fanoutExpectedHeartbeats.putAll(heartbeats); + heartbeatsSent.countDown(); + }); + + activeWorkRefresher.start(); + fakeClock.advance(Duration.millis(activeWorkRefreshPeriodMillis * 2)); + heartbeatsSent.await(); + activeWorkRefresher.stop(); + + assertThat(computationsAndWork.size()).isEqualTo(fanoutExpectedHeartbeats.size()); + for (Map.Entry>> + fanOutExpectedHeartbeat : fanoutExpectedHeartbeats.entrySet()) { + for (Map.Entry> expectedHeartbeat : + fanOutExpectedHeartbeat.getValue().entrySet()) { + String computationId = expectedHeartbeat.getKey(); + List heartbeatRequests = expectedHeartbeat.getValue(); + List work = + computationsAndWork.get(computationId).stream() + .map(ExecutableWork::work) + .collect(Collectors.toList()); + // Compare the heartbeatRequest's and Work's workTokens, cacheTokens, and shardingKeys. + assertThat(heartbeatRequests) + .comparingElementsUsing( + Correspondence.from( + (Windmill.HeartbeatRequest h, Work w) -> + h.getWorkToken() == w.getWorkItem().getWorkToken() + && h.getCacheToken() == w.getWorkItem().getWorkToken() + && h.getShardingKey() == w.getWorkItem().getShardingKey(), + "heartbeatRequest's and Work's workTokens, cacheTokens, and shardingKeys should be equal.")) + .containsExactlyElementsIn(work); + } + } + + activeWorkRefresher.stop(); + // Free the work processing threads. + workIsProcessed.countDown(); + } + + static class TestClock implements Clock { + private Instant time; + + private TestClock(Instant startTime) { + this.time = startTime; + } + + private synchronized void advance(Duration amount) { + time = time.plus(amount); + } + + @Override + public synchronized Instant now() { + return time; + } + } +} diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/HeartbeatRequestsTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/HeartbeatRequestsTest.java new file mode 100644 index 0000000000000..cd863216fe0f3 --- /dev/null +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/HeartbeatRequestsTest.java @@ -0,0 +1,289 @@ +/* + * 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.windmill.work.refresh; + +import static com.google.common.truth.Truth.assertThat; +import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList.toImmutableList; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +import com.google.auto.value.AutoValue; +import java.util.ArrayDeque; +import java.util.Collections; +import java.util.Deque; +import java.util.HashMap; +import java.util.Map; +import java.util.Objects; +import javax.annotation.Nullable; +import org.apache.beam.runners.dataflow.worker.DataflowExecutionStateSampler; +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.windmill.Windmill; +import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStream; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; +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.ImmutableListMultimap; +import org.joda.time.Instant; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +@RunWith(JUnit4.class) +public class HeartbeatRequestsTest { + + private Map> activeWork; + + private static Work createWork( + Windmill.WorkItem workItem, WindmillStream.GetDataStream getDataStream) { + return Work.create( + workItem, + Watermarks.builder().setInputDataWatermark(Instant.EPOCH).build(), + createProcessingContext(getDataStream), + Instant::now, + Collections.emptyList()); + } + + private static ShardedKey shardedKey(String str, long shardKey) { + return ShardedKey.create(ByteString.copyFromUtf8(str), shardKey); + } + + private static Work createWork(Windmill.WorkItem workItem) { + return Work.create( + workItem, + Watermarks.builder().setInputDataWatermark(Instant.EPOCH).build(), + createProcessingContext(), + Instant::now, + Collections.emptyList()); + } + + private static Work expiredWork(Windmill.WorkItem workItem) { + return Work.create( + workItem, + Watermarks.builder().setInputDataWatermark(Instant.EPOCH).build(), + createProcessingContext(), + () -> Instant.EPOCH, + Collections.emptyList()); + } + + private static Work.ProcessingContext createProcessingContext() { + return Work.createProcessingContext( + "computationId", + (computationId, request) -> Windmill.KeyedGetDataResponse.getDefaultInstance(), + ignored -> {}); + } + + private static Work.ProcessingContext createProcessingContext( + WindmillStream.GetDataStream getDataStream) { + return Work.createFanOutProcessingContext( + "computationId", + (computationId, request) -> Windmill.KeyedGetDataResponse.getDefaultInstance(), + ignored -> {}, + getDataStream); + } + + private static Work expiredWork( + Windmill.WorkItem workItem, WindmillStream.GetDataStream getDataStream) { + return Work.create( + workItem, + Watermarks.builder().setInputDataWatermark(Instant.EPOCH).build(), + createProcessingContext(getDataStream), + () -> Instant.EPOCH, + Collections.emptyList()); + } + + private static Windmill.WorkItem createWorkItem(long workToken, long cacheToken) { + return Windmill.WorkItem.newBuilder() + .setKey(ByteString.copyFromUtf8("")) + .setShardingKey(1) + .setWorkToken(workToken) + .setCacheToken(cacheToken) + .build(); + } + + @Before + public void setUp() { + activeWork = new HashMap<>(); + } + + @Test + public void testGetRefreshableKeyHeartbeats() { + Instant refreshDeadline = Instant.now(); + + Work freshWork = createWork(createWorkItem(3L, 3L)); + Work refreshableWork1 = expiredWork(createWorkItem(1L, 1L)); + refreshableWork1.setState(Work.State.COMMITTING); + Work refreshableWork2 = expiredWork(createWorkItem(2L, 2L)); + refreshableWork2.setState(Work.State.COMMITTING); + ShardedKey shardedKey1 = shardedKey("someKey", 1L); + ShardedKey shardedKey2 = shardedKey("anotherKey", 2L); + + activateWorkForKey(shardedKey1, refreshableWork1); + activateWorkForKey(shardedKey1, freshWork); + activateWorkForKey(shardedKey2, refreshableWork2); + + ImmutableList requests = + HeartbeatRequests.getRefreshableKeyHeartbeats( + currentActiveWork(), refreshDeadline, DataflowExecutionStateSampler.instance()); + + ImmutableList expected = + ImmutableList.of( + HeartbeatRequestShardingKeyWorkTokenAndCacheToken.from(shardedKey1, refreshableWork1), + HeartbeatRequestShardingKeyWorkTokenAndCacheToken.from(shardedKey2, refreshableWork2)); + + ImmutableList actual = + requests.stream() + .map(HeartbeatRequestShardingKeyWorkTokenAndCacheToken::from) + .collect(toImmutableList()); + + assertThat(actual).containsExactlyElementsIn(expected); + } + + @Test + public void testGetRefreshableFanoutKeyHeartbeats() { + Instant refreshDeadline = Instant.now(); + WindmillStream.GetDataStream getDataStream1 = mock(WindmillStream.GetDataStream.class); + when(getDataStream1.isClosed()).thenReturn(false); + WindmillStream.GetDataStream getDataStream2 = mock(WindmillStream.GetDataStream.class); + when(getDataStream2.isClosed()).thenReturn(false); + + Work freshWork = createWork(createWorkItem(3L, 3L), getDataStream1); + Work refreshableWork1 = expiredWork(createWorkItem(1L, 1L), getDataStream1); + refreshableWork1.setState(Work.State.COMMITTING); + Work refreshableWork2 = expiredWork(createWorkItem(2L, 2L), getDataStream2); + refreshableWork2.setState(Work.State.COMMITTING); + ShardedKey shardedKey1 = shardedKey("someKey", 1L); + ShardedKey shardedKey2 = shardedKey("anotherKey", 2L); + + activateWorkForKey(shardedKey1, refreshableWork1); + activateWorkForKey(shardedKey1, freshWork); + activateWorkForKey(shardedKey2, refreshableWork2); + + ImmutableListMultimap requests = + HeartbeatRequests.getRefreshableDirectKeyHeartbeats( + currentActiveWork(), refreshDeadline, DataflowExecutionStateSampler.instance()); + + ImmutableList expected = + ImmutableList.of( + HeartbeatRequestShardingKeyWorkTokenAndCacheToken.from(shardedKey1, refreshableWork1), + HeartbeatRequestShardingKeyWorkTokenAndCacheToken.from(shardedKey2, refreshableWork2)); + + ImmutableList actual = + requests.entries().stream() + .map( + entry -> + HeartbeatRequestShardingKeyWorkTokenAndCacheToken.from( + entry.getValue(), entry.getKey())) + .collect(toImmutableList()); + + assertThat(actual).containsExactlyElementsIn(expected); + } + + private void activateWorkForKey(ShardedKey shardedKey, Work work) { + Deque workQueue = activeWork.computeIfAbsent(shardedKey, ignored -> new ArrayDeque<>()); + workQueue.addLast(work); + } + + private ImmutableListMultimap currentActiveWork() { + ImmutableListMultimap.Builder currentActiveWork = + ImmutableListMultimap.builder(); + + for (Map.Entry> keyedWorkQueues : activeWork.entrySet()) { + currentActiveWork.putAll(keyedWorkQueues.getKey(), keyedWorkQueues.getValue()); + } + + return currentActiveWork.build(); + } + + @AutoValue + abstract static class HeartbeatRequestShardingKeyWorkTokenAndCacheToken { + + private static HeartbeatRequestShardingKeyWorkTokenAndCacheToken create( + long shardingKey, long workToken, long cacheToken) { + return new AutoValue_HeartbeatRequestsTest_HeartbeatRequestShardingKeyWorkTokenAndCacheToken( + shardingKey, workToken, cacheToken, null); + } + + private static HeartbeatRequestShardingKeyWorkTokenAndCacheToken create( + long shardingKey, + long workToken, + long cacheToken, + WindmillStream.GetDataStream getDataStream) { + return new AutoValue_HeartbeatRequestsTest_HeartbeatRequestShardingKeyWorkTokenAndCacheToken( + shardingKey, workToken, cacheToken, Objects.requireNonNull(getDataStream)); + } + + private static HeartbeatRequestShardingKeyWorkTokenAndCacheToken from( + Windmill.HeartbeatRequest heartbeatRequest) { + return create( + heartbeatRequest.getShardingKey(), + heartbeatRequest.getWorkToken(), + heartbeatRequest.getCacheToken()); + } + + private static HeartbeatRequestShardingKeyWorkTokenAndCacheToken from( + ShardedKey shardedKey, Work work) { + @Nullable WindmillStream.GetDataStream getDataStream = work.getDataStream(); + return getDataStream == null + ? create( + shardedKey.shardingKey(), + work.getWorkItem().getWorkToken(), + work.getWorkItem().getCacheToken()) + : create( + shardedKey.shardingKey(), + work.getWorkItem().getWorkToken(), + work.getWorkItem().getCacheToken(), + work.getDataStream()); + } + + private static HeartbeatRequestShardingKeyWorkTokenAndCacheToken from( + Windmill.HeartbeatRequest heartbeatRequest, WindmillStream.GetDataStream getDataStream) { + return create( + heartbeatRequest.getShardingKey(), + heartbeatRequest.getWorkToken(), + heartbeatRequest.getCacheToken(), + getDataStream); + } + + abstract long shardingKey(); + + abstract long workToken(); + + abstract long cacheToken(); + + abstract @Nullable WindmillStream.GetDataStream getDataStream(); + + @Override + public final boolean equals(Object obj) { + if (!(obj instanceof HeartbeatRequestShardingKeyWorkTokenAndCacheToken)) { + return false; + } + HeartbeatRequestShardingKeyWorkTokenAndCacheToken other = + (HeartbeatRequestShardingKeyWorkTokenAndCacheToken) obj; + return shardingKey() == other.shardingKey() + && workToken() == other.workToken() + && cacheToken() == other.cacheToken(); + } + + @Override + public final int hashCode() { + return Objects.hash(shardingKey(), workToken(), cacheToken()); + } + } +} From 4022f208be683dd44eca491fc6d6e786f88ca523 Mon Sep 17 00:00:00 2001 From: Martin Trieu Date: Mon, 10 Jun 2024 23:39:07 -0700 Subject: [PATCH 02/13] address PR comments --- .../DataflowStreamingPipelineOptions.java | 4 +- .../MetricTrackingWindmillServerStub.java | 185 ++++++--------- .../worker/StreamingDataflowWorker.java | 85 ++++--- .../worker/streaming/ActiveWorkState.java | 26 +-- .../worker/streaming/RefreshableWork.java | 24 ++ .../dataflow/worker/streaming/Work.java | 38 +-- .../windmill/WindmillServiceAddress.java | 2 + .../client/AbstractWindmillStream.java | 3 +- .../windmill/client/WindmillStream.java | 1 + ...ava => WindmillStreamClosedException.java} | 8 +- .../client/grpc/GrpcDirectGetWorkStream.java | 5 +- .../client/grpc/GrpcGetDataStream.java | 4 +- .../grpc/GrpcWindmillStreamFactory.java | 79 +------ .../client/grpc/StreamingEngineClient.java | 3 +- .../grpc/stubs/WindmillChannelFactory.java | 17 +- .../work/refresh/ActiveWorkRefresher.java | 46 +++- .../refresh/ApplianceHeartbeatSender.java | 60 +++++ .../work/refresh/DirectHeartbeatSender.java | 73 ++++++ .../DispatchedActiveWorkRefresher.java | 96 -------- .../refresh/FanOutActiveWorkRefresher.java | 136 ----------- .../work/refresh/HeartbeatRequests.java | 46 +--- .../work/refresh/HeartbeatSender.java | 32 +++ .../refresh/PoolBackedHeartbeatSender.java | 51 ++++ .../worker/StreamingDataflowWorkerTest.java | 8 +- .../StreamingModeExecutionContextTest.java | 1 + .../worker/WorkerCustomSourcesTest.java | 4 +- .../worker/streaming/ActiveWorkStateTest.java | 1 + .../streaming/ComputationStateCacheTest.java | 1 + .../StreamingApplianceWorkCommitterTest.java | 3 +- .../StreamingEngineWorkCommitterTest.java | 3 +- .../failures/WorkFailureProcessorTest.java | 1 + ...Test.java => ActiveWorkRefresherTest.java} | 72 +++--- .../FanOutActiveWorkRefresherTest.java | 221 ------------------ .../work/refresh/HeartbeatRequestsTest.java | 136 ++--------- 34 files changed, 558 insertions(+), 917 deletions(-) create mode 100644 runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/RefreshableWork.java rename runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/{WindmillStreamCancelledException.java => WindmillStreamClosedException.java} (83%) create mode 100644 runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/ApplianceHeartbeatSender.java create mode 100644 runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/DirectHeartbeatSender.java delete mode 100644 runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/DispatchedActiveWorkRefresher.java delete mode 100644 runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/FanOutActiveWorkRefresher.java create mode 100644 runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/HeartbeatSender.java create mode 100644 runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/PoolBackedHeartbeatSender.java rename runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/{DispatchedActiveWorkRefresherTest.java => ActiveWorkRefresherTest.java} (80%) delete mode 100644 runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/FanOutActiveWorkRefresherTest.java diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowStreamingPipelineOptions.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowStreamingPipelineOptions.java index 8fa93f3ca0e0e..07b9508a19be2 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowStreamingPipelineOptions.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowStreamingPipelineOptions.java @@ -224,9 +224,9 @@ public interface DataflowStreamingPipelineOptions extends PipelineOptions { @Description("Enables direct path mode for streaming engine.") @Default.InstanceFactory(EnableWindmillServiceDirectPathFactory.class) - boolean isEnableWindmillServiceDirectPath(); + boolean getIsWindmillServiceDirectPathEnabled(); - void setEnableWindmillServiceDirectPath(boolean value); + void setIsWindmillServiceDirectPathEnabled(boolean value); /** * Factory for creating local Windmill address. Reads from system propery 'windmill.hostport' for diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/MetricTrackingWindmillServerStub.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/MetricTrackingWindmillServerStub.java index a6297d019f8e9..fa244a6755d27 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/MetricTrackingWindmillServerStub.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/MetricTrackingWindmillServerStub.java @@ -18,6 +18,7 @@ package org.apache.beam.runners.dataflow.worker; import com.google.auto.value.AutoBuilder; +import com.google.common.collect.Iterables; import java.io.PrintWriter; import java.util.ArrayList; import java.util.HashMap; @@ -34,13 +35,13 @@ import org.apache.beam.runners.dataflow.worker.windmill.Windmill.HeartbeatRequest; import org.apache.beam.runners.dataflow.worker.windmill.WindmillServerStub; import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStream.GetDataStream; -import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStreamCancelledException; +import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStreamClosedException; import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStreamPool; +import org.apache.beam.runners.dataflow.worker.windmill.work.refresh.HeartbeatSender; import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.SettableFuture; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.ThreadFactoryBuilder; import org.checkerframework.checker.nullness.qual.Nullable; -import org.joda.time.Duration; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -57,84 +58,50 @@ }) public class MetricTrackingWindmillServerStub { private static final Logger LOG = LoggerFactory.getLogger(MetricTrackingWindmillServerStub.class); - private static final String FAN_OUT_REFRESH_WORK_EXECUTOR = "FanOutActiveWorkRefreshExecutor"; + private static final String FAN_OUT_REFRESH_WORK_EXECUTOR_NAME = + "FanOutActiveWorkRefreshExecutor"; private static final int MAX_READS_PER_BATCH = 60; private static final int MAX_ACTIVE_READS = 10; - private static final Duration STREAM_TIMEOUT = Duration.standardSeconds(30); private final AtomicInteger activeSideInputs = new AtomicInteger(); private final AtomicInteger activeStateReads = new AtomicInteger(); private final AtomicInteger activeHeartbeats = new AtomicInteger(); private final WindmillServerStub server; private final MemoryMonitor gcThrashingMonitor; private final boolean useStreamingRequests; - private final WindmillStreamPool getDataStreamPool; + private final @Nullable WindmillStreamPool getDataStreamPool; private final ExecutorService fanOutActiveWorkRefreshExecutor; - // This may be the same instance as getDataStreamPool based upon options. - private final WindmillStreamPool heartbeatStreamPool; - @GuardedBy("this") private final List pendingReadBatches; @GuardedBy("this") private int activeReadThreads = 0; - @Internal - @AutoBuilder(ofClass = MetricTrackingWindmillServerStub.class) - public abstract static class Builder { - - abstract Builder setServer(WindmillServerStub server); - - abstract Builder setGcThrashingMonitor(MemoryMonitor gcThrashingMonitor); - - abstract Builder setUseStreamingRequests(boolean useStreamingRequests); - - abstract Builder setUseSeparateHeartbeatStreams(boolean useSeparateHeartbeatStreams); - - abstract Builder setNumGetDataStreams(int numGetDataStreams); - - abstract MetricTrackingWindmillServerStub build(); - } - - public static Builder builder(WindmillServerStub server, MemoryMonitor gcThrashingMonitor) { - return new AutoBuilder_MetricTrackingWindmillServerStub_Builder() - .setServer(server) - .setGcThrashingMonitor(gcThrashingMonitor) - .setUseStreamingRequests(false) - .setUseSeparateHeartbeatStreams(false) - .setNumGetDataStreams(1); - } - MetricTrackingWindmillServerStub( WindmillServerStub server, MemoryMonitor gcThrashingMonitor, boolean useStreamingRequests, - boolean useSeparateHeartbeatStreams, - int numGetDataStreams) { + @Nullable WindmillStreamPool getDataStreamPool) { this.server = server; this.gcThrashingMonitor = gcThrashingMonitor; this.useStreamingRequests = useStreamingRequests; this.fanOutActiveWorkRefreshExecutor = Executors.newCachedThreadPool( - new ThreadFactoryBuilder().setNameFormat(FAN_OUT_REFRESH_WORK_EXECUTOR).build()); - if (useStreamingRequests) { - getDataStreamPool = - WindmillStreamPool.create( - Math.max(1, numGetDataStreams), STREAM_TIMEOUT, this.server::getDataStream); - if (useSeparateHeartbeatStreams) { - heartbeatStreamPool = - WindmillStreamPool.create(1, STREAM_TIMEOUT, this.server::getDataStream); - } else { - heartbeatStreamPool = getDataStreamPool; - } - } else { - getDataStreamPool = heartbeatStreamPool = null; - } + new ThreadFactoryBuilder().setNameFormat(FAN_OUT_REFRESH_WORK_EXECUTOR_NAME).build()); + this.getDataStreamPool = getDataStreamPool; // This is used as a queue but is expected to be less than 10 batches. this.pendingReadBatches = new ArrayList<>(); } + public static Builder builder(WindmillServerStub server, MemoryMonitor gcThrashingMonitor) { + return new AutoBuilder_MetricTrackingWindmillServerStub_Builder() + .setServer(server) + .setGcThrashingMonitor(gcThrashingMonitor) + .setUseStreamingRequests(false) + .setGetDataStreamPool(null); + } + // Adds the entry to a read batch for sending to the windmill server. If a non-null batch is // returned, this thread will be responsible for sending the batch and should wait for the batch // startRead to be notified. @@ -269,15 +236,15 @@ public Windmill.KeyedGetDataResponse getStateData( public Windmill.KeyedGetDataResponse getStateData( GetDataStream getDataStream, String computation, Windmill.KeyedGetDataRequest request) { gcThrashingMonitor.waitForResources("GetStateData"); - activeStateReads.getAndIncrement(); if (getDataStream.isClosed()) { throw new WorkItemCancelledException(request.getShardingKey()); } try { + activeStateReads.getAndIncrement(); return getDataStream.requestKeyedData(computation, request); } catch (Exception e) { - if (WindmillStreamCancelledException.isWindmillStreamCancelledException(e)) { + if (WindmillStreamClosedException.isWindmillStreamCancelledException(e)) { LOG.error("Tried to fetch keyed data from a closed stream. Work has been cancelled", e); throw new WorkItemCancelledException(request.getShardingKey()); } @@ -324,45 +291,8 @@ public Windmill.GlobalData getSideInputData( } } - /** Tells windmill processing is ongoing for the given keys. */ - public void refreshActiveWork(Map> heartbeats) { - if (heartbeats.isEmpty()) { - return; - } - activeHeartbeats.set(heartbeats.size()); - try { - if (useStreamingRequests) { - GetDataStream stream = heartbeatStreamPool.getStream(); - try { - stream.refreshActiveWork(heartbeats); - } finally { - heartbeatStreamPool.releaseStream(stream); - } - } else { - // This code path is only used by appliance which sends heartbeats (used to refresh active - // work) as KeyedGetDataRequests. So we must translate the HeartbeatRequest to a - // KeyedGetDataRequest here regardless of the value of sendKeyedGetDataRequests. - Windmill.GetDataRequest.Builder builder = Windmill.GetDataRequest.newBuilder(); - for (Map.Entry> entry : heartbeats.entrySet()) { - Windmill.ComputationGetDataRequest.Builder perComputationBuilder = - Windmill.ComputationGetDataRequest.newBuilder(); - perComputationBuilder.setComputationId(entry.getKey()); - for (HeartbeatRequest request : entry.getValue()) { - perComputationBuilder.addRequests( - Windmill.KeyedGetDataRequest.newBuilder() - .setShardingKey(request.getShardingKey()) - .setWorkToken(request.getWorkToken()) - .setCacheToken(request.getCacheToken()) - .addAllLatencyAttribution(request.getLatencyAttributionList()) - .build()); - } - builder.addRequests(perComputationBuilder.build()); - } - server.getData(builder.build()); - } - } finally { - activeHeartbeats.set(0); - } + public WindmillStreamPool getGetDataStreamPool() { + return getDataStreamPool; } /** @@ -370,46 +300,53 @@ public void refreshActiveWork(Map> heartbeats) { * * @implNote Skips closed {@link GetDataStream}(s). */ - public void refreshActiveWorkWithFanOut( - Map>> heartbeats) { + public void refreshActiveWork( + Map>> heartbeats) { if (heartbeats.isEmpty()) { return; } + try { - List> fanOutRefreshActiveWork = new ArrayList<>(); - for (Map.Entry>> heartbeat : - heartbeats.entrySet()) { - GetDataStream stream = heartbeat.getKey(); - Map> heartbeatRequests = heartbeat.getValue(); - if (stream.isClosed()) { - LOG.warn( - "Trying to refresh work on stream={} after work has moved off of worker." - + " heartbeats={}", - stream, - heartbeatRequests); - } else { - fanOutRefreshActiveWork.add(sendHeartbeatOnStreamFuture(heartbeat)); - } + // There is 1 destination to send heartbeat requests. + if (heartbeats.size() == 1) { + Map.Entry>> heartbeat = + Iterables.getOnlyElement(heartbeats.entrySet()); + HeartbeatSender sender = heartbeat.getKey(); + sender.sendHeartbeats(heartbeat.getValue()); } - // Don't block until we kick off all the refresh active work RPCs. - @SuppressWarnings("rawtypes") - CompletableFuture parallelFanOutRefreshActiveWork = - CompletableFuture.allOf(fanOutRefreshActiveWork.toArray(new CompletableFuture[0])); - parallelFanOutRefreshActiveWork.join(); + // There are multiple destinations to send heartbeat requests. Fan out requests in parallel. + else { + refreshActiveWorkWithFanOut(heartbeats); + } } finally { activeHeartbeats.set(0); } } + private void refreshActiveWorkWithFanOut( + Map>> heartbeats) { + List> fanOutRefreshActiveWork = new ArrayList<>(); + for (Map.Entry>> heartbeat : + heartbeats.entrySet()) { + fanOutRefreshActiveWork.add(sendHeartbeatOnStreamFuture(heartbeat)); + } + + // Don't block until we kick off all the refresh active work RPCs. + @SuppressWarnings("rawtypes") + CompletableFuture parallelFanOutRefreshActiveWork = + CompletableFuture.allOf(fanOutRefreshActiveWork.toArray(new CompletableFuture[0])); + parallelFanOutRefreshActiveWork.join(); + } + private CompletableFuture sendHeartbeatOnStreamFuture( - Map.Entry>> heartbeat) { + Map.Entry>> heartbeat) { return CompletableFuture.runAsync( () -> { - GetDataStream stream = heartbeat.getKey(); - Map> heartbeatRequests = heartbeat.getValue(); activeHeartbeats.getAndUpdate(existing -> existing + heartbeat.getValue().size()); - stream.refreshActiveWork(heartbeatRequests); + HeartbeatSender sender = heartbeat.getKey(); + Map> heartbeatRequests = heartbeat.getValue(); + sender.sendHeartbeats(heartbeatRequests); // Active heartbeats should never drop below 0. activeHeartbeats.getAndUpdate( existing -> Math.max(existing - heartbeat.getValue().size(), 0)); @@ -430,6 +367,22 @@ public void printHtml(PrintWriter writer) { writer.println("Heartbeat Keys Active: " + activeHeartbeats.get()); } + @Internal + @AutoBuilder(ofClass = MetricTrackingWindmillServerStub.class) + public abstract static class Builder { + + abstract Builder setServer(WindmillServerStub server); + + abstract Builder setGcThrashingMonitor(MemoryMonitor gcThrashingMonitor); + + abstract Builder setGetDataStreamPool( + @Nullable WindmillStreamPool getDataStreamPool); + + abstract Builder setUseStreamingRequests(boolean useStreamingRequests); + + abstract MetricTrackingWindmillServerStub build(); + } + private static final class ReadBatch { ArrayList reads = new ArrayList<>(); SettableFuture startRead = SettableFuture.create(); 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 4f41f85921cdc..b9124753d0209 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 @@ -67,6 +67,7 @@ import org.apache.beam.runners.dataflow.worker.windmill.WindmillServiceAddress; import org.apache.beam.runners.dataflow.worker.windmill.appliance.JniWindmillApplianceServer; import org.apache.beam.runners.dataflow.worker.windmill.client.CloseableStream; +import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStream; import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStream.GetWorkStream; import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStreamPool; import org.apache.beam.runners.dataflow.worker.windmill.client.commits.CompleteCommit; @@ -91,8 +92,9 @@ import org.apache.beam.runners.dataflow.worker.windmill.work.processing.failures.StreamingEngineFailureTracker; import org.apache.beam.runners.dataflow.worker.windmill.work.processing.failures.WorkFailureProcessor; import org.apache.beam.runners.dataflow.worker.windmill.work.refresh.ActiveWorkRefresher; -import org.apache.beam.runners.dataflow.worker.windmill.work.refresh.DispatchedActiveWorkRefresher; -import org.apache.beam.runners.dataflow.worker.windmill.work.refresh.FanOutActiveWorkRefresher; +import org.apache.beam.runners.dataflow.worker.windmill.work.refresh.ApplianceHeartbeatSender; +import org.apache.beam.runners.dataflow.worker.windmill.work.refresh.HeartbeatSender; +import org.apache.beam.runners.dataflow.worker.windmill.work.refresh.PoolBackedHeartbeatSender; import org.apache.beam.sdk.fn.IdGenerator; import org.apache.beam.sdk.fn.IdGenerators; import org.apache.beam.sdk.fn.JvmInitializers; @@ -145,6 +147,8 @@ public class StreamingDataflowWorker { /** Maximum number of failure stacktraces to report in each update sent to backend. */ private static final int MAX_FAILURES_TO_REPORT_IN_UPDATE = 1000; + private static final Duration GET_DATA_STREAM_TIMEOUT = Duration.standardSeconds(30); + private static final Random clientIdGenerator = new Random(); private static final String CHANNELZ_PATH = "/channelz"; final WindmillStateCache stateCache; @@ -167,6 +171,7 @@ public class StreamingDataflowWorker { private final StreamingWorkerStatusReporter workerStatusReporter; private final StreamingCounters streamingCounters; private final StreamingWorkScheduler streamingWorkScheduler; + private final HeartbeatSender heartbeatSender; private @Nullable StreamingEngineClient streamingEngineClient; private StreamingDataflowWorker( @@ -240,11 +245,17 @@ private StreamingDataflowWorker( dispatchThread.setName("DispatchThread"); this.clientId = clientId; this.windmillServer = windmillServer; + + WindmillStreamPool getDataStreamPool = + WindmillStreamPool.create( + Math.max(1, options.getWindmillGetDataStreamCount()), + GET_DATA_STREAM_TIMEOUT, + windmillServer::getDataStream); + this.metricTrackingWindmillServer = MetricTrackingWindmillServerStub.builder(windmillServer, memoryMonitor) .setUseStreamingRequests(windmillServiceEnabled) - .setUseSeparateHeartbeatStreams(options.getUseSeparateWindmillHeartbeatStreams()) - .setNumGetDataStreams(options.getWindmillGetDataStreamCount()) + .setGetDataStreamPool(getDataStreamPool) .build(); // Register standard file systems. @@ -328,27 +339,29 @@ private StreamingDataflowWorker( (computationId, request) -> metricTrackingWindmillServer.getStateData( getDataStream, computationId, request)); - this.activeWorkRefresher = - FanOutActiveWorkRefresher.create( - clock, - options.getActiveWorkRefreshPeriodMillis(), - stuckCommitDurationMillis, - computationStateCache::getAllPresentComputations, - sampler, - metricTrackingWindmillServer::refreshActiveWorkWithFanOut); } else { - this.activeWorkRefresher = - DispatchedActiveWorkRefresher.create( - clock, - options.getActiveWorkRefreshPeriodMillis(), - stuckCommitDurationMillis, - computationStateCache::getAllPresentComputations, - sampler, - metricTrackingWindmillServer::refreshActiveWork, - executorSupplier.apply("RefreshWork")); this.streamingEngineClient = null; } + this.heartbeatSender = + options.isEnableStreamingEngine() + ? new PoolBackedHeartbeatSender( + options.getUseSeparateWindmillHeartbeatStreams() + ? WindmillStreamPool.create( + 1, GET_DATA_STREAM_TIMEOUT, windmillServer::getDataStream) + : getDataStreamPool) + : new ApplianceHeartbeatSender(windmillServer::getData); + + this.activeWorkRefresher = + new ActiveWorkRefresher( + clock, + options.getActiveWorkRefreshPeriodMillis(), + stuckCommitDurationMillis, + computationStateCache::getAllPresentComputations, + sampler, + executorSupplier.apply("RefreshWork"), + metricTrackingWindmillServer::refreshActiveWork); + StreamingWorkerStatusPages.Builder statusPagesBuilder = StreamingWorkerStatusPages.builder() .setClock(clock) @@ -398,7 +411,7 @@ private StreamingDataflowWorker( LOG.debug( "windmillServiceEnabled: {}; isDirectPathEnabled: {}", windmillServiceEnabled, - options.isEnableWindmillServiceDirectPath()); + options.getIsWindmillServiceDirectPathEnabled()); LOG.debug("WindmillServiceEndpoint: {}", options.getWindmillServiceEndpoint()); LOG.debug("WindmillServicePort: {}", options.getWindmillServicePort()); LOG.debug("LocalWindmillHostport: {}", options.getLocalWindmillHostport()); @@ -713,13 +726,8 @@ public static void main(String[] args) throws Exception { } JvmInitializers.runBeforeProcessing(options); - try { - worker.startStatusPages(); - worker.start(); - } catch (Throwable e) { - LOG.error("Harness shutting down due to uncaught exception.", e); - worker.stop(); - } + worker.startStatusPages(); + worker.start(); } private static void validateWorkerOptions(DataflowWorkerHarnessOptions options) { @@ -735,14 +743,19 @@ private static void validateWorkerOptions(DataflowWorkerHarnessOptions options) } private static boolean isDirectPathPipeline(DataflowWorkerHarnessOptions options) { - boolean isIpV6Enabled = options.getDataflowServiceOptions().contains(ENABLE_IPV6_EXPERIMENT); - if (options.isEnableWindmillServiceDirectPath() && !isIpV6Enabled) { + boolean isIpV6Enabled = + Optional.ofNullable(options.getDataflowServiceOptions()) + .map(serviceOptions -> serviceOptions.contains(ENABLE_IPV6_EXPERIMENT)) + .orElse(false); + if (options.isEnableStreamingEngine() + && options.getIsWindmillServiceDirectPathEnabled() + && !isIpV6Enabled) { LOG.warn( "DirectPath is currently only supported with IPv6 networking stack. Defaulting to CloudPath."); } return options.isEnableStreamingEngine() - && options.isEnableWindmillServiceDirectPath() + && options.getIsWindmillServiceDirectPathEnabled() && isIpV6Enabled; } @@ -923,7 +936,10 @@ private void dispatchLoop() { workItem, watermarks.setOutputDataWatermark(workItem.getOutputDataWatermark()).build(), Work.createProcessingContext( - computationId, metricTrackingWindmillServer::getStateData, workCommitter::commit), + computationId, + metricTrackingWindmillServer::getStateData, + workCommitter::commit, + heartbeatSender), /* getWorkStreamLatencies= */ Collections.emptyList()); } } @@ -960,7 +976,8 @@ void streamingDispatchLoop() { Work.createProcessingContext( computationState.getComputationId(), metricTrackingWindmillServer::getStateData, - workCommitter::commit), + workCommitter::commit, + heartbeatSender), getWorkStreamLatencies); })); try { diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/ActiveWorkState.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/ActiveWorkState.java index 7a90b2c76f5b9..6cc55b83b04f2 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/ActiveWorkState.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/ActiveWorkState.java @@ -18,6 +18,7 @@ package org.apache.beam.runners.dataflow.worker.streaming; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList.toImmutableList; +import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableListMultimap.flatteningToImmutableListMultimap; import java.io.PrintWriter; import java.util.ArrayDeque; @@ -31,7 +32,6 @@ import java.util.Queue; import java.util.concurrent.atomic.AtomicReference; import java.util.function.BiConsumer; -import java.util.stream.Collectors; import java.util.stream.Stream; import javax.annotation.Nullable; import javax.annotation.concurrent.GuardedBy; @@ -333,20 +333,18 @@ synchronized ImmutableList getKeyHeartbeats( .collect(toImmutableList()); } + /** + * Returns a read only view of current active work. + * + * @implNote Do not return a reference to the underlying workQueue as iterations over it will + * cause a {@link java.util.ConcurrentModificationException} as it is not a thread-safe data + * structure. + */ synchronized ImmutableListMultimap getReadOnlyActiveWork() { - // Do not return a reference to the underlying workQueue as iterations over it will cause a - // ConcurrentModificationException as it is not a thread-safe data structure. - ImmutableListMultimap.Builder readOnlyActiveWork = - ImmutableListMultimap.builder(); - for (Entry> keyedWorkQueues : activeWork.entrySet()) { - readOnlyActiveWork.putAll( - keyedWorkQueues.getKey(), - keyedWorkQueues.getValue().stream() - .map(ExecutableWork::work) - .collect(Collectors.toList())); - } - - return readOnlyActiveWork.build(); + return activeWork.entrySet().stream() + .collect( + flatteningToImmutableListMultimap( + Entry::getKey, e -> e.getValue().stream().map(ExecutableWork::work))); } /** diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/RefreshableWork.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/RefreshableWork.java new file mode 100644 index 0000000000000..91c4ba9c82fda --- /dev/null +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/RefreshableWork.java @@ -0,0 +1,24 @@ +/* + * 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; + +import org.apache.beam.runners.dataflow.worker.windmill.Windmill; + +public interface RefreshableWork { + Windmill.WorkItem getWorkItem(); +} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/Work.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/Work.java index 6182f295d4b66..f16d6dd2c65fa 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/Work.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/Work.java @@ -31,7 +31,6 @@ import java.util.function.Consumer; import java.util.function.Function; import java.util.function.Supplier; -import javax.annotation.Nullable; import javax.annotation.concurrent.NotThreadSafe; import org.apache.beam.repackaged.core.org.apache.commons.lang3.tuple.Pair; import org.apache.beam.runners.dataflow.worker.ActiveMessageMetadata; @@ -44,10 +43,10 @@ import org.apache.beam.runners.dataflow.worker.windmill.Windmill.LatencyAttribution.ActiveLatencyBreakdown.Distribution; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.WorkItem; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.WorkItemCommitRequest; -import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStream.GetDataStream; import org.apache.beam.runners.dataflow.worker.windmill.client.commits.Commit; import org.apache.beam.runners.dataflow.worker.windmill.client.commits.WorkCommitter; import org.apache.beam.runners.dataflow.worker.windmill.state.WindmillStateReader; +import org.apache.beam.runners.dataflow.worker.windmill.work.refresh.HeartbeatSender; import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.joda.time.Duration; @@ -106,18 +105,11 @@ public static Work create( } public static ProcessingContext createProcessingContext( - String computationId, - BiFunction getKeyedDataFn, - Consumer workCommitter) { - return ProcessingContext.create(computationId, getKeyedDataFn, workCommitter, null); - } - - public static ProcessingContext createFanOutProcessingContext( String computationId, BiFunction getKeyedDataFn, Consumer workCommitter, - GetDataStream getDataStream) { - return ProcessingContext.create(computationId, getKeyedDataFn, workCommitter, getDataStream); + HeartbeatSender heartbeatSender) { + return ProcessingContext.create(computationId, getKeyedDataFn, workCommitter, heartbeatSender); } private static LatencyAttribution.Builder createLatencyAttributionWithActiveLatencyBreakdown( @@ -181,10 +173,6 @@ public State getState() { return currentState.state(); } - public @Nullable GetDataStream getDataStream() { - return processingContext.getDataStream(); - } - public void setState(State state) { Instant now = clock.get(); totalDurationPerState.compute( @@ -194,6 +182,20 @@ public void setState(State state) { this.currentState = TimedState.create(state, now); } + public boolean isRefreshable(Instant refreshDeadline) { + boolean isRefreshable = getStartTime().isBefore(refreshDeadline); + if (heartbeatSender().isInvalid()) { + setFailed(); + return false; + } + + return isRefreshable; + } + + public HeartbeatSender heartbeatSender() { + return processingContext.heartbeatSender(); + } + public void setFailed() { this.isFailed = true; } @@ -330,12 +332,12 @@ private static ProcessingContext create( String computationId, BiFunction getKeyedDataFn, Consumer workCommitter, - @Nullable GetDataStream getDataStream) { + HeartbeatSender heartbeatSender) { return new AutoValue_Work_ProcessingContext( computationId, request -> Optional.ofNullable(getKeyedDataFn.apply(computationId, request)), workCommitter, - getDataStream); + heartbeatSender); } /** Computation that the {@link Work} belongs to. */ @@ -351,6 +353,6 @@ private static ProcessingContext create( */ public abstract Consumer workCommitter(); - public abstract @Nullable GetDataStream getDataStream(); + public abstract HeartbeatSender heartbeatSender(); } } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/WindmillServiceAddress.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/WindmillServiceAddress.java index 3c9226be34a5d..cbd40673357e9 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/WindmillServiceAddress.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/WindmillServiceAddress.java @@ -19,9 +19,11 @@ import com.google.auto.value.AutoOneOf; import com.google.auto.value.AutoValue; +import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.net.HostAndPort; /** Used to create channels to communicate with Streaming Engine via gRpc. */ +@Internal @AutoOneOf(WindmillServiceAddress.Kind.class) public abstract class WindmillServiceAddress { diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/AbstractWindmillStream.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/AbstractWindmillStream.java index f958cbaa94ce0..bd5827b44191a 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/AbstractWindmillStream.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/AbstractWindmillStream.java @@ -176,7 +176,6 @@ protected final void startStream() { onNewStream(); if (clientClosed.get()) { close(); - streamRegistry.remove(this); } return; } @@ -257,7 +256,7 @@ public final Instant startTime() { } @Override - public boolean isClosed() { + public final boolean isClosed() { return streamClosed.get() || clientClosed.get(); } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/WindmillStream.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/WindmillStream.java index 1355837acc0dd..9be7f56d20b46 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/WindmillStream.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/WindmillStream.java @@ -41,6 +41,7 @@ public interface WindmillStream { /** Returns when the stream was opened. */ Instant startTime(); + /** Reflects that {@link #close()} was explicitly called. */ boolean isClosed(); /** Handle representing a stream of GetWork responses. */ diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/WindmillStreamCancelledException.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/WindmillStreamClosedException.java similarity index 83% rename from runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/WindmillStreamCancelledException.java rename to runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/WindmillStreamClosedException.java index 194206b21b856..bcdf6521bda2e 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/WindmillStreamCancelledException.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/WindmillStreamClosedException.java @@ -20,16 +20,16 @@ import org.apache.beam.sdk.annotations.Internal; @Internal -public final class WindmillStreamCancelledException extends RuntimeException { +public final class WindmillStreamClosedException extends RuntimeException { - public WindmillStreamCancelledException(String message) { + public WindmillStreamClosedException(String message) { super(message); } - /** Returns whether an exception was caused by a {@link WindmillStreamCancelledException}. */ + /** Returns whether an exception was caused by a {@link WindmillStreamClosedException}. */ public static boolean isWindmillStreamCancelledException(Throwable t) { while (t != null) { - if (t instanceof WindmillStreamCancelledException) { + if (t instanceof WindmillStreamClosedException) { return true; } t = t.getCause(); diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcDirectGetWorkStream.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcDirectGetWorkStream.java index f4477d6db4c9a..8f6caf9f9574a 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcDirectGetWorkStream.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcDirectGetWorkStream.java @@ -46,6 +46,7 @@ import org.apache.beam.runners.dataflow.worker.windmill.client.throttling.ThrottleTimer; import org.apache.beam.runners.dataflow.worker.windmill.work.WorkItemScheduler; import org.apache.beam.runners.dataflow.worker.windmill.work.budget.GetWorkBudget; +import org.apache.beam.runners.dataflow.worker.windmill.work.refresh.DirectHeartbeatSender; import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.sdk.util.BackOff; import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; @@ -340,12 +341,12 @@ private void runAndReset() { } private Work.ProcessingContext createProcessingContext(String computationId) { - return Work.createFanOutProcessingContext( + return Work.createProcessingContext( computationId, (computation, request) -> keyedGetDataFn.apply(getDataStream.get()).apply(computation, request), workCommitter.get()::commit, - getDataStream.get()); + new DirectHeartbeatSender(getDataStream.get())); } } } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetDataStream.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetDataStream.java index 08e1e790ed237..7fa194d13b93d 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetDataStream.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetDataStream.java @@ -49,7 +49,7 @@ import org.apache.beam.runners.dataflow.worker.windmill.Windmill.StreamingGetDataResponse; import org.apache.beam.runners.dataflow.worker.windmill.client.AbstractWindmillStream; import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStream.GetDataStream; -import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStreamCancelledException; +import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStreamClosedException; import org.apache.beam.runners.dataflow.worker.windmill.client.grpc.GrpcGetDataStreamRequests.QueuedBatch; import org.apache.beam.runners.dataflow.worker.windmill.client.grpc.GrpcGetDataStreamRequests.QueuedRequest; import org.apache.beam.runners.dataflow.worker.windmill.client.grpc.observers.StreamObserverFactory; @@ -401,7 +401,7 @@ private void queueRequestAndWait(QueuedRequest request) throws InterruptedExcept long waitFor = 10; while (!batchSent) { if (isClosed()) { - throw new WindmillStreamCancelledException( + throw new WindmillStreamClosedException( "Requests failed for batch containing " + createStreamCancelledErrorMessage(batch) + " requests. This is most likely due to the stream being explicitly closed" diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcWindmillStreamFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcWindmillStreamFactory.java index 5a7cc1dd10873..7c7122d7c37d8 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcWindmillStreamFactory.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcWindmillStreamFactory.java @@ -55,7 +55,6 @@ import org.apache.beam.sdk.util.BackOff; import org.apache.beam.sdk.util.FluentBackoff; import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.AbstractStub; -import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.StreamObserver; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Suppliers; import org.joda.time.Duration; import org.joda.time.Instant; @@ -112,7 +111,9 @@ private GrpcWindmillStreamFactory( this.streamIdGenerator = new AtomicLong(); } - /** @implNote Used for {@link AutoBuilder} {@link Builder} class, do not call directly. */ + /** + * @implNote Used for {@link AutoBuilder} {@link Builder} class, do not call directly. + */ static GrpcWindmillStreamFactory create( JobHeader jobHeader, int logEveryNStreamFailures, @@ -186,7 +187,7 @@ public GetWorkStream createGetWorkStream( responseObserver -> withDefaultDeadline(stub).getWorkStream(responseObserver), request, grpcBackOff.get(), - newBufferringStreamObserverFactory(), + newStreamObserverFactory(), streamRegistry, logEveryNStreamFailures, getWorkThrottleTimer, @@ -208,7 +209,7 @@ public GetWorkStream createDirectGetWorkStream( stub::getWorkStream, request, grpcBackOff.get(), - newSimpleStreamObserverFactory(), + newStreamObserverFactory(), streamRegistry, logEveryNStreamFailures, getWorkThrottleTimer, @@ -218,25 +219,12 @@ public GetWorkStream createDirectGetWorkStream( workItemScheduler); } - public GetDataStream createDirectGetDataStream( - CloudWindmillServiceV1Alpha1Stub stub, - ThrottleTimer getDataThrottleTimer, - boolean sendKeyedGetDataRequests, - Consumer> processHeartbeatResponses) { - return createGetDataStream( - () -> stub, - getDataThrottleTimer, - sendKeyedGetDataRequests, - processHeartbeatResponses, - newSimpleStreamObserverFactory()); - } - public GetDataStream createGetDataStream( CloudWindmillServiceV1Alpha1Stub stub, ThrottleTimer getDataThrottleTimer) { return GrpcGetDataStream.create( - responseObserver -> stub.get().getDataStream(responseObserver), + stub::getDataStream, grpcBackOff.get(), - streamObserverFactory, + newStreamObserverFactory(), streamRegistry, logEveryNStreamFailures, getDataThrottleTimer, @@ -250,15 +238,12 @@ public GetDataStream createGetDataStream( public CommitWorkStream createCommitWorkStream( CloudWindmillServiceV1Alpha1Stub stub, ThrottleTimer commitWorkThrottleTimer) { return createCommitWorkStream( - () -> withDefaultDeadline(stub), - commitWorkThrottleTimer, - newBufferringStreamObserverFactory()); + () -> withDefaultDeadline(stub), commitWorkThrottleTimer, newStreamObserverFactory()); } public CommitWorkStream createDirectCommitWorkStream( CloudWindmillServiceV1Alpha1Stub stub, ThrottleTimer commitWorkThrottleTimer) { - return createCommitWorkStream( - () -> stub, commitWorkThrottleTimer, newSimpleStreamObserverFactory()); + return createCommitWorkStream(() -> stub, commitWorkThrottleTimer, newStreamObserverFactory()); } private CommitWorkStream createCommitWorkStream( @@ -284,7 +269,7 @@ public GetWorkerMetadataStream createGetWorkerMetadataStream( return GrpcGetWorkerMetadataStream.create( responseObserver -> withDefaultDeadline(stub).getWorkerMetadata(responseObserver), grpcBackOff.get(), - newBufferringStreamObserverFactory(), + newStreamObserverFactory(), streamRegistry, logEveryNStreamFailures, jobHeader, @@ -293,53 +278,11 @@ public GetWorkerMetadataStream createGetWorkerMetadataStream( onNewWindmillEndpoints); } - private StreamObserverFactory newBufferringStreamObserverFactory() { + private StreamObserverFactory newStreamObserverFactory() { return StreamObserverFactory.direct( DEFAULT_STREAM_RPC_DEADLINE_SECONDS * 2, windmillMessagesBetweenIsReadyChecks); } - /** - * Simple {@link StreamObserverFactory} that does not buffer or provide extra functionality for - * request observers. - * - * @implNote Used to create stream observers for direct path streams that do not share any - * underlying resources between threads. - */ - private StreamObserverFactory newSimpleStreamObserverFactory() { - return new StreamObserverFactory() { - @Override - public StreamObserver from( - Function, StreamObserver> clientFactory, - StreamObserver responseObserver) { - return clientFactory.apply(responseObserver); - } - }; - } - - /** - * Schedules streaming RPC health checks to run on a background daemon thread, which will be - * cleaned up when the JVM shutdown. - */ - public void scheduleHealthChecks(int healthCheckInterval) { - if (healthCheckInterval < 0) { - return; - } - - new Timer("WindmillHealthCheckTimer") - .schedule( - new TimerTask() { - @Override - public void run() { - Instant reportThreshold = Instant.now().minus(Duration.millis(healthCheckInterval)); - for (AbstractWindmillStream stream : streamRegistry) { - stream.maybeSendHealthCheck(reportThreshold); - } - } - }, - 0, - healthCheckInterval); - } - @Override public void appendSummaryHtml(PrintWriter writer) { writer.write("Active Streams:
"); diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/StreamingEngineClient.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/StreamingEngineClient.java index d1a2d15379f5e..1aed95d74a467 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/StreamingEngineClient.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/StreamingEngineClient.java @@ -314,8 +314,6 @@ private synchronized void consumeWindmillWorkerEndpoints(WindmillEndpoints newWi createNewGlobalDataStreams(newWindmillEndpoints.globalDataEndpoints())) .build(); - closeStaleStreams.join(); - LOG.info( "Setting new connections: {}. Previous connections: {}.", newConnectionsState, @@ -323,6 +321,7 @@ private synchronized void consumeWindmillWorkerEndpoints(WindmillEndpoints newWi connections.set(newConnectionsState); isBudgetRefreshPaused.set(false); getWorkBudgetRefresher.requestBudgetRefresh(); + closeStaleStreams.join(); } /** Close the streams that are no longer valid in parallel. */ diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/stubs/WindmillChannelFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/stubs/WindmillChannelFactory.java index be565ece0ba7c..a5f327550cc1f 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/stubs/WindmillChannelFactory.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/stubs/WindmillChannelFactory.java @@ -58,7 +58,9 @@ public static ManagedChannel remoteChannel( return remoteChannel( windmillServiceAddress.gcpServiceAddress(), windmillServiceRpcChannelTimeoutSec); case AUTHENTICATED_GCP_SERVICE_ADDRESS: - return remoteDirectChannel(windmillServiceAddress.authenticatedGcpServiceAddress()); + return remoteDirectChannel( + windmillServiceAddress.authenticatedGcpServiceAddress(), + windmillServiceRpcChannelTimeoutSec); // switch is exhaustive will never happen. default: throw new UnsupportedOperationException( @@ -67,11 +69,14 @@ public static ManagedChannel remoteChannel( } static ManagedChannel remoteDirectChannel( - AuthenticatedGcpServiceAddress authenticatedGcpServiceAddress) { - return AltsChannelBuilder.forAddress( - authenticatedGcpServiceAddress.gcpServiceAddress().getHost(), - authenticatedGcpServiceAddress.gcpServiceAddress().getPort()) - .overrideAuthority(authenticatedGcpServiceAddress.authenticatingService()) + AuthenticatedGcpServiceAddress authenticatedGcpServiceAddress, + int windmillServiceRpcChannelTimeoutSec) { + return withDefaultChannelOptions( + AltsChannelBuilder.forAddress( + authenticatedGcpServiceAddress.gcpServiceAddress().getHost(), + authenticatedGcpServiceAddress.gcpServiceAddress().getPort()) + .overrideAuthority(authenticatedGcpServiceAddress.authenticatingService()), + windmillServiceRpcChannelTimeoutSec) .build(); } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/ActiveWorkRefresher.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/ActiveWorkRefresher.java index 96a6feec1da0d..9e235fa4e75c3 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/ActiveWorkRefresher.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/ActiveWorkRefresher.java @@ -17,13 +17,20 @@ */ package org.apache.beam.runners.dataflow.worker.windmill.work.refresh; +import java.util.ArrayList; import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; +import java.util.function.Consumer; import java.util.function.Supplier; import javax.annotation.concurrent.ThreadSafe; import org.apache.beam.runners.dataflow.worker.DataflowExecutionStateSampler; import org.apache.beam.runners.dataflow.worker.streaming.ComputationState; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableListMultimap; import org.joda.time.Duration; import org.joda.time.Instant; import org.slf4j.Logger; @@ -37,7 +44,7 @@ * threshold is determined by {@link #activeWorkRefreshPeriodMillis} */ @ThreadSafe -public abstract class ActiveWorkRefresher { +public class ActiveWorkRefresher { private static final Logger LOG = LoggerFactory.getLogger(ActiveWorkRefresher.class); protected final Supplier clock; @@ -46,20 +53,25 @@ public abstract class ActiveWorkRefresher { protected final DataflowExecutionStateSampler sampler; private final int stuckCommitDurationMillis; private final ScheduledExecutorService activeWorkRefreshExecutor; + private final Consumer>>> + heartbeatSender; - protected ActiveWorkRefresher( + public ActiveWorkRefresher( Supplier clock, int activeWorkRefreshPeriodMillis, int stuckCommitDurationMillis, Supplier> computations, DataflowExecutionStateSampler sampler, - ScheduledExecutorService activeWorkRefreshExecutor) { + ScheduledExecutorService activeWorkRefreshExecutor, + Consumer>>> + heartbeatSender) { this.clock = clock; this.activeWorkRefreshPeriodMillis = activeWorkRefreshPeriodMillis; this.stuckCommitDurationMillis = stuckCommitDurationMillis; this.computations = computations; this.sampler = sampler; this.activeWorkRefreshExecutor = activeWorkRefreshExecutor; + this.heartbeatSender = heartbeatSender; } @SuppressWarnings("FutureReturnValueIgnored") @@ -103,5 +115,31 @@ private void invalidateStuckCommits() { } } - protected abstract void refreshActiveWork(); + private void refreshActiveWork() { + Instant refreshDeadline = clock.get().minus(Duration.millis(activeWorkRefreshPeriodMillis)); + + Map>> fannedOutHeartbeatRequests = + new HashMap<>(); + for (ComputationState computationState : computations.get()) { + String computationId = computationState.getComputationId(); + + // Get heartbeat requests for computation's current active work, aggregated by GetDataStream + // to correctly fan-out the heartbeat requests. + ImmutableListMultimap heartbeats = + HeartbeatRequests.getRefreshableKeyHeartbeats( + computationState.currentActiveWorkReadOnly(), refreshDeadline, sampler); + // Aggregate the heartbeats across computations by GetDataStream for correct fan out. + for (Map.Entry> heartbeatsPerStream : + heartbeats.asMap().entrySet()) { + Map> existingHeartbeats = + fannedOutHeartbeatRequests.computeIfAbsent( + heartbeatsPerStream.getKey(), ignored -> new HashMap<>()); + List existingHeartbeatsForComputation = + existingHeartbeats.computeIfAbsent(computationId, ignored -> new ArrayList<>()); + existingHeartbeatsForComputation.addAll(heartbeatsPerStream.getValue()); + } + } + + heartbeatSender.accept(fannedOutHeartbeatRequests); + } } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/ApplianceHeartbeatSender.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/ApplianceHeartbeatSender.java new file mode 100644 index 0000000000000..732887489bc79 --- /dev/null +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/ApplianceHeartbeatSender.java @@ -0,0 +1,60 @@ +/* + * 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.windmill.work.refresh; + +import java.util.List; +import java.util.Map; +import java.util.function.Consumer; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill; +import org.apache.beam.sdk.annotations.Internal; + +/** Streaming appliance implementation of {@link HeartbeatSender}. */ +@Internal +public final class ApplianceHeartbeatSender implements HeartbeatSender { + private final Consumer sendHeartbeatFn; + + public ApplianceHeartbeatSender(Consumer sendHeartbeatFn) { + this.sendHeartbeatFn = sendHeartbeatFn; + } + + @Override + public void sendHeartbeats(Map> heartbeats) { + // This code path is only used by appliance which sends heartbeats (used to refresh active + // work) as KeyedGetDataRequests. So we must translate the HeartbeatRequest to a + // KeyedGetDataRequest here regardless of the value of sendKeyedGetDataRequests. + Windmill.GetDataRequest.Builder builder = Windmill.GetDataRequest.newBuilder(); + + for (Map.Entry> entry : heartbeats.entrySet()) { + Windmill.ComputationGetDataRequest.Builder perComputationBuilder = + Windmill.ComputationGetDataRequest.newBuilder(); + perComputationBuilder.setComputationId(entry.getKey()); + for (Windmill.HeartbeatRequest request : entry.getValue()) { + perComputationBuilder.addRequests( + Windmill.KeyedGetDataRequest.newBuilder() + .setShardingKey(request.getShardingKey()) + .setWorkToken(request.getWorkToken()) + .setCacheToken(request.getCacheToken()) + .addAllLatencyAttribution(request.getLatencyAttributionList()) + .build()); + } + builder.addRequests(perComputationBuilder.build()); + } + + sendHeartbeatFn.accept(builder.build()); + } +} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/DirectHeartbeatSender.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/DirectHeartbeatSender.java new file mode 100644 index 0000000000000..8cfe5c8d73f05 --- /dev/null +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/DirectHeartbeatSender.java @@ -0,0 +1,73 @@ +/* + * 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.windmill.work.refresh; + +import java.util.List; +import java.util.Map; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill; +import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStream; +import org.apache.beam.sdk.annotations.Internal; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * {@link HeartbeatSender} implementation that sends heartbeats directly on the underlying stream. + * If the stream is closed, does nothing. + * + * @implNote {@link #equals(Object)} and {@link #hashCode()} implementations delegate to internal + * {@link org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStream.GetDataStream} + * implementations so that requests can be grouped and sent on the same stream. + */ +@Internal +public final class DirectHeartbeatSender implements HeartbeatSender { + private static final Logger LOG = LoggerFactory.getLogger(DirectHeartbeatSender.class); + private final WindmillStream.GetDataStream getDataStream; + + public DirectHeartbeatSender(WindmillStream.GetDataStream getDataStream) { + this.getDataStream = getDataStream; + } + + @Override + public void sendHeartbeats(Map> heartbeats) { + if (isInvalid()) { + LOG.warn( + "Trying to refresh work on stream={} after work has moved off of worker." + + " heartbeats={}", + getDataStream, + heartbeats); + } else { + getDataStream.refreshActiveWork(heartbeats); + } + } + + @Override + public synchronized boolean isInvalid() { + return getDataStream.isClosed(); + } + + @Override + public int hashCode() { + return getDataStream.hashCode(); + } + + @Override + public boolean equals(Object obj) { + return obj instanceof DirectHeartbeatSender + && getDataStream.equals(((DirectHeartbeatSender) obj).getDataStream); + } +} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/DispatchedActiveWorkRefresher.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/DispatchedActiveWorkRefresher.java deleted file mode 100644 index 5a9d214a7ecab..0000000000000 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/DispatchedActiveWorkRefresher.java +++ /dev/null @@ -1,96 +0,0 @@ -/* - * 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.windmill.work.refresh; - -import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap.toImmutableMap; - -import java.util.Collection; -import java.util.List; -import java.util.Map; -import java.util.concurrent.ScheduledExecutorService; -import java.util.function.Consumer; -import java.util.function.Supplier; -import javax.annotation.concurrent.ThreadSafe; -import org.apache.beam.runners.dataflow.worker.DataflowExecutionStateSampler; -import org.apache.beam.runners.dataflow.worker.streaming.ComputationState; -import org.apache.beam.runners.dataflow.worker.windmill.Windmill; -import org.apache.beam.sdk.annotations.Internal; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; -import org.joda.time.Duration; -import org.joda.time.Instant; - -@Internal -@ThreadSafe -public final class DispatchedActiveWorkRefresher extends ActiveWorkRefresher { - - private final Consumer>> activeWorkRefresherFn; - - private DispatchedActiveWorkRefresher( - Supplier clock, - int activeWorkRefreshPeriodMillis, - int stuckCommitDurationMillis, - Supplier> computations, - DataflowExecutionStateSampler sampler, - Consumer>> activeWorkRefresherFn, - ScheduledExecutorService scheduledExecutorService) { - super( - clock, - activeWorkRefreshPeriodMillis, - stuckCommitDurationMillis, - computations, - sampler, - scheduledExecutorService); - this.activeWorkRefresherFn = activeWorkRefresherFn; - } - - public static DispatchedActiveWorkRefresher create( - Supplier clock, - int activeWorkRefreshPeriodMillis, - int stuckCommitDurationMillis, - Supplier> computations, - DataflowExecutionStateSampler sampler, - Consumer>> activeWorkRefresherFn, - ScheduledExecutorService scheduledExecutorService) { - return new DispatchedActiveWorkRefresher( - clock, - activeWorkRefreshPeriodMillis, - stuckCommitDurationMillis, - computations, - sampler, - activeWorkRefresherFn, - scheduledExecutorService); - } - - @Override - protected void refreshActiveWork() { - Instant refreshDeadline = clock.get().minus(Duration.millis(activeWorkRefreshPeriodMillis)); - - ImmutableMap> heartbeats = - computations.get().stream() - .collect( - toImmutableMap( - ComputationState::getComputationId, - computationState -> - HeartbeatRequests.getRefreshableKeyHeartbeats( - computationState.currentActiveWorkReadOnly(), - refreshDeadline, - sampler))); - - activeWorkRefresherFn.accept(heartbeats); - } -} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/FanOutActiveWorkRefresher.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/FanOutActiveWorkRefresher.java deleted file mode 100644 index 42f1aeeae7141..0000000000000 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/FanOutActiveWorkRefresher.java +++ /dev/null @@ -1,136 +0,0 @@ -/* - * 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.windmill.work.refresh; - -import java.util.ArrayList; -import java.util.Collection; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.concurrent.Executors; -import java.util.concurrent.ScheduledExecutorService; -import java.util.function.Consumer; -import java.util.function.Supplier; -import javax.annotation.concurrent.ThreadSafe; -import org.apache.beam.runners.dataflow.worker.DataflowExecutionStateSampler; -import org.apache.beam.runners.dataflow.worker.streaming.ComputationState; -import org.apache.beam.runners.dataflow.worker.windmill.Windmill.HeartbeatRequest; -import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStream.GetDataStream; -import org.apache.beam.sdk.annotations.Internal; -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.collect.ImmutableListMultimap; -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; - -/** - * {@link ActiveWorkRefresher} implementation that fans out refreshes active work to multiple {@link - * GetDataStream}. - */ -@Internal -@ThreadSafe -public class FanOutActiveWorkRefresher extends ActiveWorkRefresher { - - private static final String REFRESH_ACTIVE_WORK_EXECUTOR = "RefreshWorkWithFanOut"; - private final Consumer>>> - refreshActiveWorkFn; - - private FanOutActiveWorkRefresher( - Supplier clock, - int activeWorkRefreshPeriodMillis, - int stuckCommitDurationMillis, - Supplier> computations, - DataflowExecutionStateSampler sampler, - Consumer>>> refreshActiveWorkFn, - ScheduledExecutorService activeWorkRefreshExecutor) { - super( - clock, - activeWorkRefreshPeriodMillis, - stuckCommitDurationMillis, - computations, - sampler, - activeWorkRefreshExecutor); - this.refreshActiveWorkFn = refreshActiveWorkFn; - } - - public static ActiveWorkRefresher create( - Supplier clock, - int activeWorkRefreshPeriodMillis, - int stuckCommitDurationMillis, - Supplier> computations, - DataflowExecutionStateSampler sampler, - Consumer>>> refreshActiveWorkFn) { - return new FanOutActiveWorkRefresher( - clock, - activeWorkRefreshPeriodMillis, - stuckCommitDurationMillis, - computations, - sampler, - refreshActiveWorkFn, - Executors.newSingleThreadScheduledExecutor( - new ThreadFactoryBuilder().setNameFormat(REFRESH_ACTIVE_WORK_EXECUTOR).build())); - } - - @VisibleForTesting - static FanOutActiveWorkRefresher forTesting( - Supplier clock, - int activeWorkRefreshPeriodMillis, - int stuckCommitDurationMillis, - Supplier> computations, - DataflowExecutionStateSampler sampler, - Consumer>>> refreshActiveWorkFn, - ScheduledExecutorService activeWorkRefreshExecutor) { - return new FanOutActiveWorkRefresher( - clock, - activeWorkRefreshPeriodMillis, - stuckCommitDurationMillis, - computations, - sampler, - refreshActiveWorkFn, - activeWorkRefreshExecutor); - } - - @Override - protected void refreshActiveWork() { - Instant refreshDeadline = clock.get().minus(Duration.millis(activeWorkRefreshPeriodMillis)); - - Map>> fannedOutHeartbeatRequests = - new HashMap<>(); - for (ComputationState computationState : computations.get()) { - String computationId = computationState.getComputationId(); - - // Get heartbeat requests for computation's current active work, aggregated by GetDataStream - // to correctly fan-out the heartbeat requests. - ImmutableListMultimap heartbeats = - HeartbeatRequests.getRefreshableDirectKeyHeartbeats( - computationState.currentActiveWorkReadOnly(), refreshDeadline, sampler); - // Aggregate the heartbeats across computations by GetDataStream for correct fan out. - for (Map.Entry> heartbeatsPerStream : - heartbeats.asMap().entrySet()) { - Map> existingHeartbeats = - fannedOutHeartbeatRequests.computeIfAbsent( - heartbeatsPerStream.getKey(), ignored -> new HashMap<>()); - List existingHeartbeatsForComputation = - existingHeartbeats.computeIfAbsent(computationId, ignored -> new ArrayList<>()); - existingHeartbeatsForComputation.addAll(heartbeatsPerStream.getValue()); - } - } - - refreshActiveWorkFn.accept(fannedOutHeartbeatRequests); - } -} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/HeartbeatRequests.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/HeartbeatRequests.java index 9b79d587b0cbd..78a7c24bb3a27 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/HeartbeatRequests.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/HeartbeatRequests.java @@ -17,7 +17,6 @@ */ package org.apache.beam.runners.dataflow.worker.windmill.work.refresh; -import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList.toImmutableList; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableListMultimap.toImmutableListMultimap; import java.util.Collection; @@ -28,10 +27,7 @@ import org.apache.beam.runners.dataflow.worker.streaming.ShardedKey; import org.apache.beam.runners.dataflow.worker.streaming.Work; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.HeartbeatRequest; -import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStream.GetDataStream; import org.apache.beam.sdk.annotations.Internal; -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.collect.ImmutableList; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableListMultimap; import org.joda.time.Instant; @@ -41,53 +37,27 @@ public final class HeartbeatRequests { private HeartbeatRequests() {} - static ImmutableList getRefreshableKeyHeartbeats( + static ImmutableListMultimap getRefreshableKeyHeartbeats( ImmutableListMultimap activeWork, Instant refreshDeadline, DataflowExecutionStateSampler sampler) { return activeWork.asMap().entrySet().stream() - .flatMap(entry -> toHeartbeatRequestStream(entry, refreshDeadline, sampler)) - .collect(toImmutableList()); - } - - static ImmutableListMultimap getRefreshableDirectKeyHeartbeats( - ImmutableListMultimap activeWork, - Instant refreshDeadline, - DataflowExecutionStateSampler sampler) { - return activeWork.asMap().entrySet().stream() - .flatMap(e -> toDirectHeartbeatRequest(e, refreshDeadline, sampler)) + .flatMap(e -> toHeartbeatRequest(e, refreshDeadline, sampler)) .collect(toImmutableListMultimap(Pair::getKey, Pair::getValue)); } - private static Stream> toDirectHeartbeatRequest( + private static Stream> toHeartbeatRequest( Map.Entry> shardedKeyAndWorkQueue, Instant refreshDeadline, DataflowExecutionStateSampler sampler) { ShardedKey shardedKey = shardedKeyAndWorkQueue.getKey(); Collection workQueue = shardedKeyAndWorkQueue.getValue(); return getRefreshableWork(workQueue, refreshDeadline) - // If the stream was explicitly closed fail the Work as the backend worker is invalid. - .peek(HeartbeatRequests::failWorkForClosedStream) // Don't send heartbeats for queued work we already know is failed. .filter(work -> !work.isFailed()) .map( work -> - Pair.of( - Preconditions.checkNotNull(work.getDataStream()), - createHeartbeatRequest(shardedKey, work, sampler))); - } - - private static Stream toHeartbeatRequestStream( - Map.Entry> shardedKeyAndWorkQueue, - Instant refreshDeadline, - DataflowExecutionStateSampler sampler) { - ShardedKey shardedKey = shardedKeyAndWorkQueue.getKey(); - Collection workQueue = shardedKeyAndWorkQueue.getValue(); - - return getRefreshableWork(workQueue, refreshDeadline) - // Don't send heartbeats for queued work we already know is failed. - .filter(work -> !work.isFailed()) - .map(work -> createHeartbeatRequest(shardedKey, work, sampler)); + Pair.of(work.heartbeatSender(), createHeartbeatRequest(shardedKey, work, sampler))); } private static HeartbeatRequest createHeartbeatRequest( @@ -102,12 +72,6 @@ private static HeartbeatRequest createHeartbeatRequest( private static Stream getRefreshableWork( Collection workQueue, Instant refreshDeadline) { - return workQueue.stream().filter(work -> work.getStartTime().isBefore(refreshDeadline)); - } - - private static void failWorkForClosedStream(Work work) { - if (Preconditions.checkNotNull(work.getDataStream()).isClosed()) { - work.setFailed(); - } + return workQueue.stream().filter(work -> work.isRefreshable(refreshDeadline)); } } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/HeartbeatSender.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/HeartbeatSender.java new file mode 100644 index 0000000000000..3a2d0cb05fc5a --- /dev/null +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/HeartbeatSender.java @@ -0,0 +1,32 @@ +/* + * 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.windmill.work.refresh; + +import java.util.List; +import java.util.Map; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill; + +/** Interface for sending heartbeats. */ +@FunctionalInterface +public interface HeartbeatSender { + void sendHeartbeats(Map> heartbeats); + + default boolean isInvalid() { + return false; + } +} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/PoolBackedHeartbeatSender.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/PoolBackedHeartbeatSender.java new file mode 100644 index 0000000000000..bf4ea360e6146 --- /dev/null +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/PoolBackedHeartbeatSender.java @@ -0,0 +1,51 @@ +/* + * 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.windmill.work.refresh; + +import java.util.List; +import java.util.Map; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill; +import org.apache.beam.runners.dataflow.worker.windmill.client.CloseableStream; +import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStream; +import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStreamPool; +import org.apache.beam.sdk.annotations.Internal; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** StreamingEngine stream pool based implementation of {@link HeartbeatSender}. */ +@Internal +public final class PoolBackedHeartbeatSender implements HeartbeatSender { + private static final Logger LOG = LoggerFactory.getLogger(PoolBackedHeartbeatSender.class); + + private final WindmillStreamPool heartbeatStreamPool; + + public PoolBackedHeartbeatSender( + WindmillStreamPool heartbeatStreamPool) { + this.heartbeatStreamPool = heartbeatStreamPool; + } + + @Override + public void sendHeartbeats(Map> heartbeats) { + try (CloseableStream closeableStream = + heartbeatStreamPool.getCloseableStream()) { + closeableStream.stream().refreshActiveWork(heartbeats); + } catch (Exception e) { + LOG.warn("Error occurred sending heartbeats.", e); + } + } +} 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 9b39a86055af3..0e0baeac72dee 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 @@ -335,7 +335,8 @@ private static ExecutableWork createMockWork( Work.createProcessingContext( computationId, (a, b) -> Windmill.KeyedGetDataResponse.getDefaultInstance(), - ignored -> {}), + ignored -> {}, + heartbeats -> {}), Instant::now, Collections.emptyList()), processWorkFn); @@ -3419,7 +3420,8 @@ public void testLatencyAttributionProtobufsPopulated() { Work.createProcessingContext( "computationId", (a, b) -> Windmill.KeyedGetDataResponse.getDefaultInstance(), - ignored -> {}), + ignored -> {}, + heartbeats -> {}), clock, Collections.emptyList()); @@ -3721,7 +3723,7 @@ public void testDoFnActiveMessageMetadataReportedOnHeartbeat() throws Exception Map result = server.waitForAndGetCommits(1); assertThat(server.numGetDataRequests(), greaterThan(0)); - Windmill.GetDataRequest heartbeat = server.getGetDataRequests().get(2); + Windmill.GetDataRequest heartbeat = server.getGetDataRequests().get(1); for (LatencyAttribution la : heartbeat 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 2193f20f3fe3e..fb5b59d2cc41a 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 @@ -132,6 +132,7 @@ private static Work createMockWork(Windmill.WorkItem workItem, Watermarks waterm Work.createProcessingContext( COMPUTATION_ID, (a, b) -> Windmill.KeyedGetDataResponse.getDefaultInstance(), + ignored -> {}, ignored -> {}), Instant::now, Collections.emptyList()); 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 9f97c9835ddc8..5db892721f1d9 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 @@ -199,6 +199,7 @@ private static Work createMockWork(Windmill.WorkItem workItem, Watermarks waterm Work.createProcessingContext( COMPUTATION_ID, (a, b) -> Windmill.KeyedGetDataResponse.getDefaultInstance(), + ignored -> {}, ignored -> {}), Instant::now, Collections.emptyList()); @@ -997,7 +998,8 @@ public void testFailedWorkItemsAbort() throws Exception { Work.createProcessingContext( COMPUTATION_ID, (a, b) -> Windmill.KeyedGetDataResponse.getDefaultInstance(), - gnored -> {}), + ignored -> {}, + ignored -> {}), Instant::now, Collections.emptyList()); context.start( diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/ActiveWorkStateTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/ActiveWorkStateTest.java index 3a3e0a34c2179..1ba79e0250d87 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/ActiveWorkStateTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/ActiveWorkStateTest.java @@ -87,6 +87,7 @@ private static Work.ProcessingContext createWorkProcessingContext() { return Work.createProcessingContext( "computationId", (a, b) -> Windmill.KeyedGetDataResponse.getDefaultInstance(), + ignored -> {}, ignored -> {}); } diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/ComputationStateCacheTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/ComputationStateCacheTest.java index 3c1683ecf436a..6176fe6c900c1 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/ComputationStateCacheTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/ComputationStateCacheTest.java @@ -69,6 +69,7 @@ private static ExecutableWork createWork(ShardedKey shardedKey, long workToken, Work.createProcessingContext( "computationId", (a, b) -> Windmill.KeyedGetDataResponse.getDefaultInstance(), + ignored -> {}, ignored -> {}), Instant::now, Collections.emptyList()), diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/commits/StreamingApplianceWorkCommitterTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/commits/StreamingApplianceWorkCommitterTest.java index 85e07c3bd797c..ad612611c96bd 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/commits/StreamingApplianceWorkCommitterTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/commits/StreamingApplianceWorkCommitterTest.java @@ -67,7 +67,8 @@ private static Work createMockWork(long workToken) { (a, b) -> Windmill.KeyedGetDataResponse.getDefaultInstance(), ignored -> { throw new UnsupportedOperationException(); - }), + }, + ignored -> {}), Instant::now, Collections.emptyList()); } diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/commits/StreamingEngineWorkCommitterTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/commits/StreamingEngineWorkCommitterTest.java index 9ed8eacc3c69d..2e345ad8ef8d2 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/commits/StreamingEngineWorkCommitterTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/commits/StreamingEngineWorkCommitterTest.java @@ -84,7 +84,8 @@ private static Work createMockWork(long workToken) { (a, b) -> Windmill.KeyedGetDataResponse.getDefaultInstance(), ignored -> { throw new UnsupportedOperationException(); - }), + }, + ignored -> {}), Instant::now, Collections.emptyList()); } diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/work/processing/failures/WorkFailureProcessorTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/work/processing/failures/WorkFailureProcessorTest.java index bd55595da1355..719cf730bd95e 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/work/processing/failures/WorkFailureProcessorTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/work/processing/failures/WorkFailureProcessorTest.java @@ -87,6 +87,7 @@ private static ExecutableWork createWork(Supplier clock, Consumer Work.createProcessingContext( "computationId", (a, b) -> Windmill.KeyedGetDataResponse.getDefaultInstance(), + ignored -> {}, ignored -> {}), clock, new ArrayList<>()), diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/DispatchedActiveWorkRefresherTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/ActiveWorkRefresherTest.java similarity index 80% rename from runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/DispatchedActiveWorkRefresherTest.java rename to runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/ActiveWorkRefresherTest.java index 01a46a12e3eb1..c467af7bbc6fb 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/DispatchedActiveWorkRefresherTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/ActiveWorkRefresherTest.java @@ -38,6 +38,7 @@ import java.util.concurrent.TimeUnit; import java.util.function.Consumer; import java.util.function.Supplier; +import java.util.stream.Collectors; import org.apache.beam.runners.dataflow.worker.DataflowExecutionStateSampler; import org.apache.beam.runners.dataflow.worker.streaming.ComputationState; import org.apache.beam.runners.dataflow.worker.streaming.ExecutableWork; @@ -46,12 +47,12 @@ import org.apache.beam.runners.dataflow.worker.streaming.Work; import org.apache.beam.runners.dataflow.worker.util.BoundedQueueExecutor; import org.apache.beam.runners.dataflow.worker.windmill.Windmill; -import org.apache.beam.runners.dataflow.worker.windmill.Windmill.HeartbeatRequest; import org.apache.beam.runners.dataflow.worker.windmill.state.WindmillStateCache; import org.apache.beam.runners.direct.Clock; import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.HashBasedTable; 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.Iterables; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Table; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.ThreadFactoryBuilder; import org.joda.time.Duration; @@ -61,8 +62,7 @@ import org.junit.runners.JUnit4; @RunWith(JUnit4.class) -public class DispatchedActiveWorkRefresherTest { - +public class ActiveWorkRefresherTest { private static final Supplier A_LONG_TIME_AGO = () -> Instant.parse("1998-09-04T00:00:00Z"); private static final String COMPUTATION_ID_PREFIX = "ComputationId-"; @@ -97,15 +97,21 @@ private ActiveWorkRefresher createActiveWorkRefresher( int activeWorkRefreshPeriodMillis, int stuckCommitDurationMillis, Supplier> computations, - Consumer>> activeWorkRefresherFn) { - return DispatchedActiveWorkRefresher.create( + Consumer>>> + activeWorkRefresherFn) { + return new ActiveWorkRefresher( clock, activeWorkRefreshPeriodMillis, stuckCommitDurationMillis, computations, DataflowExecutionStateSampler.instance(), - activeWorkRefresherFn, - Executors.newSingleThreadScheduledExecutor()); + Executors.newSingleThreadScheduledExecutor(), + activeWorkRefresherFn); + } + + private ExecutableWork createOldWork(int workIds, Consumer processWork) { + ShardedKey shardedKey = ShardedKey.create(ByteString.EMPTY, workIds); + return createOldWork(shardedKey, workIds, processWork); } private ExecutableWork createOldWork( @@ -122,8 +128,9 @@ private ExecutableWork createOldWork( Work.createProcessingContext( "computationId", (a, b) -> Windmill.KeyedGetDataResponse.getDefaultInstance(), + ignored -> {}, ignored -> {}), - DispatchedActiveWorkRefresherTest.A_LONG_TIME_AGO, + A_LONG_TIME_AGO, ImmutableList.of()), processWork); } @@ -147,8 +154,7 @@ public void testActiveWorkRefresh() throws InterruptedException { Map> computationsAndWork = new HashMap<>(); for (int i = 0; i < 5; i++) { ComputationState computationState = createComputationState(i); - ExecutableWork fakeWork = - createOldWork(ShardedKey.create(ByteString.EMPTY, i), i, processWork); + ExecutableWork fakeWork = createOldWork(i, processWork); computationState.activateWork(fakeWork); computations.add(computationState); @@ -158,10 +164,10 @@ public void testActiveWorkRefresh() throws InterruptedException { activeWorkForComputation.add(fakeWork); } - Map> expectedHeartbeats = new HashMap<>(); + Map>> fanoutExpectedHeartbeats = + new HashMap<>(); CountDownLatch heartbeatsSent = new CountDownLatch(1); TestClock fakeClock = new TestClock(Instant.now()); - ActiveWorkRefresher activeWorkRefresher = createActiveWorkRefresher( fakeClock::now, @@ -169,7 +175,7 @@ public void testActiveWorkRefresh() throws InterruptedException { 0, () -> computations, heartbeats -> { - expectedHeartbeats.putAll(heartbeats); + fanoutExpectedHeartbeats.putAll(heartbeats); heartbeatsSent.countDown(); }); @@ -178,23 +184,29 @@ public void testActiveWorkRefresh() throws InterruptedException { heartbeatsSent.await(); activeWorkRefresher.stop(); - assertThat(computationsAndWork.size()).isEqualTo(expectedHeartbeats.size()); - for (Map.Entry> expectedHeartbeat : - expectedHeartbeats.entrySet()) { - String computationId = expectedHeartbeat.getKey(); - List heartbeatRequests = expectedHeartbeat.getValue(); - List work = computationsAndWork.get(computationId); - - // Compare the heartbeatRequest's and Work's workTokens, cacheTokens, and shardingKeys. - assertThat(heartbeatRequests) - .comparingElementsUsing( - Correspondence.from( - (HeartbeatRequest h, ExecutableWork w) -> - h.getWorkToken() == w.getWorkItem().getWorkToken() - && h.getCacheToken() == w.getWorkItem().getWorkToken() - && h.getShardingKey() == w.getWorkItem().getShardingKey(), - "heartbeatRequest's and Work's workTokens, cacheTokens, and shardingKeys should be equal.")) - .containsExactlyElementsIn(work); + assertThat(computationsAndWork.size()) + .isEqualTo(Iterables.getOnlyElement(fanoutExpectedHeartbeats.values()).size()); + for (Map.Entry>> + fanOutExpectedHeartbeat : fanoutExpectedHeartbeats.entrySet()) { + for (Map.Entry> expectedHeartbeat : + fanOutExpectedHeartbeat.getValue().entrySet()) { + String computationId = expectedHeartbeat.getKey(); + List heartbeatRequests = expectedHeartbeat.getValue(); + List work = + computationsAndWork.get(computationId).stream() + .map(ExecutableWork::work) + .collect(Collectors.toList()); + // Compare the heartbeatRequest's and Work's workTokens, cacheTokens, and shardingKeys. + assertThat(heartbeatRequests) + .comparingElementsUsing( + Correspondence.from( + (Windmill.HeartbeatRequest h, Work w) -> + h.getWorkToken() == w.getWorkItem().getWorkToken() + && h.getCacheToken() == w.getWorkItem().getWorkToken() + && h.getShardingKey() == w.getWorkItem().getShardingKey(), + "heartbeatRequest's and Work's workTokens, cacheTokens, and shardingKeys should be equal.")) + .containsExactlyElementsIn(work); + } } activeWorkRefresher.stop(); diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/FanOutActiveWorkRefresherTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/FanOutActiveWorkRefresherTest.java deleted file mode 100644 index 9a34760d860b1..0000000000000 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/FanOutActiveWorkRefresherTest.java +++ /dev/null @@ -1,221 +0,0 @@ -/* - * 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.windmill.work.refresh; - -import static com.google.common.truth.Truth.assertThat; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; - -import com.google.api.services.dataflow.model.MapTask; -import com.google.common.truth.Correspondence; -import java.util.ArrayList; -import java.util.Collection; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.Executors; -import java.util.concurrent.TimeUnit; -import java.util.function.Consumer; -import java.util.function.Supplier; -import java.util.stream.Collectors; -import org.apache.beam.runners.dataflow.worker.DataflowExecutionStateSampler; -import org.apache.beam.runners.dataflow.worker.streaming.ComputationState; -import org.apache.beam.runners.dataflow.worker.streaming.ExecutableWork; -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.util.BoundedQueueExecutor; -import org.apache.beam.runners.dataflow.worker.windmill.Windmill; -import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStream; -import org.apache.beam.runners.dataflow.worker.windmill.state.WindmillStateCache; -import org.apache.beam.runners.direct.Clock; -import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; -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.util.concurrent.ThreadFactoryBuilder; -import org.joda.time.Duration; -import org.joda.time.Instant; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; - -@RunWith(JUnit4.class) -public class FanOutActiveWorkRefresherTest { - private static final Supplier A_LONG_TIME_AGO = - () -> Instant.parse("1998-09-04T00:00:00Z"); - private static final String COMPUTATION_ID_PREFIX = "ComputationId-"; - - private static BoundedQueueExecutor workExecutor() { - return new BoundedQueueExecutor( - 1, - 60, - TimeUnit.SECONDS, - 1, - 10000000, - new ThreadFactoryBuilder().setNameFormat("DataflowWorkUnits-%d").setDaemon(true).build()); - } - - private static ComputationState createComputationState(int computationIdSuffix) { - return createComputationState( - computationIdSuffix, mock(WindmillStateCache.ForComputation.class)); - } - - private static ComputationState createComputationState( - int computationIdSuffix, WindmillStateCache.ForComputation stateCache) { - return new ComputationState( - COMPUTATION_ID_PREFIX + computationIdSuffix, - new MapTask().setStageName("stageName").setSystemName("systemName"), - workExecutor(), - new HashMap<>(), - stateCache); - } - - private ActiveWorkRefresher createActiveWorkRefresher( - Supplier clock, - int activeWorkRefreshPeriodMillis, - int stuckCommitDurationMillis, - Supplier> computations, - Consumer>>> - activeWorkRefresherFn) { - return FanOutActiveWorkRefresher.forTesting( - clock, - activeWorkRefreshPeriodMillis, - stuckCommitDurationMillis, - computations, - DataflowExecutionStateSampler.instance(), - activeWorkRefresherFn, - Executors.newSingleThreadScheduledExecutor()); - } - - private ExecutableWork createOldWork(int workIds, Consumer processWork) { - ShardedKey shardedKey = ShardedKey.create(ByteString.EMPTY, workIds); - return ExecutableWork.create( - Work.create( - Windmill.WorkItem.newBuilder() - .setWorkToken(workIds) - .setCacheToken(workIds) - .setKey(shardedKey.key()) - .setShardingKey(shardedKey.shardingKey()) - .build(), - Watermarks.builder().setInputDataWatermark(Instant.EPOCH).build(), - Work.createFanOutProcessingContext( - "computation", - (computationId, request) -> Windmill.KeyedGetDataResponse.getDefaultInstance(), - ignored -> {}, - mock(WindmillStream.GetDataStream.class)), - A_LONG_TIME_AGO, - ImmutableList.of()), - processWork); - } - - @Test - public void testActiveWorkRefresh() throws InterruptedException { - int activeWorkRefreshPeriodMillis = 100; - - // Block work processing to queue up the work. - CountDownLatch workIsProcessed = new CountDownLatch(1); - Consumer processWork = - ignored -> { - try { - workIsProcessed.await(); - } catch (InterruptedException e) { - throw new RuntimeException(e); - } - }; - - List computations = new ArrayList<>(); - Map> computationsAndWork = new HashMap<>(); - for (int i = 0; i < 5; i++) { - ComputationState computationState = createComputationState(i); - ExecutableWork fakeWork = createOldWork(i, processWork); - computationState.activateWork(fakeWork); - - computations.add(computationState); - List activeWorkForComputation = - computationsAndWork.computeIfAbsent( - computationState.getComputationId(), ignored -> new ArrayList<>()); - activeWorkForComputation.add(fakeWork); - when(fakeWork.work().getDataStream().isClosed()).thenReturn(false); - } - - Map>> - fanoutExpectedHeartbeats = new HashMap<>(); - CountDownLatch heartbeatsSent = new CountDownLatch(1); - TestClock fakeClock = new TestClock(Instant.now()); - ActiveWorkRefresher activeWorkRefresher = - createActiveWorkRefresher( - fakeClock::now, - activeWorkRefreshPeriodMillis, - 0, - () -> computations, - heartbeats -> { - fanoutExpectedHeartbeats.putAll(heartbeats); - heartbeatsSent.countDown(); - }); - - activeWorkRefresher.start(); - fakeClock.advance(Duration.millis(activeWorkRefreshPeriodMillis * 2)); - heartbeatsSent.await(); - activeWorkRefresher.stop(); - - assertThat(computationsAndWork.size()).isEqualTo(fanoutExpectedHeartbeats.size()); - for (Map.Entry>> - fanOutExpectedHeartbeat : fanoutExpectedHeartbeats.entrySet()) { - for (Map.Entry> expectedHeartbeat : - fanOutExpectedHeartbeat.getValue().entrySet()) { - String computationId = expectedHeartbeat.getKey(); - List heartbeatRequests = expectedHeartbeat.getValue(); - List work = - computationsAndWork.get(computationId).stream() - .map(ExecutableWork::work) - .collect(Collectors.toList()); - // Compare the heartbeatRequest's and Work's workTokens, cacheTokens, and shardingKeys. - assertThat(heartbeatRequests) - .comparingElementsUsing( - Correspondence.from( - (Windmill.HeartbeatRequest h, Work w) -> - h.getWorkToken() == w.getWorkItem().getWorkToken() - && h.getCacheToken() == w.getWorkItem().getWorkToken() - && h.getShardingKey() == w.getWorkItem().getShardingKey(), - "heartbeatRequest's and Work's workTokens, cacheTokens, and shardingKeys should be equal.")) - .containsExactlyElementsIn(work); - } - } - - activeWorkRefresher.stop(); - // Free the work processing threads. - workIsProcessed.countDown(); - } - - static class TestClock implements Clock { - private Instant time; - - private TestClock(Instant startTime) { - this.time = startTime; - } - - private synchronized void advance(Duration amount) { - time = time.plus(amount); - } - - @Override - public synchronized Instant now() { - return time; - } - } -} diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/HeartbeatRequestsTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/HeartbeatRequestsTest.java index cd863216fe0f3..f4bafe6fb3ade 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/HeartbeatRequestsTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/HeartbeatRequestsTest.java @@ -19,8 +19,6 @@ import static com.google.common.truth.Truth.assertThat; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList.toImmutableList; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; import com.google.auto.value.AutoValue; import java.util.ArrayDeque; @@ -29,13 +27,11 @@ import java.util.HashMap; import java.util.Map; import java.util.Objects; -import javax.annotation.Nullable; import org.apache.beam.runners.dataflow.worker.DataflowExecutionStateSampler; 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.windmill.Windmill; -import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStream; import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; 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.ImmutableListMultimap; @@ -50,12 +46,11 @@ public class HeartbeatRequestsTest { private Map> activeWork; - private static Work createWork( - Windmill.WorkItem workItem, WindmillStream.GetDataStream getDataStream) { + private static Work createWork(Windmill.WorkItem workItem, HeartbeatSender heartbeatSender) { return Work.create( workItem, Watermarks.builder().setInputDataWatermark(Instant.EPOCH).build(), - createProcessingContext(getDataStream), + createProcessingContext(heartbeatSender), Instant::now, Collections.emptyList()); } @@ -64,46 +59,19 @@ private static ShardedKey shardedKey(String str, long shardKey) { return ShardedKey.create(ByteString.copyFromUtf8(str), shardKey); } - private static Work createWork(Windmill.WorkItem workItem) { - return Work.create( - workItem, - Watermarks.builder().setInputDataWatermark(Instant.EPOCH).build(), - createProcessingContext(), - Instant::now, - Collections.emptyList()); - } - - private static Work expiredWork(Windmill.WorkItem workItem) { - return Work.create( - workItem, - Watermarks.builder().setInputDataWatermark(Instant.EPOCH).build(), - createProcessingContext(), - () -> Instant.EPOCH, - Collections.emptyList()); - } - - private static Work.ProcessingContext createProcessingContext() { + private static Work.ProcessingContext createProcessingContext(HeartbeatSender heartbeatSender) { return Work.createProcessingContext( - "computationId", - (computationId, request) -> Windmill.KeyedGetDataResponse.getDefaultInstance(), - ignored -> {}); - } - - private static Work.ProcessingContext createProcessingContext( - WindmillStream.GetDataStream getDataStream) { - return Work.createFanOutProcessingContext( "computationId", (computationId, request) -> Windmill.KeyedGetDataResponse.getDefaultInstance(), ignored -> {}, - getDataStream); + heartbeatSender); } - private static Work expiredWork( - Windmill.WorkItem workItem, WindmillStream.GetDataStream getDataStream) { + private static Work expiredWork(Windmill.WorkItem workItem, HeartbeatSender heartbeatSender) { return Work.create( workItem, Watermarks.builder().setInputDataWatermark(Instant.EPOCH).build(), - createProcessingContext(getDataStream), + createProcessingContext(heartbeatSender), () -> Instant.EPOCH, Collections.emptyList()); } @@ -122,51 +90,16 @@ public void setUp() { activeWork = new HashMap<>(); } - @Test - public void testGetRefreshableKeyHeartbeats() { - Instant refreshDeadline = Instant.now(); - - Work freshWork = createWork(createWorkItem(3L, 3L)); - Work refreshableWork1 = expiredWork(createWorkItem(1L, 1L)); - refreshableWork1.setState(Work.State.COMMITTING); - Work refreshableWork2 = expiredWork(createWorkItem(2L, 2L)); - refreshableWork2.setState(Work.State.COMMITTING); - ShardedKey shardedKey1 = shardedKey("someKey", 1L); - ShardedKey shardedKey2 = shardedKey("anotherKey", 2L); - - activateWorkForKey(shardedKey1, refreshableWork1); - activateWorkForKey(shardedKey1, freshWork); - activateWorkForKey(shardedKey2, refreshableWork2); - - ImmutableList requests = - HeartbeatRequests.getRefreshableKeyHeartbeats( - currentActiveWork(), refreshDeadline, DataflowExecutionStateSampler.instance()); - - ImmutableList expected = - ImmutableList.of( - HeartbeatRequestShardingKeyWorkTokenAndCacheToken.from(shardedKey1, refreshableWork1), - HeartbeatRequestShardingKeyWorkTokenAndCacheToken.from(shardedKey2, refreshableWork2)); - - ImmutableList actual = - requests.stream() - .map(HeartbeatRequestShardingKeyWorkTokenAndCacheToken::from) - .collect(toImmutableList()); - - assertThat(actual).containsExactlyElementsIn(expected); - } - @Test public void testGetRefreshableFanoutKeyHeartbeats() { Instant refreshDeadline = Instant.now(); - WindmillStream.GetDataStream getDataStream1 = mock(WindmillStream.GetDataStream.class); - when(getDataStream1.isClosed()).thenReturn(false); - WindmillStream.GetDataStream getDataStream2 = mock(WindmillStream.GetDataStream.class); - when(getDataStream2.isClosed()).thenReturn(false); + HeartbeatSender sender1 = ignored -> {}; + HeartbeatSender sender2 = ignored -> {}; - Work freshWork = createWork(createWorkItem(3L, 3L), getDataStream1); - Work refreshableWork1 = expiredWork(createWorkItem(1L, 1L), getDataStream1); + Work freshWork = createWork(createWorkItem(3L, 3L), sender1); + Work refreshableWork1 = expiredWork(createWorkItem(1L, 1L), sender1); refreshableWork1.setState(Work.State.COMMITTING); - Work refreshableWork2 = expiredWork(createWorkItem(2L, 2L), getDataStream2); + Work refreshableWork2 = expiredWork(createWorkItem(2L, 2L), sender2); refreshableWork2.setState(Work.State.COMMITTING); ShardedKey shardedKey1 = shardedKey("someKey", 1L); ShardedKey shardedKey2 = shardedKey("anotherKey", 2L); @@ -175,8 +108,8 @@ public void testGetRefreshableFanoutKeyHeartbeats() { activateWorkForKey(shardedKey1, freshWork); activateWorkForKey(shardedKey2, refreshableWork2); - ImmutableListMultimap requests = - HeartbeatRequests.getRefreshableDirectKeyHeartbeats( + ImmutableListMultimap requests = + HeartbeatRequests.getRefreshableKeyHeartbeats( currentActiveWork(), refreshDeadline, DataflowExecutionStateSampler.instance()); ImmutableList expected = @@ -215,50 +148,27 @@ private ImmutableListMultimap currentActiveWork() { abstract static class HeartbeatRequestShardingKeyWorkTokenAndCacheToken { private static HeartbeatRequestShardingKeyWorkTokenAndCacheToken create( - long shardingKey, long workToken, long cacheToken) { - return new AutoValue_HeartbeatRequestsTest_HeartbeatRequestShardingKeyWorkTokenAndCacheToken( - shardingKey, workToken, cacheToken, null); - } - - private static HeartbeatRequestShardingKeyWorkTokenAndCacheToken create( - long shardingKey, - long workToken, - long cacheToken, - WindmillStream.GetDataStream getDataStream) { + long shardingKey, long workToken, long cacheToken, HeartbeatSender sender) { return new AutoValue_HeartbeatRequestsTest_HeartbeatRequestShardingKeyWorkTokenAndCacheToken( - shardingKey, workToken, cacheToken, Objects.requireNonNull(getDataStream)); - } - - private static HeartbeatRequestShardingKeyWorkTokenAndCacheToken from( - Windmill.HeartbeatRequest heartbeatRequest) { - return create( - heartbeatRequest.getShardingKey(), - heartbeatRequest.getWorkToken(), - heartbeatRequest.getCacheToken()); + shardingKey, workToken, cacheToken, sender); } private static HeartbeatRequestShardingKeyWorkTokenAndCacheToken from( ShardedKey shardedKey, Work work) { - @Nullable WindmillStream.GetDataStream getDataStream = work.getDataStream(); - return getDataStream == null - ? create( - shardedKey.shardingKey(), - work.getWorkItem().getWorkToken(), - work.getWorkItem().getCacheToken()) - : create( - shardedKey.shardingKey(), - work.getWorkItem().getWorkToken(), - work.getWorkItem().getCacheToken(), - work.getDataStream()); + return create( + shardedKey.shardingKey(), + work.getWorkItem().getWorkToken(), + work.getWorkItem().getCacheToken(), + work.heartbeatSender()); } private static HeartbeatRequestShardingKeyWorkTokenAndCacheToken from( - Windmill.HeartbeatRequest heartbeatRequest, WindmillStream.GetDataStream getDataStream) { + Windmill.HeartbeatRequest heartbeatRequest, HeartbeatSender sender) { return create( heartbeatRequest.getShardingKey(), heartbeatRequest.getWorkToken(), heartbeatRequest.getCacheToken(), - getDataStream); + sender); } abstract long shardingKey(); @@ -267,7 +177,7 @@ private static HeartbeatRequestShardingKeyWorkTokenAndCacheToken from( abstract long cacheToken(); - abstract @Nullable WindmillStream.GetDataStream getDataStream(); + abstract HeartbeatSender heartbeatSender(); @Override public final boolean equals(Object obj) { From ab51953280f9dc4565ff760c88e2cbede51916b7 Mon Sep 17 00:00:00 2001 From: Martin Trieu Date: Tue, 11 Jun 2024 16:16:34 -0700 Subject: [PATCH 03/13] address PR comments fix get data stream --- .../client/grpc/AppendableInputStream.java | 8 +- .../client/grpc/GrpcGetDataStream.java | 105 +++------ .../grpc/GrpcGetDataStreamRequests.java | 82 ++++++- .../refresh/PoolBackedHeartbeatSender.java | 2 +- .../worker/util/BoundedQueueExecutorTest.java | 1 + .../client/grpc/GrpcGetDataStreamTest.java | 214 ++++++++++++++++++ 6 files changed, 336 insertions(+), 76 deletions(-) create mode 100644 runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetDataStreamTest.java diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/AppendableInputStream.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/AppendableInputStream.java index 98545a4294615..b15f73645dee6 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/AppendableInputStream.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/AppendableInputStream.java @@ -134,6 +134,12 @@ public void close() throws IOException { stream.close(); } + static class InvalidInputStreamStateException extends IllegalStateException { + public InvalidInputStreamStateException() { + super("Got poison pill or timeout but stream is not done."); + } + } + @SuppressWarnings("NullableProblems") private class InputStreamEnumeration implements Enumeration { // The first stream is eagerly read on SequenceInputStream creation. For this reason @@ -159,7 +165,7 @@ public boolean hasMoreElements() { if (complete.get()) { return false; } - throw new IllegalStateException("Got poison pill or timeout but stream is not done."); + throw new InvalidInputStreamStateException(); } catch (InterruptedException e) { Thread.currentThread().interrupt(); throw new CancellationException(); diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetDataStream.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetDataStream.java index 7fa194d13b93d..aa30e6653c7b0 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetDataStream.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetDataStream.java @@ -30,11 +30,10 @@ import java.util.concurrent.CancellationException; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentLinkedDeque; -import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicLong; import java.util.function.Consumer; import java.util.function.Function; -import java.util.stream.Collectors; +import javax.annotation.Nullable; import org.apache.beam.runners.dataflow.worker.windmill.Windmill; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.ComputationGetDataRequest; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.ComputationHeartbeatRequest; @@ -131,43 +130,8 @@ public static GrpcGetDataStream create( return getDataStream; } - private static String createStreamCancelledErrorMessage(QueuedBatch batch) { - return batch.requests().stream() - .map( - request -> { - switch (request.getDataRequest().getKind()) { - case GLOBAL: - return "GetSideInput=" + request.getDataRequest().global(); - case COMPUTATION: - return request.getDataRequest().computation().getRequestsList().stream() - .map( - keyedRequest -> - "KeyedGetState=[" - + "key=" - + keyedRequest.getKey() - + "shardingKey=" - + keyedRequest.getShardingKey() - + "cacheToken=" - + keyedRequest.getCacheToken() - + "workToken" - + keyedRequest.getWorkToken() - + "]") - .collect(Collectors.joining()); - default: - // Will never happen switch is exhaustive. - throw new IllegalStateException(); - } - }) - .collect(Collectors.joining(",")); - } - @Override protected synchronized void onNewStream() { - // Stream has been explicitly closed. - if (isClosed()) { - return; - } - send(StreamingGetDataRequest.newBuilder().setHeader(jobHeader).build()); if (clientClosed.get()) { // We rely on close only occurring after all methods on the stream have returned. @@ -181,6 +145,22 @@ protected synchronized void onNewStream() { } } + @Override + public synchronized void close() { + super.close(); + + // Stream has been explicitly closed. Drain pending input streams and request batches. + // Future calls to send RPCs will fail. + pending.values().forEach(AppendableInputStream::cancel); + pending.clear(); + batches.forEach( + batch -> { + batch.markFinalized(); + batch.notifyFailed(); + }); + batches.clear(); + } + @Override protected boolean hasPendingRequests() { return !pending.isEmpty() || !batches.isEmpty(); @@ -300,17 +280,6 @@ public void sendHealthCheck() { } } - @Override - public synchronized void close() { - super.close(); - for (AppendableInputStream responseStream : pending.values()) { - responseStream.cancel(); - } - // Stream has been explicitly closed. - pending.clear(); - batches.clear(); - } - @Override public void appendSpecificHtml(PrintWriter writer) { writer.format( @@ -337,12 +306,24 @@ public void appendSpecificHtml(PrintWriter writer) { private ResponseT issueRequest(QueuedRequest request, ParseFn parseFn) { while (true) { + // Handle stream closure during loop. + if (isClosed()) { + throw new WindmillStreamClosedException( + "Cannot send request=[" + request + "] on closed stream."); + } + request.resetResponseStream(); try { queueRequestAndWait(request); return parseFn.parse(request.getResponseStream()); } catch (CancellationException e) { // Retry issuing the request since the response stream was cancelled. + } catch (AppendableInputStream.InvalidInputStreamStateException e) { + // Handle stream failure when trying to parse response stream. + if (isClosed()) { + throw new WindmillStreamClosedException( + "Cannot send request=[" + request + "] on closed stream."); + } } catch (IOException e) { LOG.error("Parsing GetData response failed: ", e); } catch (InterruptedException e) { @@ -357,7 +338,7 @@ private ResponseT issueRequest(QueuedRequest request, ParseFn= streamingRpcBatchLimit || batch.byteSize() + request.byteSize() > AbstractWindmillStream.RPC_STREAM_CHUNK_SIZE) { if (batch != null) { - waitForSendLatch = batch.getLatch(); + prevBatch = batch; } batch = new QueuedBatch(); batches.addLast(batch); @@ -374,12 +355,12 @@ private void queueRequestAndWait(QueuedRequest request) throws InterruptedExcept batch.addRequest(request); } if (responsibleForSend) { - if (waitForSendLatch == null) { + if (prevBatch == null) { // If there was not a previous batch wait a little while to improve // batching. Thread.sleep(1); } else { - waitForSendLatch.await(); + prevBatch.waitForSendOrFailNotification(); } // Finalize the batch so that no additional requests will be added. Leave the batch in the // queue so that a subsequent batch will wait for its completion. @@ -393,26 +374,10 @@ private void queueRequestAndWait(QueuedRequest request) throws InterruptedExcept } // Notify all waiters with requests in this batch as well as the sender // of the next batch (if one exists). - batch.countDown(); + batch.notifySent(); } else { // Wait for this batch to be sent before parsing the response. - boolean batchSent = false; - long secondsWaited = 0; - long waitFor = 10; - while (!batchSent) { - if (isClosed()) { - throw new WindmillStreamClosedException( - "Requests failed for batch containing " - + createStreamCancelledErrorMessage(batch) - + " requests. This is most likely due to the stream being explicitly closed" - + " which happens when the work is marked as invalid on the streaming" - + " backend when key ranges shuffle around. This is transient corresponding " - + " work will eventually be retried"); - } - batchSent = batch.await(waitFor); - LOG.debug("Waiting for batch={} to be sent for {}", batch, secondsWaited); - secondsWaited += waitFor; - } + batch.waitForSendOrFailNotification(); } } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetDataStreamRequests.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetDataStreamRequests.java index f6907618ba40a..3a5b516d217a9 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetDataStreamRequests.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetDataStreamRequests.java @@ -22,11 +22,13 @@ import java.util.Comparator; import java.util.List; import java.util.concurrent.CountDownLatch; -import java.util.concurrent.TimeUnit; +import java.util.function.Consumer; +import java.util.stream.Collectors; import org.apache.beam.runners.dataflow.worker.windmill.Windmill; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.ComputationGetDataRequest; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.GlobalDataRequest; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.KeyedGetDataRequest; +import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStreamClosedException; /** Utility data classes for {@link GrpcGetDataStream}. */ final class GrpcGetDataStreamRequests { @@ -101,6 +103,7 @@ static class QueuedBatch { private final CountDownLatch sent = new CountDownLatch(1); private long byteSize = 0; private boolean finalized = false; + private volatile boolean failed = false; CountDownLatch getLatch() { return sent; @@ -110,6 +113,23 @@ List requests() { return requests; } + /** + * Put all global data requests first because there is only a single repeated field for request + * ids and the initial ids correspond to global data requests if they are present. + */ + List sortedRequests() { + requests.sort(QueuedRequest.globalRequestsFirst()); + return requests; + } + + void validateRequests(Consumer requestValidator) { + requests.forEach(requestValidator); + } + + int requestCount() { + return requests.size(); + } + long byteSize() { return byteSize; } @@ -127,12 +147,66 @@ void addRequest(QueuedRequest request) { byteSize += request.byteSize(); } - void countDown() { + /** Let waiting for threads know that the request has been successfully sent. */ + synchronized void notifySent() { sent.countDown(); } - boolean await(long seconds) throws InterruptedException { - return sent.await(seconds, TimeUnit.SECONDS); + /** Let waiting for threads know that a failure occurred. */ + synchronized void notifyFailed() { + sent.countDown(); + failed = true; + } + + /** + * Block until notified of a successful send via {@link #notifySent()} or a non-retryable + * failure via {@link #notifyFailed()}. On failure, throw an exception to on calling threads. + */ + void waitForSendOrFailNotification() throws InterruptedException { + sent.await(); + if (failed) { + fail(); + } + } + + void fail() { + throw new WindmillStreamClosedException( + "Requests failed for batch containing " + + createStreamCancelledErrorMessage() + + " requests. This is most likely due to the stream being explicitly closed" + + " which happens when the work is marked as invalid on the streaming" + + " backend when key ranges shuffle around. This is transient and corresponding" + + " work will eventually be retried"); + } + + String createStreamCancelledErrorMessage() { + return requests.stream() + .map( + request -> { + switch (request.getDataRequest().getKind()) { + case GLOBAL: + return "GetSideInput=" + request.getDataRequest().global(); + case COMPUTATION: + return request.getDataRequest().computation().getRequestsList().stream() + .map( + keyedRequest -> + "KeyedGetState=[" + + "key=" + + keyedRequest.getKey() + + "shardingKey=" + + keyedRequest.getShardingKey() + + "cacheToken=" + + keyedRequest.getCacheToken() + + "workToken" + + keyedRequest.getWorkToken() + + "]") + .collect(Collectors.joining()); + default: + // Will never happen switch is exhaustive. + throw new IllegalStateException(); + } + }) + .collect(Collectors.joining(",")); } } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/PoolBackedHeartbeatSender.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/PoolBackedHeartbeatSender.java index bf4ea360e6146..ae582dc02fad1 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/PoolBackedHeartbeatSender.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/PoolBackedHeartbeatSender.java @@ -45,7 +45,7 @@ public void sendHeartbeats(Map> heartbea heartbeatStreamPool.getCloseableStream()) { closeableStream.stream().refreshActiveWork(heartbeats); } catch (Exception e) { - LOG.warn("Error occurred sending heartbeats.", e); + LOG.warn("Error occurred sending heartbeats=[{}].", heartbeats, e); } } } diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/BoundedQueueExecutorTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/BoundedQueueExecutorTest.java index e08c951975fa3..3a58f4945ff22 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/BoundedQueueExecutorTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/BoundedQueueExecutorTest.java @@ -66,6 +66,7 @@ private static ExecutableWork createWork(Consumer executeWorkFn) { Work.createProcessingContext( "computationId", (a, b) -> Windmill.KeyedGetDataResponse.getDefaultInstance(), + ignored -> {}, ignored -> {}), Instant::now, Collections.emptyList()), diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetDataStreamTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetDataStreamTest.java new file mode 100644 index 0000000000000..414d1e39c1f42 --- /dev/null +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetDataStreamTest.java @@ -0,0 +1,214 @@ +/* + * 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.windmill.client.grpc; + +import static com.google.common.truth.Truth.assertThat; +import static org.apache.beam.runners.dataflow.worker.windmill.client.AbstractWindmillStream.DEFAULT_STREAM_RPC_DEADLINE_SECONDS; +import static org.junit.Assert.assertThrows; +import static org.junit.Assert.assertTrue; + +import java.io.IOException; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.atomic.AtomicLong; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import javax.annotation.Nullable; +import org.apache.beam.runners.dataflow.worker.windmill.CloudWindmillServiceV1Alpha1Grpc; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill; +import org.apache.beam.runners.dataflow.worker.windmill.client.AbstractWindmillStream; +import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStreamClosedException; +import org.apache.beam.runners.dataflow.worker.windmill.client.grpc.observers.StreamObserverFactory; +import org.apache.beam.runners.dataflow.worker.windmill.client.throttling.ThrottleTimer; +import org.apache.beam.sdk.util.FluentBackoff; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.ManagedChannel; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.Server; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.inprocess.InProcessChannelBuilder; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.inprocess.InProcessServerBuilder; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.ServerCallStreamObserver; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.StreamObserver; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.testing.GrpcCleanupRule; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.util.MutableHandlerRegistry; +import org.junit.After; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.Timeout; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +@RunWith(JUnit4.class) +public class GrpcGetDataStreamTest { + private static final String FAKE_SERVER_NAME = "Fake server for GrpcGetDataStreamTest"; + private static final Windmill.JobHeader TEST_JOB_HEADER = + Windmill.JobHeader.newBuilder() + .setJobId("test_job") + .setWorkerId("test_worker") + .setProjectId("test_project") + .build(); + + @Rule public final GrpcCleanupRule grpcCleanup = new GrpcCleanupRule(); + private final MutableHandlerRegistry serviceRegistry = new MutableHandlerRegistry(); + private final Set> streamRegistry = new HashSet<>(); + @Rule public transient Timeout globalTimeout = Timeout.seconds(600); + private ManagedChannel inProcessChannel; + + @Before + public void setUp() throws IOException { + Server server = + InProcessServerBuilder.forName(FAKE_SERVER_NAME) + .fallbackHandlerRegistry(serviceRegistry) + .directExecutor() + .build() + .start(); + + inProcessChannel = + grpcCleanup.register( + InProcessChannelBuilder.forName(FAKE_SERVER_NAME).directExecutor().build()); + grpcCleanup.register(server); + grpcCleanup.register(inProcessChannel); + } + + @After + public void cleanUp() { + inProcessChannel.shutdownNow(); + } + + private GrpcGetDataStream createGetDataStream(GetDataStreamTestStub testStub) { + serviceRegistry.addService(testStub); + return GrpcGetDataStream.create( + responseObserver -> + CloudWindmillServiceV1Alpha1Grpc.newStub(inProcessChannel) + .getDataStream(responseObserver), + FluentBackoff.DEFAULT.backoff(), + StreamObserverFactory.direct(DEFAULT_STREAM_RPC_DEADLINE_SECONDS * 2, 1), + streamRegistry, + 1, + new ThrottleTimer(), + TEST_JOB_HEADER, + new AtomicLong(), + Integer.MAX_VALUE, + false, + ignored -> {}); + } + + @Test + public void testRequestKeyedData_sendOnClosedStreamThrowsWindmillStreamClosedException() { + GetDataStreamTestStub testStub = + new GetDataStreamTestStub(new TestGetDataStreamRequestObserver()); + GrpcGetDataStream getDataStream = createGetDataStream(testStub); + int numSendThreads = 5; + ExecutorService getDataStreamSenders = Executors.newFixedThreadPool(numSendThreads); + CountDownLatch waitForSendAttempt = new CountDownLatch(1); + // These will block until they are successfully sent. + List> sendFutures = + IntStream.range(0, 5) + .sequential() + .mapToObj( + i -> + (Runnable) + () -> { + // Prevent some threads from sending until we close the stream. + if (i % 2 == 0) { + try { + waitForSendAttempt.await(); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + } + getDataStream.requestKeyedData( + "computationId", + Windmill.KeyedGetDataRequest.newBuilder() + .setKey(ByteString.EMPTY) + .setShardingKey(i) + .setCacheToken(i) + .setWorkToken(i) + .build()); + }) + // Run the code above on multiple threads. + .map(runnable -> CompletableFuture.runAsync(runnable, getDataStreamSenders)) + .collect(Collectors.toList()); + + getDataStream.close(); + + // Free up waiting threads so that they can try to send on a closed stream. + waitForSendAttempt.countDown(); + + for (int i = 0; i < numSendThreads; i++) { + CompletableFuture sendFuture = sendFutures.get(i); + try { + // Wait for future to complete. + sendFuture.join(); + } catch (Exception ignored) { + } + if (i % 2 == 0) { + assertTrue(sendFuture.isCompletedExceptionally()); + ExecutionException e = assertThrows(ExecutionException.class, sendFuture::get); + assertThat(e).hasCauseThat().isInstanceOf(WindmillStreamClosedException.class); + } + } + } + + private static class TestGetDataStreamRequestObserver + implements StreamObserver { + private @Nullable StreamObserver responseObserver; + + @Override + public void onNext(Windmill.StreamingGetDataRequest streamingGetDataRequest) {} + + @Override + public void onError(Throwable throwable) {} + + @Override + public void onCompleted() { + if (responseObserver != null) { + responseObserver.onCompleted(); + } + } + } + + private static class GetDataStreamTestStub + extends CloudWindmillServiceV1Alpha1Grpc.CloudWindmillServiceV1Alpha1ImplBase { + private final TestGetDataStreamRequestObserver requestObserver; + private @Nullable StreamObserver responseObserver; + + private GetDataStreamTestStub(TestGetDataStreamRequestObserver requestObserver) { + this.requestObserver = requestObserver; + } + + @Override + public StreamObserver getDataStream( + StreamObserver responseObserver) { + if (this.responseObserver == null) { + ((ServerCallStreamObserver) responseObserver) + .setOnCancelHandler(() -> {}); + this.responseObserver = responseObserver; + requestObserver.responseObserver = this.responseObserver; + } + + return requestObserver; + } + } +} From e6d61e14d31e50738238b2b4d7d9d5675fe5ed11 Mon Sep 17 00:00:00 2001 From: Martin Trieu Date: Fri, 14 Jun 2024 17:54:38 -0700 Subject: [PATCH 04/13] add short circuit to phaser waiting when we explicitly close request observer --- .../client/AbstractWindmillStream.java | 10 +- .../grpc/observers/DirectStreamObserver.java | 21 ++-- .../StreamObserverCancelledException.java | 31 +++++ .../grpc/observers/StreamObserverFactory.java | 2 + .../observers/DirectStreamObserverTest.java | 107 ++++++++++++++++++ 5 files changed, 163 insertions(+), 8 deletions(-) create mode 100644 runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/observers/StreamObserverCancelledException.java create mode 100644 runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/observers/DirectStreamObserverTest.java diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/AbstractWindmillStream.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/AbstractWindmillStream.java index bd5827b44191a..8b9879f6c7cb8 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/AbstractWindmillStream.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/AbstractWindmillStream.java @@ -30,6 +30,7 @@ import java.util.concurrent.atomic.AtomicReference; import java.util.function.Function; import java.util.function.Supplier; +import org.apache.beam.runners.dataflow.worker.windmill.client.grpc.observers.StreamObserverCancelledException; import org.apache.beam.runners.dataflow.worker.windmill.client.grpc.observers.StreamObserverFactory; import org.apache.beam.sdk.util.BackOff; import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.Status; @@ -157,7 +158,14 @@ protected final void send(RequestT request) { throw new IllegalStateException("Send called on a client closed stream."); } - requestObserver().onNext(request); + try { + requestObserver().onNext(request); + } catch (StreamObserverCancelledException e) { + if (isClosed()) { + LOG.warn("Stream was closed during send.", e); + } + throw e; + } } } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/observers/DirectStreamObserver.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/observers/DirectStreamObserver.java index 9d57df1af3177..96beebf42814c 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/observers/DirectStreamObserver.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/observers/DirectStreamObserver.java @@ -37,7 +37,7 @@ * becomes ready. */ @ThreadSafe -public final class DirectStreamObserver implements StreamObserver { +final class DirectStreamObserver implements StreamObserver { private static final Logger LOG = LoggerFactory.getLogger(DirectStreamObserver.class); private final Phaser phaser; @@ -52,7 +52,7 @@ public final class DirectStreamObserver implements StreamObserver { @GuardedBy("lock") private int messagesSinceReady = 0; - public DirectStreamObserver( + DirectStreamObserver( Phaser phaser, CallStreamObserver outboundObserver, long deadlineSeconds, @@ -71,7 +71,7 @@ public void onNext(T value) { int awaitPhase = -1; long totalSecondsWaited = 0; long waitSeconds = 1; - while (true) { + while (phaser.getRegisteredParties() != 0) { try { synchronized (lock) { // We only check isReady periodically to effectively allow for increasing the outbound @@ -99,6 +99,12 @@ public void onNext(T value) { // documentation stating otherwise) so we poll periodically and enforce an overall // timeout related to the stream deadline. phaser.awaitAdvanceInterruptibly(awaitPhase, waitSeconds, TimeUnit.SECONDS); + // Check to see if onCompleted has been called and the stream observer was closed before + // trying to send the next value. + if (phaser.getRegisteredParties() == 0) { + return; + } + synchronized (lock) { messagesSinceReady = 0; outboundObserver.onNext(value); @@ -107,10 +113,10 @@ public void onNext(T value) { } catch (TimeoutException e) { totalSecondsWaited += waitSeconds; if (totalSecondsWaited > deadlineSeconds) { - LOG.error( + throw new StreamObserverCancelledException( "Exceeded timeout waiting for the outboundObserver to become ready meaning " - + "that the stream deadline was not respected."); - throw new RuntimeException(e); + + "that the stream deadline was not respected.", + e); } if (totalSecondsWaited > 30) { LOG.info( @@ -121,7 +127,7 @@ public void onNext(T value) { waitSeconds = waitSeconds * 2; } catch (InterruptedException e) { Thread.currentThread().interrupt(); - throw new RuntimeException(e); + throw new StreamObserverCancelledException(e); } } } @@ -136,6 +142,7 @@ public void onError(Throwable t) { @Override public void onCompleted() { synchronized (lock) { + phaser.arriveAndDeregister(); outboundObserver.onCompleted(); } } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/observers/StreamObserverCancelledException.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/observers/StreamObserverCancelledException.java new file mode 100644 index 0000000000000..4ea209f31b1d9 --- /dev/null +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/observers/StreamObserverCancelledException.java @@ -0,0 +1,31 @@ +/* + * 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.windmill.client.grpc.observers; + +import org.apache.beam.sdk.annotations.Internal; + +@Internal +public final class StreamObserverCancelledException extends RuntimeException { + public StreamObserverCancelledException(Throwable cause) { + super(cause); + } + + public StreamObserverCancelledException(String message, Throwable cause) { + super(message, cause); + } +} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/observers/StreamObserverFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/observers/StreamObserverFactory.java index cb4415bdab181..1afc9c34d28b1 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/observers/StreamObserverFactory.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/observers/StreamObserverFactory.java @@ -18,6 +18,7 @@ package org.apache.beam.runners.dataflow.worker.windmill.client.grpc.observers; import java.util.function.Function; +import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.sdk.fn.stream.AdvancingPhaser; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.CallStreamObserver; @@ -27,6 +28,7 @@ * Uses {@link PipelineOptions} to configure which underlying {@link StreamObserver} implementation * to use. */ +@Internal public abstract class StreamObserverFactory { public static StreamObserverFactory direct( long deadlineSeconds, int messagesBetweenIsReadyChecks) { diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/observers/DirectStreamObserverTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/observers/DirectStreamObserverTest.java new file mode 100644 index 0000000000000..ebf18d5315212 --- /dev/null +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/observers/DirectStreamObserverTest.java @@ -0,0 +1,107 @@ +/* + * 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.windmill.client.grpc.observers; + +import static com.google.common.truth.Truth.assertThat; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertThrows; + +import java.util.HashSet; +import java.util.Set; +import org.apache.beam.sdk.fn.stream.AdvancingPhaser; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.CallStreamObserver; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +@RunWith(JUnit4.class) +public class DirectStreamObserverTest { + private final Set sentOnStream = new HashSet<>(); + + private DirectStreamObserver newStreamObserver(long deadlineSeconds) { + return new DirectStreamObserver<>( + new AdvancingPhaser(1), + new CallStreamObserver() { + @Override + public boolean isReady() { + return false; + } + + @Override + public void setOnReadyHandler(Runnable runnable) {} + + @Override + public void disableAutoInboundFlowControl() {} + + @Override + public void request(int i) {} + + @Override + public void setMessageCompression(boolean b) {} + + @Override + public void onNext(Integer integer) { + sentOnStream.add(integer); + } + + @Override + public void onError(Throwable throwable) {} + + @Override + public void onCompleted() {} + }, + deadlineSeconds, + 1); + } + + @Test + public void testOnCompleted_shutsdownStreamObserverGracefully() throws InterruptedException { + DirectStreamObserver streamObserver = newStreamObserver(30); + streamObserver.onNext(1); + Thread onNextWaiting = new Thread(() -> streamObserver.onNext(10)); + onNextWaiting.start(); + streamObserver.onCompleted(); + onNextWaiting.join(); + assertFalse(sentOnStream.contains(10)); + assertThat(sentOnStream).containsExactly(1); + } + + @Test + public void testOnNext_timeoutThrowsStreamObserverCancelledException() { + DirectStreamObserver streamObserver = newStreamObserver(5); + streamObserver.onNext(1); + assertThrows(StreamObserverCancelledException.class, () -> streamObserver.onNext(10)); + assertFalse(sentOnStream.contains(10)); + assertThat(sentOnStream).containsExactly(1); + } + + @Test + public void testOnNext_interruptedThrowsStreamObserverCancelledException() { + DirectStreamObserver streamObserver = newStreamObserver(30); + streamObserver.onNext(1); + Thread onNextWaiting = + new Thread( + () -> + assertThrows( + StreamObserverCancelledException.class, () -> streamObserver.onNext(10))); + onNextWaiting.start(); + onNextWaiting.interrupt(); + assertFalse(sentOnStream.contains(10)); + assertThat(sentOnStream).containsExactly(1); + } +} From ff09734f447df9667c06076e5f628cca3348a725 Mon Sep 17 00:00:00 2001 From: Martin Trieu Date: Mon, 17 Jun 2024 22:10:07 -0700 Subject: [PATCH 05/13] add readonly refreshable view of Work --- .../worker/streaming/ActiveWorkState.java | 44 ++---------- .../worker/streaming/ComputationState.java | 13 +--- .../dataflow/worker/streaming/Work.java | 58 ++++++++++++++- .../client/AbstractWindmillStream.java | 5 +- .../grpc/observers/DirectStreamObserver.java | 12 ++-- .../work/refresh/ActiveWorkRefresher.java | 2 +- .../work/refresh/HeartbeatRequests.java | 34 ++++----- .../worker/streaming/ActiveWorkStateTest.java | 71 ------------------- .../observers/DirectStreamObserverTest.java | 15 ++-- 9 files changed, 102 insertions(+), 152 deletions(-) diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/ActiveWorkState.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/ActiveWorkState.java index 6cc55b83b04f2..5a3ea4f6e0230 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/ActiveWorkState.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/ActiveWorkState.java @@ -17,7 +17,6 @@ */ package org.apache.beam.runners.dataflow.worker.streaming; -import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList.toImmutableList; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableListMultimap.flatteningToImmutableListMultimap; import java.io.PrintWriter; @@ -32,20 +31,16 @@ import java.util.Queue; import java.util.concurrent.atomic.AtomicReference; import java.util.function.BiConsumer; -import java.util.stream.Stream; import javax.annotation.Nullable; import javax.annotation.concurrent.GuardedBy; import javax.annotation.concurrent.ThreadSafe; import org.apache.beam.runners.dataflow.worker.DataflowExecutionStateSampler; -import org.apache.beam.runners.dataflow.worker.windmill.Windmill; -import org.apache.beam.runners.dataflow.worker.windmill.Windmill.HeartbeatRequest; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.WorkItem; import org.apache.beam.runners.dataflow.worker.windmill.state.WindmillStateCache; import org.apache.beam.runners.dataflow.worker.windmill.work.budget.GetWorkBudget; import org.apache.beam.sdk.annotations.Internal; 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.collect.ImmutableList; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableListMultimap; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Multimap; @@ -108,29 +103,6 @@ private static String elapsedString(Instant start, Instant end) { return activeFor.toString().substring(2); } - private static Stream toHeartbeatRequestStream( - Entry> shardedKeyAndWorkQueue, - Instant refreshDeadline, - DataflowExecutionStateSampler sampler) { - ShardedKey shardedKey = shardedKeyAndWorkQueue.getKey(); - Deque workQueue = shardedKeyAndWorkQueue.getValue(); - - return workQueue.stream() - .map(ExecutableWork::work) - .filter(work -> work.getStartTime().isBefore(refreshDeadline)) - // Don't send heartbeats for queued work we already know is failed. - .filter(work -> !work.isFailed()) - .map( - work -> - Windmill.HeartbeatRequest.newBuilder() - .setShardingKey(shardedKey.shardingKey()) - .setWorkToken(work.getWorkItem().getWorkToken()) - .setCacheToken(work.getWorkItem().getCacheToken()) - .addAllLatencyAttribution( - work.getLatencyAttributions(/* isHeartbeat= */ true, sampler)) - .build()); - } - /** * Activates {@link Work} for the {@link ShardedKey}. Outcome can be 1 of 4 {@link * ActivateWorkResult} @@ -326,13 +298,6 @@ private synchronized ImmutableMap getStuckCommitsAt( return stuckCommits.build(); } - synchronized ImmutableList getKeyHeartbeats( - Instant refreshDeadline, DataflowExecutionStateSampler sampler) { - return activeWork.entrySet().stream() - .flatMap(entry -> toHeartbeatRequestStream(entry, refreshDeadline, sampler)) - .collect(toImmutableList()); - } - /** * Returns a read only view of current active work. * @@ -340,11 +305,16 @@ synchronized ImmutableList getKeyHeartbeats( * cause a {@link java.util.ConcurrentModificationException} as it is not a thread-safe data * structure. */ - synchronized ImmutableListMultimap getReadOnlyActiveWork() { + synchronized ImmutableListMultimap getReadOnlyActiveWork( + DataflowExecutionStateSampler sampler) { return activeWork.entrySet().stream() .collect( flatteningToImmutableListMultimap( - Entry::getKey, e -> e.getValue().stream().map(ExecutableWork::work))); + Entry::getKey, + e -> + e.getValue().stream() + .map(ExecutableWork::work) + .map(work -> work.refreshableView(sampler)))); } /** diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/ComputationState.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/ComputationState.java index 594c629b5fb73..e2ce5e1aef798 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/ComputationState.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/ComputationState.java @@ -25,12 +25,10 @@ import javax.annotation.Nullable; import org.apache.beam.runners.dataflow.worker.DataflowExecutionStateSampler; import org.apache.beam.runners.dataflow.worker.util.BoundedQueueExecutor; -import org.apache.beam.runners.dataflow.worker.windmill.Windmill.HeartbeatRequest; import org.apache.beam.runners.dataflow.worker.windmill.state.WindmillStateCache; import org.apache.beam.runners.dataflow.worker.windmill.work.budget.GetWorkBudget; 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.collect.ImmutableList; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableListMultimap; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Multimap; @@ -140,8 +138,9 @@ public void invalidateStuckCommits(Instant stuckCommitDeadline) { stuckCommitDeadline, this::completeWorkAndScheduleNextWorkForKey); } - public ImmutableListMultimap currentActiveWorkReadOnly() { - return activeWorkState.getReadOnlyActiveWork(); + public ImmutableListMultimap currentActiveWorkReadOnly( + DataflowExecutionStateSampler sampler) { + return activeWorkState.getReadOnlyActiveWork(sampler); } private void execute(ExecutableWork executableWork) { @@ -152,12 +151,6 @@ private void forceExecute(ExecutableWork executableWork) { executor.forceExecute(executableWork, executableWork.work().getWorkItem().getSerializedSize()); } - /** Gets HeartbeatRequests for any work started before refreshDeadline. */ - public ImmutableList getKeyHeartbeats( - Instant refreshDeadline, DataflowExecutionStateSampler sampler) { - return activeWorkState.getKeyHeartbeats(refreshDeadline, sampler); - } - public GetWorkBudget getActiveWorkBudget() { return activeWorkState.currentActiveWorkBudget(); } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/Work.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/Work.java index f16d6dd2c65fa..ade16a98fb00d 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/Work.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/Work.java @@ -182,7 +182,7 @@ public void setState(State state) { this.currentState = TimedState.create(state, now); } - public boolean isRefreshable(Instant refreshDeadline) { + private boolean isRefreshable(Instant refreshDeadline) { boolean isRefreshable = getStartTime().isBefore(refreshDeadline); if (heartbeatSender().isInvalid()) { setFailed(); @@ -281,6 +281,17 @@ boolean isStuckCommittingAt(Instant stuckCommitDeadline) { && currentState.startTime().isBefore(stuckCommitDeadline); } + /** Returns a read-only snapshot of this {@link Work} instance's state for work refreshing. */ + RefreshableView refreshableView(DataflowExecutionStateSampler sampler) { + return RefreshableView.builder() + .setWorkId(id) + .setHeartbeatSender(heartbeatSender()) + .setIsFailed(isFailed) + .setIsRefreshable(this::isRefreshable) + .setLatencyAttributions(getLatencyAttributions(/* isHeartbeat= */ true, sampler)) + .build(); + } + public enum State { QUEUED(LatencyAttribution.State.QUEUED), PROCESSING(LatencyAttribution.State.ACTIVE), @@ -355,4 +366,49 @@ private static ProcessingContext create( public abstract HeartbeatSender heartbeatSender(); } + + @AutoValue + public abstract static class RefreshableView { + + private static RefreshableView.Builder builder() { + return new AutoValue_Work_RefreshableView.Builder(); + } + + abstract WorkId workId(); + + public final long workToken() { + return workId().workToken(); + } + + public final long cacheToken() { + return workId().cacheToken(); + } + + abstract Function isRefreshable(); + + public final boolean isRefreshable(Instant refreshDeadline) { + return isRefreshable().apply(refreshDeadline); + } + + public abstract HeartbeatSender heartbeatSender(); + + public abstract boolean isFailed(); + + public abstract ImmutableList latencyAttributions(); + + @AutoValue.Builder + abstract static class Builder { + abstract Builder setWorkId(WorkId value); + + abstract Builder setIsRefreshable(Function value); + + abstract Builder setHeartbeatSender(HeartbeatSender value); + + abstract Builder setIsFailed(boolean value); + + abstract Builder setLatencyAttributions(ImmutableList value); + + abstract RefreshableView build(); + } + } } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/AbstractWindmillStream.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/AbstractWindmillStream.java index 8b9879f6c7cb8..7c4380db8bc60 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/AbstractWindmillStream.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/AbstractWindmillStream.java @@ -97,7 +97,7 @@ protected AbstractWindmillStream( Executors.newSingleThreadExecutor( new ThreadFactoryBuilder() .setDaemon(true) - .setNameFormat("WindmillStream-thread") + .setNameFormat("WindmillStream-" + getClass() + "-thread") .build()); this.backoff = backoff; this.streamRegistry = streamRegistry; @@ -305,10 +305,11 @@ private void onStreamFinished(@Nullable Throwable t) { } } if (t != null) { - Status status = null; + Status status = Status.fromThrowable(t); if (t instanceof StatusRuntimeException) { status = ((StatusRuntimeException) t).getStatus(); } + String statusError = status == null ? "" : status.toString(); setLastError(statusError); if (errorCount.getAndIncrement() % logEveryNStreamFailures == 0) { diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/observers/DirectStreamObserver.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/observers/DirectStreamObserver.java index 96beebf42814c..3aeff166ea491 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/observers/DirectStreamObserver.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/observers/DirectStreamObserver.java @@ -71,7 +71,7 @@ public void onNext(T value) { int awaitPhase = -1; long totalSecondsWaited = 0; long waitSeconds = 1; - while (phaser.getRegisteredParties() != 0) { + while (!isTerminated()) { try { synchronized (lock) { // We only check isReady periodically to effectively allow for increasing the outbound @@ -99,9 +99,8 @@ public void onNext(T value) { // documentation stating otherwise) so we poll periodically and enforce an overall // timeout related to the stream deadline. phaser.awaitAdvanceInterruptibly(awaitPhase, waitSeconds, TimeUnit.SECONDS); - // Check to see if onCompleted has been called and the stream observer was closed before - // trying to send the next value. - if (phaser.getRegisteredParties() == 0) { + // Exit early if the phaser was terminated. + if (isTerminated()) { return; } @@ -132,9 +131,14 @@ public void onNext(T value) { } } + private boolean isTerminated() { + return phaser.isTerminated() || phaser.getRegisteredParties() == 0; + } + @Override public void onError(Throwable t) { synchronized (lock) { + phaser.arriveAndDeregister(); outboundObserver.onError(t); } } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/ActiveWorkRefresher.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/ActiveWorkRefresher.java index 9e235fa4e75c3..cdb6844479287 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/ActiveWorkRefresher.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/ActiveWorkRefresher.java @@ -127,7 +127,7 @@ private void refreshActiveWork() { // to correctly fan-out the heartbeat requests. ImmutableListMultimap heartbeats = HeartbeatRequests.getRefreshableKeyHeartbeats( - computationState.currentActiveWorkReadOnly(), refreshDeadline, sampler); + computationState.currentActiveWorkReadOnly(sampler), refreshDeadline); // Aggregate the heartbeats across computations by GetDataStream for correct fan out. for (Map.Entry> heartbeatsPerStream : heartbeats.asMap().entrySet()) { diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/HeartbeatRequests.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/HeartbeatRequests.java index 78a7c24bb3a27..3ef2259fa1a3c 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/HeartbeatRequests.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/HeartbeatRequests.java @@ -23,7 +23,6 @@ import java.util.Map; import java.util.stream.Stream; import org.apache.beam.repackaged.core.org.apache.commons.lang3.tuple.Pair; -import org.apache.beam.runners.dataflow.worker.DataflowExecutionStateSampler; import org.apache.beam.runners.dataflow.worker.streaming.ShardedKey; import org.apache.beam.runners.dataflow.worker.streaming.Work; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.HeartbeatRequest; @@ -38,40 +37,31 @@ public final class HeartbeatRequests { private HeartbeatRequests() {} static ImmutableListMultimap getRefreshableKeyHeartbeats( - ImmutableListMultimap activeWork, - Instant refreshDeadline, - DataflowExecutionStateSampler sampler) { + ImmutableListMultimap activeWork, Instant refreshDeadline) { return activeWork.asMap().entrySet().stream() - .flatMap(e -> toHeartbeatRequest(e, refreshDeadline, sampler)) + .flatMap(e -> toHeartbeatRequest(e, refreshDeadline)) .collect(toImmutableListMultimap(Pair::getKey, Pair::getValue)); } private static Stream> toHeartbeatRequest( - Map.Entry> shardedKeyAndWorkQueue, - Instant refreshDeadline, - DataflowExecutionStateSampler sampler) { + Map.Entry> shardedKeyAndWorkQueue, + Instant refreshDeadline) { ShardedKey shardedKey = shardedKeyAndWorkQueue.getKey(); - Collection workQueue = shardedKeyAndWorkQueue.getValue(); - return getRefreshableWork(workQueue, refreshDeadline) + Collection workQueue = shardedKeyAndWorkQueue.getValue(); + return workQueue.stream() + .filter(work -> work.isRefreshable(refreshDeadline)) // Don't send heartbeats for queued work we already know is failed. .filter(work -> !work.isFailed()) - .map( - work -> - Pair.of(work.heartbeatSender(), createHeartbeatRequest(shardedKey, work, sampler))); + .map(work -> Pair.of(work.heartbeatSender(), createHeartbeatRequest(shardedKey, work))); } private static HeartbeatRequest createHeartbeatRequest( - ShardedKey shardedKey, Work work, DataflowExecutionStateSampler sampler) { + ShardedKey shardedKey, Work.RefreshableView work) { return HeartbeatRequest.newBuilder() .setShardingKey(shardedKey.shardingKey()) - .setWorkToken(work.getWorkItem().getWorkToken()) - .setCacheToken(work.getWorkItem().getCacheToken()) - .addAllLatencyAttribution(work.getLatencyAttributions(/* isHeartbeat= */ true, sampler)) + .setWorkToken(work.workToken()) + .setCacheToken(work.cacheToken()) + .addAllLatencyAttribution(work.latencyAttributions()) .build(); } - - private static Stream getRefreshableWork( - Collection workQueue, Instant refreshDeadline) { - return workQueue.stream().filter(work -> work.isRefreshable(refreshDeadline)); - } } diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/ActiveWorkStateTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/ActiveWorkStateTest.java index 1ba79e0250d87..58939b7f45201 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/ActiveWorkStateTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/ActiveWorkStateTest.java @@ -18,7 +18,6 @@ package org.apache.beam.runners.dataflow.worker.streaming; import static com.google.common.truth.Truth.assertThat; -import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList.toImmutableList; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertSame; @@ -26,20 +25,16 @@ import static org.mockito.Mockito.mock; import static org.mockito.Mockito.verify; -import com.google.auto.value.AutoValue; import java.util.Collections; import java.util.Deque; import java.util.HashMap; import java.util.Map; import java.util.Optional; -import org.apache.beam.runners.dataflow.worker.DataflowExecutionStateSampler; import org.apache.beam.runners.dataflow.worker.streaming.ActiveWorkState.ActivateWorkResult; import org.apache.beam.runners.dataflow.worker.windmill.Windmill; -import org.apache.beam.runners.dataflow.worker.windmill.Windmill.HeartbeatRequest; import org.apache.beam.runners.dataflow.worker.windmill.state.WindmillStateCache; import org.apache.beam.runners.dataflow.worker.windmill.work.budget.GetWorkBudget; import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.joda.time.Instant; import org.junit.Before; import org.junit.Rule; @@ -448,70 +443,4 @@ public void testActivateWorkForKey_matchingCacheTokens_newWorkTokenLesser_STALE( assertFalse(readOnlyActiveWork.get(shardedKey).contains(newWork)); assertEquals(queuedWork, readOnlyActiveWork.get(shardedKey).peek()); } - - @Test - public void testGetKeyHeartbeats() { - Instant refreshDeadline = Instant.now(); - ShardedKey shardedKey1 = shardedKey("someKey", 1L); - ShardedKey shardedKey2 = shardedKey("anotherKey", 2L); - - ExecutableWork freshWork = createWork(createWorkItem(3L, 3L, shardedKey1)); - ExecutableWork refreshableWork1 = expiredWork(createWorkItem(1L, 1L, shardedKey1)); - refreshableWork1.work().setState(Work.State.COMMITTING); - ExecutableWork refreshableWork2 = expiredWork(createWorkItem(2L, 2L, shardedKey2)); - refreshableWork2.work().setState(Work.State.COMMITTING); - - activeWorkState.activateWorkForKey(refreshableWork1); - activeWorkState.activateWorkForKey(freshWork); - activeWorkState.activateWorkForKey(refreshableWork2); - - ImmutableList requests = - activeWorkState.getKeyHeartbeats(refreshDeadline, DataflowExecutionStateSampler.instance()); - - ImmutableList expected = - ImmutableList.of( - HeartbeatRequestShardingKeyWorkTokenAndCacheToken.from( - shardedKey1, refreshableWork1.work()), - HeartbeatRequestShardingKeyWorkTokenAndCacheToken.from( - shardedKey2, refreshableWork2.work())); - - ImmutableList actual = - requests.stream() - .map(HeartbeatRequestShardingKeyWorkTokenAndCacheToken::from) - .collect(toImmutableList()); - - assertThat(actual).containsExactlyElementsIn(expected); - } - - @AutoValue - abstract static class HeartbeatRequestShardingKeyWorkTokenAndCacheToken { - - private static HeartbeatRequestShardingKeyWorkTokenAndCacheToken create( - long shardingKey, long workToken, long cacheToken) { - return new AutoValue_ActiveWorkStateTest_HeartbeatRequestShardingKeyWorkTokenAndCacheToken( - shardingKey, workToken, cacheToken); - } - - private static HeartbeatRequestShardingKeyWorkTokenAndCacheToken from( - HeartbeatRequest heartbeatRequest) { - return create( - heartbeatRequest.getShardingKey(), - heartbeatRequest.getWorkToken(), - heartbeatRequest.getCacheToken()); - } - - private static HeartbeatRequestShardingKeyWorkTokenAndCacheToken from( - ShardedKey shardedKey, Work work) { - return create( - shardedKey.shardingKey(), - work.getWorkItem().getWorkToken(), - work.getWorkItem().getCacheToken()); - } - - abstract long shardingKey(); - - abstract long workToken(); - - abstract long cacheToken(); - } } diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/observers/DirectStreamObserverTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/observers/DirectStreamObserverTest.java index ebf18d5315212..3e4474cf6eb93 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/observers/DirectStreamObserverTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/observers/DirectStreamObserverTest.java @@ -34,8 +34,9 @@ public class DirectStreamObserverTest { private final Set sentOnStream = new HashSet<>(); private DirectStreamObserver newStreamObserver(long deadlineSeconds) { + AdvancingPhaser phaser = new AdvancingPhaser(1); return new DirectStreamObserver<>( - new AdvancingPhaser(1), + phaser, new CallStreamObserver() { @Override public boolean isReady() { @@ -43,7 +44,9 @@ public boolean isReady() { } @Override - public void setOnReadyHandler(Runnable runnable) {} + public void setOnReadyHandler(Runnable runnable) { + phaser.arrive(); + } @Override public void disableAutoInboundFlowControl() {} @@ -60,10 +63,14 @@ public void onNext(Integer integer) { } @Override - public void onError(Throwable throwable) {} + public void onError(Throwable throwable) { + phaser.forceTermination(); + } @Override - public void onCompleted() {} + public void onCompleted() { + phaser.forceTermination(); + } }, deadlineSeconds, 1); From e7b544cc03a950b8cd89d77615f4f024374407ac Mon Sep 17 00:00:00 2001 From: Martin Trieu Date: Tue, 18 Jun 2024 00:41:09 -0700 Subject: [PATCH 06/13] address PR comments --- .../MetricTrackingWindmillServerStub.java | 18 ++- .../worker/WorkItemCancelledException.java | 4 + .../worker/streaming/ActiveWorkState.java | 21 +--- .../worker/streaming/ComputationState.java | 6 +- .../worker/streaming/RefreshableWork.java | 17 ++- .../dataflow/worker/streaming/Work.java | 110 ++++++++---------- .../dataflow/worker/streaming/WorkId.java | 4 +- .../client/WindmillStreamClosedException.java | 2 +- .../commits/StreamingEngineWorkCommitter.java | 2 +- .../client/grpc/GrpcDirectGetWorkStream.java | 49 ++++---- .../client/grpc/GrpcGetDataStream.java | 3 + .../grpc/GrpcGetDataStreamRequests.java | 24 ---- .../work/refresh/ActiveWorkRefresher.java | 2 +- .../work/refresh/DirectHeartbeatSender.java | 38 +++--- .../work/refresh/HeartbeatRequests.java | 30 ++--- .../work/refresh/HeartbeatSender.java | 4 - .../worker/WorkerCustomSourcesTest.java | 2 +- .../StreamingEngineWorkCommitterTest.java | 2 +- .../work/refresh/HeartbeatRequestsTest.java | 12 +- 19 files changed, 170 insertions(+), 180 deletions(-) diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/MetricTrackingWindmillServerStub.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/MetricTrackingWindmillServerStub.java index fa244a6755d27..aea6fddc4ddfc 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/MetricTrackingWindmillServerStub.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/MetricTrackingWindmillServerStub.java @@ -244,7 +244,7 @@ public Windmill.KeyedGetDataResponse getStateData( activeStateReads.getAndIncrement(); return getDataStream.requestKeyedData(computation, request); } catch (Exception e) { - if (WindmillStreamClosedException.isWindmillStreamCancelledException(e)) { + if (WindmillStreamClosedException.wasCauseOf(e)) { LOG.error("Tried to fetch keyed data from a closed stream. Work has been cancelled", e); throw new WorkItemCancelledException(request.getShardingKey()); } @@ -285,16 +285,16 @@ public Windmill.GlobalData getSideInputData( try { return getDataStream.requestGlobalData(request); } catch (Exception e) { + if (WindmillStreamClosedException.wasCauseOf(e)) { + LOG.error("Tried to fetch global data from a closed stream. Work has been cancelled", e); + throw new WorkItemCancelledException("Failed to get side input.", e); + } throw new RuntimeException("Failed to get side input: ", e); } finally { activeSideInputs.getAndDecrement(); } } - public WindmillStreamPool getGetDataStreamPool() { - return getDataStreamPool; - } - /** * Attempts to refresh active work, fanning out to each {@link GetDataStream} in parallel. * @@ -307,16 +307,14 @@ public void refreshActiveWork( } try { - // There is 1 destination to send heartbeat requests. if (heartbeats.size() == 1) { + // There is 1 destination to send heartbeat requests. Map.Entry>> heartbeat = Iterables.getOnlyElement(heartbeats.entrySet()); HeartbeatSender sender = heartbeat.getKey(); sender.sendHeartbeats(heartbeat.getValue()); - } - - // There are multiple destinations to send heartbeat requests. Fan out requests in parallel. - else { + } else { + // There are multiple destinations to send heartbeat requests. Fan out requests in parallel. refreshActiveWorkWithFanOut(heartbeats); } } finally { diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WorkItemCancelledException.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WorkItemCancelledException.java index 8b33aa1823d47..f992ad714346e 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WorkItemCancelledException.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WorkItemCancelledException.java @@ -25,6 +25,10 @@ public WorkItemCancelledException(long shardingKey) { super("Work item cancelled for key " + shardingKey); } + public WorkItemCancelledException(String message, Throwable t) { + super(message, t); + } + /** Returns whether an exception was caused by a {@link WorkItemCancelledException}. */ public static boolean isWorkItemCancelledException(@Nullable Throwable t) { while (t != null) { diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/ActiveWorkState.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/ActiveWorkState.java index 5a3ea4f6e0230..39454c7d329d2 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/ActiveWorkState.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/ActiveWorkState.java @@ -34,7 +34,6 @@ import javax.annotation.Nullable; import javax.annotation.concurrent.GuardedBy; import javax.annotation.concurrent.ThreadSafe; -import org.apache.beam.runners.dataflow.worker.DataflowExecutionStateSampler; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.WorkItem; import org.apache.beam.runners.dataflow.worker.windmill.state.WindmillStateCache; import org.apache.beam.runners.dataflow.worker.windmill.work.budget.GetWorkBudget; @@ -175,17 +174,7 @@ synchronized void failWorkForKey(Multimap failedWork) { WorkItem workItem = queuedWork.work().getWorkItem(); if (workItem.getWorkToken() == failedWorkId.workToken() && workItem.getCacheToken() == failedWorkId.cacheToken()) { - LOG.debug( - "Failing work " - + computationStateCache.getComputation() - + " " - + entry.getKey().shardingKey() - + " " - + failedWorkId.workToken() - + " " - + failedWorkId.cacheToken() - + ". The work will be retried and is not lost."); - queuedWork.work().setFailed(); + queuedWork.work().fail(); break; } } @@ -305,16 +294,12 @@ private synchronized ImmutableMap getStuckCommitsAt( * cause a {@link java.util.ConcurrentModificationException} as it is not a thread-safe data * structure. */ - synchronized ImmutableListMultimap getReadOnlyActiveWork( - DataflowExecutionStateSampler sampler) { + synchronized ImmutableListMultimap getReadOnlyActiveWork() { return activeWork.entrySet().stream() .collect( flatteningToImmutableListMultimap( Entry::getKey, - e -> - e.getValue().stream() - .map(ExecutableWork::work) - .map(work -> work.refreshableView(sampler)))); + e -> e.getValue().stream().map(ExecutableWork::work).map(Work::refreshableView))); } /** diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/ComputationState.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/ComputationState.java index e2ce5e1aef798..4e4a78803823f 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/ComputationState.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/ComputationState.java @@ -23,7 +23,6 @@ import java.util.Optional; import java.util.concurrent.ConcurrentLinkedQueue; import javax.annotation.Nullable; -import org.apache.beam.runners.dataflow.worker.DataflowExecutionStateSampler; import org.apache.beam.runners.dataflow.worker.util.BoundedQueueExecutor; import org.apache.beam.runners.dataflow.worker.windmill.state.WindmillStateCache; import org.apache.beam.runners.dataflow.worker.windmill.work.budget.GetWorkBudget; @@ -138,9 +137,8 @@ public void invalidateStuckCommits(Instant stuckCommitDeadline) { stuckCommitDeadline, this::completeWorkAndScheduleNextWorkForKey); } - public ImmutableListMultimap currentActiveWorkReadOnly( - DataflowExecutionStateSampler sampler) { - return activeWorkState.getReadOnlyActiveWork(sampler); + public ImmutableListMultimap currentActiveWorkReadOnly() { + return activeWorkState.getReadOnlyActiveWork(); } private void execute(ExecutableWork executableWork) { diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/RefreshableWork.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/RefreshableWork.java index 91c4ba9c82fda..1d821990fb86f 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/RefreshableWork.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/RefreshableWork.java @@ -17,8 +17,23 @@ */ package org.apache.beam.runners.dataflow.worker.streaming; +import org.apache.beam.runners.dataflow.worker.DataflowExecutionStateSampler; import org.apache.beam.runners.dataflow.worker.windmill.Windmill; +import org.apache.beam.runners.dataflow.worker.windmill.work.refresh.HeartbeatSender; +import org.apache.beam.sdk.annotations.Internal; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; +import org.joda.time.Instant; +/** View of {@link Work} that exposes an interface for work refreshing. */ +@Internal public interface RefreshableWork { - Windmill.WorkItem getWorkItem(); + + WorkId id(); + + boolean isRefreshable(Instant refreshDeadline); + + HeartbeatSender heartbeatSender(); + + ImmutableList getLatencyAttributions( + boolean isHeartbeat, DataflowExecutionStateSampler sampler); } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/Work.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/Work.java index ade16a98fb00d..f187c41676b3f 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/Work.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/Work.java @@ -46,11 +46,14 @@ import org.apache.beam.runners.dataflow.worker.windmill.client.commits.Commit; import org.apache.beam.runners.dataflow.worker.windmill.client.commits.WorkCommitter; import org.apache.beam.runners.dataflow.worker.windmill.state.WindmillStateReader; +import org.apache.beam.runners.dataflow.worker.windmill.work.refresh.DirectHeartbeatSender; import org.apache.beam.runners.dataflow.worker.windmill.work.refresh.HeartbeatSender; import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.joda.time.Duration; import org.joda.time.Instant; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Represents the state of an attempt to process a {@link WorkItem} by executing user code. @@ -59,7 +62,9 @@ */ @NotThreadSafe @Internal -public final class Work { +public final class Work implements RefreshableWork { + private static final Logger LOG = LoggerFactory.getLogger(Work.class); + private final ShardedKey shardedKey; private final WorkItem workItem; private final ProcessingContext processingContext; @@ -79,7 +84,6 @@ private Work( Supplier clock) { this.shardedKey = ShardedKey.create(workItem.getKey(), workItem.getShardingKey()); this.workItem = workItem; - this.processingContext = processingContext; this.watermarks = watermarks; this.clock = clock; this.startTime = clock.get(); @@ -91,6 +95,15 @@ private Work( + Long.toHexString(workItem.getWorkToken()); this.currentState = TimedState.initialState(startTime); this.isFailed = false; + this.processingContext = + processingContext.heartbeatSender() instanceof DirectHeartbeatSender + ? processingContext + .toBuilder() + .setHeartbeatSender( + ((DirectHeartbeatSender) processingContext.heartbeatSender()) + .withStreamClosedHandler(() -> isFailed = true)) + .build() + : processingContext; } public static Work create( @@ -182,21 +195,27 @@ public void setState(State state) { this.currentState = TimedState.create(state, now); } - private boolean isRefreshable(Instant refreshDeadline) { - boolean isRefreshable = getStartTime().isBefore(refreshDeadline); - if (heartbeatSender().isInvalid()) { - setFailed(); - return false; - } - - return isRefreshable; + @Override + public boolean isRefreshable(Instant refreshDeadline) { + return getStartTime().isBefore(refreshDeadline) && !isFailed; } + @Override public HeartbeatSender heartbeatSender() { return processingContext.heartbeatSender(); } - public void setFailed() { + public void fail() { + LOG.debug( + "Failing work " + + processingContext.computationId() + + " " + + shardedKey + + " " + + id.workToken() + + " " + + id.cacheToken() + + ". The work will be retried and is not lost."); this.isFailed = true; } @@ -221,6 +240,7 @@ public WindmillStateReader createWindmillStateReader() { return WindmillStateReader.forWork(this); } + @Override public WorkId id() { return id; } @@ -232,6 +252,7 @@ private void recordGetWorkStreamLatencies(Collection getWork } } + @Override public ImmutableList getLatencyAttributions( boolean isHeartbeat, DataflowExecutionStateSampler sampler) { return Arrays.stream(LatencyAttribution.State.values()) @@ -281,15 +302,9 @@ boolean isStuckCommittingAt(Instant stuckCommitDeadline) { && currentState.startTime().isBefore(stuckCommitDeadline); } - /** Returns a read-only snapshot of this {@link Work} instance's state for work refreshing. */ - RefreshableView refreshableView(DataflowExecutionStateSampler sampler) { - return RefreshableView.builder() - .setWorkId(id) - .setHeartbeatSender(heartbeatSender()) - .setIsFailed(isFailed) - .setIsRefreshable(this::isRefreshable) - .setLatencyAttributions(getLatencyAttributions(/* isHeartbeat= */ true, sampler)) - .build(); + /** Returns a view of this {@link Work} instance for work refreshing. */ + public RefreshableWork refreshableView() { + return this; } public enum State { @@ -344,11 +359,13 @@ private static ProcessingContext create( BiFunction getKeyedDataFn, Consumer workCommitter, HeartbeatSender heartbeatSender) { - return new AutoValue_Work_ProcessingContext( - computationId, - request -> Optional.ofNullable(getKeyedDataFn.apply(computationId, request)), - workCommitter, - heartbeatSender); + return new AutoValue_Work_ProcessingContext.Builder() + .setComputationId(computationId) + .setHeartbeatSender(heartbeatSender) + .setWorkCommitter(workCommitter) + .setKeyedDataFetcher( + request -> Optional.ofNullable(getKeyedDataFn.apply(computationId, request))) + .build(); } /** Computation that the {@link Work} belongs to. */ @@ -365,50 +382,21 @@ private static ProcessingContext create( public abstract Consumer workCommitter(); public abstract HeartbeatSender heartbeatSender(); - } - - @AutoValue - public abstract static class RefreshableView { - - private static RefreshableView.Builder builder() { - return new AutoValue_Work_RefreshableView.Builder(); - } - abstract WorkId workId(); - - public final long workToken() { - return workId().workToken(); - } - - public final long cacheToken() { - return workId().cacheToken(); - } - - abstract Function isRefreshable(); - - public final boolean isRefreshable(Instant refreshDeadline) { - return isRefreshable().apply(refreshDeadline); - } - - public abstract HeartbeatSender heartbeatSender(); - - public abstract boolean isFailed(); - - public abstract ImmutableList latencyAttributions(); + abstract Builder toBuilder(); @AutoValue.Builder abstract static class Builder { - abstract Builder setWorkId(WorkId value); + abstract Builder setComputationId(String value); - abstract Builder setIsRefreshable(Function value); + abstract Builder setKeyedDataFetcher( + Function> value); - abstract Builder setHeartbeatSender(HeartbeatSender value); + abstract Builder setWorkCommitter(Consumer value); - abstract Builder setIsFailed(boolean value); - - abstract Builder setLatencyAttributions(ImmutableList value); + abstract Builder setHeartbeatSender(HeartbeatSender value); - abstract RefreshableView build(); + abstract ProcessingContext build(); } } } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/WorkId.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/WorkId.java index f8f8d1901914e..d4e7f05d255fa 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/WorkId.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/WorkId.java @@ -41,9 +41,9 @@ public static WorkId of(Windmill.WorkItem workItem) { .build(); } - abstract long cacheToken(); + public abstract long cacheToken(); - abstract long workToken(); + public abstract long workToken(); @AutoValue.Builder public abstract static class Builder { diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/WindmillStreamClosedException.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/WindmillStreamClosedException.java index bcdf6521bda2e..412b4b01244fc 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/WindmillStreamClosedException.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/WindmillStreamClosedException.java @@ -27,7 +27,7 @@ public WindmillStreamClosedException(String message) { } /** Returns whether an exception was caused by a {@link WindmillStreamClosedException}. */ - public static boolean isWindmillStreamCancelledException(Throwable t) { + public static boolean wasCauseOf(Throwable t) { while (t != null) { if (t instanceof WindmillStreamClosedException) { return true; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/commits/StreamingEngineWorkCommitter.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/commits/StreamingEngineWorkCommitter.java index ed4dcfa212f1d..948e51fce4644 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/commits/StreamingEngineWorkCommitter.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/commits/StreamingEngineWorkCommitter.java @@ -126,7 +126,7 @@ private void drainCommitQueue() { } private void failCommit(Commit commit) { - commit.work().setFailed(); + commit.work().fail(); onCommitComplete.accept(CompleteCommit.forFailedWork(commit)); } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcDirectGetWorkStream.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcDirectGetWorkStream.java index 8f6caf9f9574a..641e3e323bb7c 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcDirectGetWorkStream.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcDirectGetWorkStream.java @@ -19,7 +19,9 @@ import com.google.auto.value.AutoValue; import java.io.IOException; +import java.io.InputStream; import java.io.PrintWriter; +import java.util.Optional; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; @@ -170,6 +172,15 @@ private static Watermarks createWatermarks(WorkItem workItem, ComputationMetadat .build(); } + private static Optional parseWorkItem(InputStream serializedWorkItem) { + try { + return Optional.of(WorkItem.parseFrom(serializedWorkItem)); + } catch (IOException e) { + LOG.error("Failed to parse work item from stream: ", e); + return Optional.empty(); + } + } + private synchronized GetWorkBudget getThenResetBudgetAdjustment() { return nextBudgetAdjustment.getAndUpdate(unused -> GetWorkBudget.noBudget()); } @@ -322,31 +333,27 @@ private long bufferedSize() { } private void runAndReset() { - try { - WorkItem workItem = WorkItem.parseFrom(data.newInput()); - updatePendingResponseBudget(1, workItem.getSerializedSize()); - workItemScheduler.scheduleWork( - workItem, - createWatermarks(workItem, Preconditions.checkNotNull(metadata)), - createProcessingContext(Preconditions.checkNotNull(metadata.computationId())), - // After the work item is successfully queued or dropped by ActiveWorkState, remove it - // from the pendingResponseBudget. - queuedWorkItem -> updatePendingResponseBudget(-1, -workItem.getSerializedSize()), - workTimingInfosTracker.getLatencyAttributions()); - } catch (IOException e) { - LOG.error("Failed to parse work item from stream: ", e); - } + parseWorkItem(data.newInput()).ifPresent(this::scheduleWorkItem); workTimingInfosTracker.reset(); data = ByteString.EMPTY; } - private Work.ProcessingContext createProcessingContext(String computationId) { - return Work.createProcessingContext( - computationId, - (computation, request) -> - keyedGetDataFn.apply(getDataStream.get()).apply(computation, request), - workCommitter.get()::commit, - new DirectHeartbeatSender(getDataStream.get())); + private void scheduleWorkItem(WorkItem workItem) { + updatePendingResponseBudget(1, workItem.getSerializedSize()); + ComputationMetadata metadata = Preconditions.checkNotNull(this.metadata); + workItemScheduler.scheduleWork( + workItem, + createWatermarks(workItem, metadata), + Work.createProcessingContext( + metadata.computationId(), + (computation, request) -> + keyedGetDataFn.apply(getDataStream.get()).apply(computation, request), + workCommitter.get()::commit, + DirectHeartbeatSender.create(getDataStream.get())), + // After the work item is successfully queued or dropped by ActiveWorkState, remove it + // from the pendingResponseBudget. + queuedWorkItem -> updatePendingResponseBudget(-1, -workItem.getSerializedSize()), + workTimingInfosTracker.getLatencyAttributions()); } } } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetDataStream.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetDataStream.java index aa30e6653c7b0..1734a73586572 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetDataStream.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetDataStream.java @@ -316,6 +316,9 @@ private ResponseT issueRequest(QueuedRequest request, ParseFn requests() { return requests; } - /** - * Put all global data requests first because there is only a single repeated field for request - * ids and the initial ids correspond to global data requests if they are present. - */ - List sortedRequests() { - requests.sort(QueuedRequest.globalRequestsFirst()); - return requests; - } - - void validateRequests(Consumer requestValidator) { - requests.forEach(requestValidator); - } - - int requestCount() { - return requests.size(); - } - long byteSize() { return byteSize; } @@ -191,8 +169,6 @@ String createStreamCancelledErrorMessage() { .map( keyedRequest -> "KeyedGetState=[" - + "key=" - + keyedRequest.getKey() + "shardingKey=" + keyedRequest.getShardingKey() + "cacheToken=" diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/ActiveWorkRefresher.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/ActiveWorkRefresher.java index cdb6844479287..9e235fa4e75c3 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/ActiveWorkRefresher.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/ActiveWorkRefresher.java @@ -127,7 +127,7 @@ private void refreshActiveWork() { // to correctly fan-out the heartbeat requests. ImmutableListMultimap heartbeats = HeartbeatRequests.getRefreshableKeyHeartbeats( - computationState.currentActiveWorkReadOnly(sampler), refreshDeadline); + computationState.currentActiveWorkReadOnly(), refreshDeadline, sampler); // Aggregate the heartbeats across computations by GetDataStream for correct fan out. for (Map.Entry> heartbeatsPerStream : heartbeats.asMap().entrySet()) { diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/DirectHeartbeatSender.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/DirectHeartbeatSender.java index 8cfe5c8d73f05..2059039183451 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/DirectHeartbeatSender.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/DirectHeartbeatSender.java @@ -19,45 +19,55 @@ import java.util.List; import java.util.Map; -import org.apache.beam.runners.dataflow.worker.windmill.Windmill; -import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStream; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill.HeartbeatRequest; +import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStream.GetDataStream; import org.apache.beam.sdk.annotations.Internal; import org.slf4j.Logger; import org.slf4j.LoggerFactory; /** - * {@link HeartbeatSender} implementation that sends heartbeats directly on the underlying stream. - * If the stream is closed, does nothing. + * {@link HeartbeatSender} implementation that sends heartbeats directly on the underlying stream if + * the stream is not closed. * - * @implNote {@link #equals(Object)} and {@link #hashCode()} implementations delegate to internal - * {@link org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStream.GetDataStream} - * implementations so that requests can be grouped and sent on the same stream. + * @implNote + *

{@link #equals(Object)} and {@link #hashCode()} implementations delegate to internal + * {@link GetDataStream} implementations so that requests can be grouped and sent on the same + * stream. + *

{@link #onStreamClosed} is a hook used to bind side effects if {@link + * #sendHeartbeats(Map)} is called when the underlying stream is closed, and defaults as a + * no-op. */ @Internal public final class DirectHeartbeatSender implements HeartbeatSender { private static final Logger LOG = LoggerFactory.getLogger(DirectHeartbeatSender.class); - private final WindmillStream.GetDataStream getDataStream; + private final GetDataStream getDataStream; + private final Runnable onStreamClosed; - public DirectHeartbeatSender(WindmillStream.GetDataStream getDataStream) { + private DirectHeartbeatSender(GetDataStream getDataStream, Runnable onStreamClosed) { this.getDataStream = getDataStream; + this.onStreamClosed = onStreamClosed; + } + + public static DirectHeartbeatSender create(GetDataStream getDataStream) { + return new DirectHeartbeatSender(getDataStream, () -> {}); } @Override - public void sendHeartbeats(Map> heartbeats) { - if (isInvalid()) { + public void sendHeartbeats(Map> heartbeats) { + if (getDataStream.isClosed()) { LOG.warn( "Trying to refresh work on stream={} after work has moved off of worker." + " heartbeats={}", getDataStream, heartbeats); + onStreamClosed.run(); } else { getDataStream.refreshActiveWork(heartbeats); } } - @Override - public synchronized boolean isInvalid() { - return getDataStream.isClosed(); + public HeartbeatSender withStreamClosedHandler(Runnable onStreamClosed) { + return new DirectHeartbeatSender(getDataStream, onStreamClosed); } @Override diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/HeartbeatRequests.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/HeartbeatRequests.java index 3ef2259fa1a3c..ad22e0d7b07e1 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/HeartbeatRequests.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/HeartbeatRequests.java @@ -23,8 +23,9 @@ import java.util.Map; import java.util.stream.Stream; import org.apache.beam.repackaged.core.org.apache.commons.lang3.tuple.Pair; +import org.apache.beam.runners.dataflow.worker.DataflowExecutionStateSampler; +import org.apache.beam.runners.dataflow.worker.streaming.RefreshableWork; import org.apache.beam.runners.dataflow.worker.streaming.ShardedKey; -import org.apache.beam.runners.dataflow.worker.streaming.Work; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.HeartbeatRequest; import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableListMultimap; @@ -37,31 +38,34 @@ public final class HeartbeatRequests { private HeartbeatRequests() {} static ImmutableListMultimap getRefreshableKeyHeartbeats( - ImmutableListMultimap activeWork, Instant refreshDeadline) { + ImmutableListMultimap activeWork, + Instant refreshDeadline, + DataflowExecutionStateSampler sampler) { return activeWork.asMap().entrySet().stream() - .flatMap(e -> toHeartbeatRequest(e, refreshDeadline)) + .flatMap(e -> toHeartbeatRequest(e, refreshDeadline, sampler)) .collect(toImmutableListMultimap(Pair::getKey, Pair::getValue)); } private static Stream> toHeartbeatRequest( - Map.Entry> shardedKeyAndWorkQueue, - Instant refreshDeadline) { + Map.Entry> shardedKeyAndWorkQueue, + Instant refreshDeadline, + DataflowExecutionStateSampler sampler) { ShardedKey shardedKey = shardedKeyAndWorkQueue.getKey(); - Collection workQueue = shardedKeyAndWorkQueue.getValue(); + Collection workQueue = shardedKeyAndWorkQueue.getValue(); return workQueue.stream() .filter(work -> work.isRefreshable(refreshDeadline)) - // Don't send heartbeats for queued work we already know is failed. - .filter(work -> !work.isFailed()) - .map(work -> Pair.of(work.heartbeatSender(), createHeartbeatRequest(shardedKey, work))); + .map( + work -> + Pair.of(work.heartbeatSender(), createHeartbeatRequest(shardedKey, work, sampler))); } private static HeartbeatRequest createHeartbeatRequest( - ShardedKey shardedKey, Work.RefreshableView work) { + ShardedKey shardedKey, RefreshableWork work, DataflowExecutionStateSampler sampler) { return HeartbeatRequest.newBuilder() .setShardingKey(shardedKey.shardingKey()) - .setWorkToken(work.workToken()) - .setCacheToken(work.cacheToken()) - .addAllLatencyAttribution(work.latencyAttributions()) + .setWorkToken(work.id().workToken()) + .setCacheToken(work.id().cacheToken()) + .addAllLatencyAttribution(work.getLatencyAttributions(/* isHeartbeat= */ true, sampler)) .build(); } } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/HeartbeatSender.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/HeartbeatSender.java index 3a2d0cb05fc5a..85918353b3331 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/HeartbeatSender.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/HeartbeatSender.java @@ -25,8 +25,4 @@ @FunctionalInterface public interface HeartbeatSender { void sendHeartbeats(Map> heartbeats); - - default boolean isInvalid() { - return 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 5db892721f1d9..23c66f51ad8c5 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 @@ -1028,7 +1028,7 @@ public void testFailedWorkItemsAbort() throws Exception { numReads++; // Fail the work item after reading two elements. if (numReads == 2) { - dummyWork.setFailed(); + dummyWork.fail(); } } assertThat(numReads, equalTo(2)); diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/commits/StreamingEngineWorkCommitterTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/commits/StreamingEngineWorkCommitterTest.java index 2e345ad8ef8d2..7ede4c8b8242d 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/commits/StreamingEngineWorkCommitterTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/commits/StreamingEngineWorkCommitterTest.java @@ -168,7 +168,7 @@ public void testCommit_handlesFailedCommits() { Work work = createMockWork(i); // Fail half of the work. if (i % 2 == 0) { - work.setFailed(); + work.fail(); } WorkItemCommitRequest commitRequest = WorkItemCommitRequest.newBuilder() diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/HeartbeatRequestsTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/HeartbeatRequestsTest.java index f4bafe6fb3ade..4a7ad3a714ca7 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/HeartbeatRequestsTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/HeartbeatRequestsTest.java @@ -27,7 +27,9 @@ import java.util.HashMap; import java.util.Map; import java.util.Objects; +import java.util.stream.Collectors; import org.apache.beam.runners.dataflow.worker.DataflowExecutionStateSampler; +import org.apache.beam.runners.dataflow.worker.streaming.RefreshableWork; 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; @@ -133,12 +135,16 @@ private void activateWorkForKey(ShardedKey shardedKey, Work work) { workQueue.addLast(work); } - private ImmutableListMultimap currentActiveWork() { - ImmutableListMultimap.Builder currentActiveWork = + private ImmutableListMultimap currentActiveWork() { + ImmutableListMultimap.Builder currentActiveWork = ImmutableListMultimap.builder(); for (Map.Entry> keyedWorkQueues : activeWork.entrySet()) { - currentActiveWork.putAll(keyedWorkQueues.getKey(), keyedWorkQueues.getValue()); + currentActiveWork.putAll( + keyedWorkQueues.getKey(), + keyedWorkQueues.getValue().stream() + .map(Work::refreshableView) + .collect(Collectors.toList())); } return currentActiveWork.build(); From 5c3165870c6ed68ad58c7a0d00ccaecf7e6585f4 Mon Sep 17 00:00:00 2001 From: Martin Trieu Date: Thu, 20 Jun 2024 14:32:28 -0700 Subject: [PATCH 07/13] fix dangling commits, add budget freeing when failing work --- .../worker/streaming/ActiveWorkState.java | 56 ++++++-- .../worker/streaming/ComputationState.java | 14 +- .../dataflow/worker/streaming/Work.java | 124 +++++++++++++++--- .../client/AbstractWindmillStream.java | 86 ++++++------ .../windmill/client/WindmillStream.java | 3 +- .../commits/StreamingEngineWorkCommitter.java | 30 ++++- .../client/grpc/GrpcCommitWorkStream.java | 68 ++++++---- .../grpc/observers/DirectStreamObserver.java | 41 +++--- .../work/refresh/DirectHeartbeatSender.java | 7 +- .../dataflow/worker/FakeWindmillServer.java | 117 +++++++++-------- .../worker/streaming/ActiveWorkStateTest.java | 79 +++++++---- .../StreamingEngineWorkCommitterTest.java | 29 ++-- .../client/grpc/GrpcWindmillServerTest.java | 13 +- 13 files changed, 433 insertions(+), 234 deletions(-) diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/ActiveWorkState.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/ActiveWorkState.java index 39454c7d329d2..0597bdf6c836c 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/ActiveWorkState.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/ActiveWorkState.java @@ -19,6 +19,7 @@ import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableListMultimap.flatteningToImmutableListMultimap; +import com.google.auto.value.AutoValue; import java.io.PrintWriter; import java.util.ArrayDeque; import java.util.Collection; @@ -73,7 +74,8 @@ public final class ActiveWorkState { /** * Current budget that is being processed or queued on the user worker. Incremented when work is * activated in {@link #activateWorkForKey(ExecutableWork)}, and decremented when work is - * completed in {@link #completeWorkAndGetNextWorkForKey(ShardedKey, WorkId)}. + * completed in {@link #completeWorkAndGetNextWorkForKey(ShardedKey, WorkId)} or failed via {@link + * Work#fail()}. */ private final AtomicReference activeGetWorkBudget; @@ -119,27 +121,36 @@ private static String elapsedString(Instant start, Instant end) { *

4. STALE: A work queue for the {@link ShardedKey} exists, and there is a queued {@link Work} * with a greater workToken than the passed in {@link Work}. */ - synchronized ActivateWorkResult activateWorkForKey(ExecutableWork executableWork) { - ShardedKey shardedKey = executableWork.work().getShardedKey(); + synchronized ActivatedWork activateWorkForKey(ExecutableWork executableWork) { + // Attach a failure handler to the work so that we remove it from active work budget when it is + // failed. + ExecutableWork workWithFailureHandler = + ExecutableWork.create( + executableWork + .work() + .withFailureHandler(() -> decrementActiveWorkBudget(executableWork.work())), + executableWork.executeWorkFn()); + + ShardedKey shardedKey = workWithFailureHandler.work().getShardedKey(); Deque workQueue = activeWork.getOrDefault(shardedKey, new ArrayDeque<>()); // This key does not have any work queued up on it. Create one, insert Work, and mark the work // to be executed. if (!activeWork.containsKey(shardedKey) || workQueue.isEmpty()) { - workQueue.addLast(executableWork); + workQueue.addLast(workWithFailureHandler); activeWork.put(shardedKey, workQueue); - incrementActiveWorkBudget(executableWork.work()); - return ActivateWorkResult.EXECUTE; + incrementActiveWorkBudget(workWithFailureHandler.work()); + return ActivatedWork.executeNow(workWithFailureHandler); } // Check to see if we have this work token queued. Iterator workIterator = workQueue.iterator(); while (workIterator.hasNext()) { ExecutableWork queuedWork = workIterator.next(); - if (queuedWork.id().equals(executableWork.id())) { - return ActivateWorkResult.DUPLICATE; + if (queuedWork.id().equals(workWithFailureHandler.id())) { + return ActivatedWork.notImmediatelyExecutable(ActivateWorkResult.DUPLICATE); } - if (queuedWork.id().cacheToken() == executableWork.id().cacheToken()) { - if (executableWork.id().workToken() > queuedWork.id().workToken()) { + if (queuedWork.id().cacheToken() == workWithFailureHandler.id().cacheToken()) { + if (workWithFailureHandler.id().workToken() > queuedWork.id().workToken()) { // Check to see if the queuedWork is active. We only want to remove it if it is NOT // currently active. if (!queuedWork.equals(workQueue.peek())) { @@ -148,15 +159,15 @@ synchronized ActivateWorkResult activateWorkForKey(ExecutableWork executableWork } // Continue here to possibly remove more non-active stale work that is queued. } else { - return ActivateWorkResult.STALE; + return ActivatedWork.notImmediatelyExecutable(ActivateWorkResult.STALE); } } } // Queue the work for later processing. - workQueue.addLast(executableWork); - incrementActiveWorkBudget(executableWork.work()); - return ActivateWorkResult.QUEUED; + workQueue.addLast(workWithFailureHandler); + incrementActiveWorkBudget(workWithFailureHandler.work()); + return ActivatedWork.notImmediatelyExecutable(ActivateWorkResult.QUEUED); } /** @@ -367,4 +378,21 @@ enum ActivateWorkResult { DUPLICATE, STALE } + + @AutoValue + abstract static class ActivatedWork { + private static ActivatedWork executeNow(ExecutableWork executableWork) { + return new AutoValue_ActiveWorkState_ActivatedWork( + ActivateWorkResult.EXECUTE, Optional.of(executableWork)); + } + + private static ActivatedWork notImmediatelyExecutable(ActivateWorkResult activateWorkResult) { + Preconditions.checkState(activateWorkResult != ActivateWorkResult.EXECUTE); + return new AutoValue_ActiveWorkState_ActivatedWork(activateWorkResult, Optional.empty()); + } + + abstract ActivateWorkResult result(); + + abstract Optional executableWork(); + } } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/ComputationState.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/ComputationState.java index 4e4a78803823f..8b9eac1c7a297 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/ComputationState.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/ComputationState.java @@ -101,7 +101,9 @@ public Optional acquireComputationWorkExecutor() { * whether the {@link Work} will be activated, either immediately or sometime in the future. */ public boolean activateWork(ExecutableWork executableWork) { - switch (activeWorkState.activateWorkForKey(executableWork)) { + ActiveWorkState.ActivatedWork activatedWork = + activeWorkState.activateWorkForKey(executableWork); + switch (activatedWork.result()) { case DUPLICATE: // Fall through intentionally. Work was not and will not be activated in these cases. case STALE: @@ -110,7 +112,15 @@ public boolean activateWork(ExecutableWork executableWork) { return true; case EXECUTE: { - execute(executableWork); + execute( + activatedWork + .executableWork() + // This will never happen it is not possible to create ActivatedWork with state + // EXECUTE without an ExecutableWork instance. + .orElseThrow( + () -> + new IllegalStateException( + "ExecutableWork is required for ActivateWorkResult EXECUTE."))); return true; } default: diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/Work.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/Work.java index f187c41676b3f..92f681071d6b0 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/Work.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/Work.java @@ -20,6 +20,7 @@ import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList.toImmutableList; import com.google.auto.value.AutoValue; +import com.google.common.base.Objects; import java.util.Arrays; import java.util.Collection; import java.util.EnumMap; @@ -31,6 +32,7 @@ import java.util.function.Consumer; import java.util.function.Function; import java.util.function.Supplier; +import javax.annotation.Nullable; import javax.annotation.concurrent.NotThreadSafe; import org.apache.beam.repackaged.core.org.apache.commons.lang3.tuple.Pair; import org.apache.beam.runners.dataflow.worker.ActiveMessageMetadata; @@ -74,34 +76,43 @@ public final class Work implements RefreshableWork { private final Map totalDurationPerState; private final WorkId id; private final String latencyTrackingId; + private final Runnable onFailed; private TimedState currentState; private volatile boolean isFailed; private Work( + ShardedKey shardedKey, WorkItem workItem, - Watermarks watermarks, ProcessingContext processingContext, - Supplier clock) { - this.shardedKey = ShardedKey.create(workItem.getKey(), workItem.getShardingKey()); + Watermarks watermarks, + Supplier clock, + Instant startTime, + Map totalDurationPerState, + WorkId id, + String latencyTrackingId, + Runnable onFailed, + TimedState currentState, + boolean isFailed) { + this.shardedKey = shardedKey; this.workItem = workItem; this.watermarks = watermarks; this.clock = clock; - this.startTime = clock.get(); - this.totalDurationPerState = new EnumMap<>(LatencyAttribution.State.class); - this.id = WorkId.of(workItem); - this.latencyTrackingId = - Long.toHexString(workItem.getShardingKey()) - + '-' - + Long.toHexString(workItem.getWorkToken()); - this.currentState = TimedState.initialState(startTime); - this.isFailed = false; + this.startTime = startTime; + this.totalDurationPerState = totalDurationPerState; + this.id = id; + this.latencyTrackingId = latencyTrackingId; + this.onFailed = onFailed; + this.currentState = currentState; + this.isFailed = isFailed; this.processingContext = processingContext.heartbeatSender() instanceof DirectHeartbeatSender + && !((DirectHeartbeatSender) processingContext.heartbeatSender()) + .hasStreamClosedHandler() ? processingContext .toBuilder() .setHeartbeatSender( ((DirectHeartbeatSender) processingContext.heartbeatSender()) - .withStreamClosedHandler(() -> isFailed = true)) + .withStreamClosedHandler(() -> this.isFailed = true)) .build() : processingContext; } @@ -112,7 +123,21 @@ public static Work create( ProcessingContext processingContext, Supplier clock, Collection getWorkStreamLatencies) { - Work work = new Work(workItem, watermarks, processingContext, clock); + Instant startTime = clock.get(); + Work work = + new Work( + ShardedKey.create(workItem.getKey(), workItem.getShardingKey()), + workItem, + processingContext, + watermarks, + clock, + startTime, + new EnumMap<>(LatencyAttribution.State.class), + WorkId.of(workItem), + buildLatencyTrackingId(workItem), + () -> {}, + TimedState.initialState(startTime), + false); work.recordGetWorkStreamLatencies(getWorkStreamLatencies); return work; } @@ -162,6 +187,29 @@ private static LatencyAttribution.Builder createLatencyAttributionWithActiveLate return latencyAttribution; } + private static String buildLatencyTrackingId(WorkItem workItem) { + return Long.toHexString(workItem.getShardingKey()) + + '-' + + Long.toHexString(workItem.getWorkToken()); + } + + /** Returns a new {@link Work} instance with the same state and a different failure handler. */ + public Work withFailureHandler(Runnable onFailed) { + return new Work( + shardedKey, + workItem, + processingContext, + watermarks, + clock, + startTime, + totalDurationPerState, + id, + latencyTrackingId, + onFailed, + currentState, + isFailed); + } + public WorkItem getWorkItem() { return workItem; } @@ -207,16 +255,15 @@ public HeartbeatSender heartbeatSender() { public void fail() { LOG.debug( - "Failing work " + "Failing work: [computationId= " + processingContext.computationId() - + " " + + ", key=" + shardedKey - + " " - + id.workToken() - + " " - + id.cacheToken() - + ". The work will be retried and is not lost."); + + ", workId=" + + id + + "]. The work will be retried and is not lost."); this.isFailed = true; + onFailed.run(); } public boolean isCommitPending() { @@ -307,6 +354,41 @@ public RefreshableWork refreshableView() { return this; } + @Override + public boolean equals(@Nullable Object o) { + if (o == null) return false; + if (this == o) return true; + if (!(o instanceof Work)) return false; + Work work = (Work) o; + return isFailed == work.isFailed + && Objects.equal(shardedKey, work.shardedKey) + && Objects.equal(workItem, work.workItem) + && Objects.equal(processingContext, work.processingContext) + && Objects.equal(watermarks, work.watermarks) + && Objects.equal(clock, work.clock) + && Objects.equal(startTime, work.startTime) + && Objects.equal(totalDurationPerState, work.totalDurationPerState) + && Objects.equal(id, work.id) + && Objects.equal(latencyTrackingId, work.latencyTrackingId) + && Objects.equal(currentState, work.currentState); + } + + @Override + public int hashCode() { + return Objects.hashCode( + shardedKey, + workItem, + processingContext, + watermarks, + clock, + startTime, + totalDurationPerState, + id, + latencyTrackingId, + currentState, + isFailed); + } + public enum State { QUEUED(LatencyAttribution.State.QUEUED), PROCESSING(LatencyAttribution.State.ACTIVE), diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/AbstractWindmillStream.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/AbstractWindmillStream.java index 7c4380db8bc60..7693deb1206de 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/AbstractWindmillStream.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/AbstractWindmillStream.java @@ -34,7 +34,6 @@ import org.apache.beam.runners.dataflow.worker.windmill.client.grpc.observers.StreamObserverFactory; import org.apache.beam.sdk.util.BackOff; import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.Status; -import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.StatusRuntimeException; import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.StreamObserver; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.ThreadFactoryBuilder; import org.checkerframework.checker.nullness.qual.Nullable; @@ -229,6 +228,8 @@ public final void appendSummaryHtml(PrintWriter writer) { if (clientClosed.get()) { writer.write(", client closed"); } + + writer.format(", isClosed=[%s]", isClosed()); long nowMs = Instant.now().getMillis(); long sleepLeft = sleepUntil.get() - nowMs; if (sleepLeft > 0) { @@ -265,7 +266,7 @@ public final Instant startTime() { @Override public final boolean isClosed() { - return streamClosed.get() || clientClosed.get(); + return clientClosed.get() || streamClosed.get(); } private void setLastError(String error) { @@ -274,7 +275,6 @@ private void setLastError(String error) { } private class ResponseObserver implements StreamObserver { - @Override public void onNext(ResponseT response) { try { @@ -305,55 +305,53 @@ private void onStreamFinished(@Nullable Throwable t) { } } if (t != null) { - Status status = Status.fromThrowable(t); - if (t instanceof StatusRuntimeException) { - status = ((StatusRuntimeException) t).getStatus(); - } - - String statusError = status == null ? "" : status.toString(); - setLastError(statusError); - if (errorCount.getAndIncrement() % logEveryNStreamFailures == 0) { - long nowMillis = Instant.now().getMillis(); - String responseDebug; - if (lastResponseTimeMs.get() == 0) { - responseDebug = "never received response"; - } else { - responseDebug = - "received response " + (nowMillis - lastResponseTimeMs.get()) + "ms ago"; - } - LOG.debug( - "{} streaming Windmill RPC errors for {}, last was: {} with status {}." - + " created {}ms ago, {}. This is normal with autoscaling.", - AbstractWindmillStream.this.getClass(), - errorCount.get(), - t, - statusError, - nowMillis - startTimeMs.get(), - responseDebug); - } - // If the stream was stopped due to a resource exhausted error then we are throttled. - if (status != null && status.getCode() == Status.Code.RESOURCE_EXHAUSTED) { - startThrottleTimer(); - } - - try { - long sleep = backoff.nextBackOffMillis(); - sleepUntil.set(Instant.now().getMillis() + sleep); - Thread.sleep(sleep); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - } catch (IOException e) { - // Ignore. - } + handleStreamErrorResponse(t); } else { errorCount.incrementAndGet(); String error = "Stream completed successfully but did not complete requested operations, " - + "recreating"; + + "recreating."; LOG.warn(error); setLastError(error); } + executor.execute(AbstractWindmillStream.this::startStream); } + + private void handleStreamErrorResponse(Throwable t) { + Status errorStatus = Status.fromThrowable(t); + setLastError(errorStatus.toString()); + if (errorCount.getAndIncrement() % logEveryNStreamFailures == 0) { + long nowMillis = Instant.now().getMillis(); + String logMessage = + lastResponseTimeMs.get() == 0 + ? "never received response" + : "received response " + (nowMillis - lastResponseTimeMs.get()) + "ms ago"; + + LOG.debug( + "{} streaming Windmill RPC errors for {}, last was: {} with status {}." + + " created {}ms ago, {}. This is normal with autoscaling.", + AbstractWindmillStream.this.getClass(), + errorCount.get(), + t, + errorStatus, + nowMillis - startTimeMs.get(), + logMessage); + } + // If the stream was stopped due to a resource exhausted error then we are throttled. + if (errorStatus.getCode() == Status.Code.RESOURCE_EXHAUSTED) { + startThrottleTimer(); + } + + try { + long sleep = backoff.nextBackOffMillis(); + sleepUntil.set(Instant.now().getMillis() + sleep); + Thread.sleep(sleep); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } catch (IOException e) { + // Ignore. + } + } } } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/WindmillStream.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/WindmillStream.java index 9be7f56d20b46..531586881221c 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/WindmillStream.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/WindmillStream.java @@ -20,6 +20,7 @@ import java.io.Closeable; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.concurrent.TimeUnit; import java.util.function.Consumer; import javax.annotation.concurrent.NotThreadSafe; @@ -100,7 +101,7 @@ default void close() { * Returns a builder that can be used for sending requests. Each builder is not thread-safe but * different builders for the same stream may be used simultaneously. */ - RequestBatcher batcher(); + Optional newBatcher(); } /** Interface for streaming GetWorkerMetadata requests to Windmill. */ diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/commits/StreamingEngineWorkCommitter.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/commits/StreamingEngineWorkCommitter.java index 948e51fce4644..057faef1d49ba 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/commits/StreamingEngineWorkCommitter.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/commits/StreamingEngineWorkCommitter.java @@ -20,11 +20,13 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; import java.util.function.Consumer; import java.util.function.Supplier; import javax.annotation.Nullable; import javax.annotation.concurrent.ThreadSafe; +import org.apache.beam.runners.dataflow.worker.WorkItemCancelledException; import org.apache.beam.runners.dataflow.worker.streaming.WeightedBoundedQueue; import org.apache.beam.runners.dataflow.worker.streaming.Work; import org.apache.beam.runners.dataflow.worker.windmill.client.CloseableStream; @@ -52,6 +54,7 @@ public final class StreamingEngineWorkCommitter implements WorkCommitter { private final AtomicLong activeCommitBytes; private final Consumer onCommitComplete; private final int numCommitSenders; + private final AtomicBoolean isRunning; private StreamingEngineWorkCommitter( Supplier> commitWorkStreamFactory, @@ -72,6 +75,7 @@ private StreamingEngineWorkCommitter( this.activeCommitBytes = new AtomicLong(); this.onCommitComplete = onCommitComplete; this.numCommitSenders = numCommitSenders; + this.isRunning = new AtomicBoolean(false); } public static StreamingEngineWorkCommitter create( @@ -85,7 +89,7 @@ public static StreamingEngineWorkCommitter create( @Override @SuppressWarnings("FutureReturnValueIgnored") public void start() { - if (!commitSenders.isShutdown()) { + if (isRunning.compareAndSet(false, true) && !commitSenders.isShutdown()) { for (int i = 0; i < numCommitSenders; i++) { commitSenders.submit(this::streamingCommitLoop); } @@ -94,7 +98,16 @@ public void start() { @Override public void commit(Commit commit) { - commitQueue.put(commit); + if (commit.work().isFailed() || !isRunning.get()) { + LOG.debug( + "Trying to queue commit on shutdown, failing commit=[computationId={}, shardingKey={}, workId={} ].", + commit.computationId(), + commit.work().getShardedKey(), + commit.work().id()); + failCommit(commit); + } else { + commitQueue.put(commit); + } } @Override @@ -104,17 +117,17 @@ public long currentActiveCommitBytes() { @Override public void stop() { - if (!commitSenders.isTerminated()) { + if (isRunning.compareAndSet(true, false) && !commitSenders.isTerminated()) { commitSenders.shutdownNow(); try { commitSenders.awaitTermination(10, TimeUnit.SECONDS); } catch (InterruptedException e) { LOG.warn( - "Commit senders didn't complete shutdown within 10 seconds, continuing to drain queue", + "Commit senders didn't complete shutdown within 10 seconds, continuing to drain queue.", e); } + drainCommitQueue(); } - drainCommitQueue(); } private void drainCommitQueue() { @@ -144,6 +157,7 @@ private void streamingCommitLoop() { // Block until we have a commit or are shutting down. initialCommit = commitQueue.take(); } catch (InterruptedException e) { + Thread.currentThread().interrupt(); return; } } @@ -158,7 +172,11 @@ private void streamingCommitLoop() { try (CloseableStream closeableCommitStream = commitWorkStreamFactory.get()) { CommitWorkStream commitStream = closeableCommitStream.stream(); - try (CommitWorkStream.RequestBatcher batcher = commitStream.batcher()) { + long shardingKey = initialCommit.work().getWorkItem().getShardingKey(); + try (CommitWorkStream.RequestBatcher batcher = + commitStream + .newBatcher() + .orElseThrow(() -> new WorkItemCancelledException(shardingKey))) { if (!tryAddToCommitBatch(initialCommit, batcher)) { throw new AssertionError( "Initial commit on flushed stream should always be accepted."); diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcCommitWorkStream.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcCommitWorkStream.java index f9f579119d616..fefcc13be65c8 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcCommitWorkStream.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcCommitWorkStream.java @@ -22,11 +22,14 @@ import java.io.PrintWriter; import java.util.HashMap; import java.util.Map; +import java.util.Optional; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.atomic.AtomicLong; import java.util.function.Consumer; import java.util.function.Function; +import javax.annotation.Nullable; +import javax.annotation.concurrent.ThreadSafe; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.CommitStatus; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.JobHeader; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.StreamingCommitRequestChunk; @@ -37,20 +40,22 @@ import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStream.CommitWorkStream; import org.apache.beam.runners.dataflow.worker.windmill.client.grpc.observers.StreamObserverFactory; import org.apache.beam.runners.dataflow.worker.windmill.client.throttling.ThrottleTimer; +import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.sdk.util.BackOff; import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.StreamObserver; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +@Internal +@ThreadSafe public final class GrpcCommitWorkStream extends AbstractWindmillStream implements CommitWorkStream { private static final Logger LOG = LoggerFactory.getLogger(GrpcCommitWorkStream.class); - private static final long HEARTBEAT_REQUEST_ID = Long.MAX_VALUE; - private final Map pending; + private final Map pendingRequests; private final AtomicLong idGenerator; private final JobHeader jobHeader; private final ThrottleTimer commitWorkThrottleTimer; @@ -73,7 +78,7 @@ private GrpcCommitWorkStream( streamObserverFactory, streamRegistry, logEveryNStreamFailures); - pending = new ConcurrentHashMap<>(); + pendingRequests = new ConcurrentHashMap<>(); this.idGenerator = idGenerator; this.jobHeader = jobHeader; this.commitWorkThrottleTimer = commitWorkThrottleTimer; @@ -108,14 +113,14 @@ public static GrpcCommitWorkStream create( @Override public void appendSpecificHtml(PrintWriter writer) { - writer.format("CommitWorkStream: %d pending", pending.size()); + writer.format("CommitWorkStream: %d pending", pendingRequests.size()); } @Override protected synchronized void onNewStream() { send(StreamingCommitWorkRequest.newBuilder().setHeader(jobHeader).build()); try (Batcher resendBatcher = new Batcher()) { - for (Map.Entry entry : pending.entrySet()) { + for (Map.Entry entry : pendingRequests.entrySet()) { if (!resendBatcher.canAccept(entry.getValue().getBytes())) { resendBatcher.flush(); } @@ -125,17 +130,19 @@ protected synchronized void onNewStream() { } /** - * Returns a builder that can be used for sending requests. Each builder is not thread-safe but - * different builders for the same stream may be used simultaneously. + * Returns a builder that can be used for sending requests if the stream is not closed. + * + * @implNote Each builder is not thread-safe but different builders for the same stream may be + * used simultaneously. */ @Override - public CommitWorkStream.RequestBatcher batcher() { - return new Batcher(); + public Optional newBatcher() { + return isClosed() ? Optional.empty() : Optional.of(new Batcher()); } @Override protected boolean hasPendingRequests() { - return !pending.isEmpty(); + return !pendingRequests.isEmpty(); } @Override @@ -150,20 +157,24 @@ public void sendHealthCheck() { @Override protected void onResponse(StreamingCommitResponse response) { commitWorkThrottleTimer.stop(); - - RuntimeException finalException = null; - for (int i = 0; i < response.getRequestIdCount(); ++i) { + @Nullable RuntimeException finalException = null; + for (int i = 0; i < response.getRequestIdCount() && !isClosed(); ++i) { long requestId = response.getRequestId(i); if (requestId == HEARTBEAT_REQUEST_ID) { continue; } - PendingRequest done = pending.remove(requestId); - if (done == null) { - LOG.error("Got unknown commit request ID: {}", requestId); + @Nullable PendingRequest pendingRequest = pendingRequests.remove(requestId); + if (pendingRequest == null) { + LOG.warn( + "Got unknown commit request ID: [{}] in response: [{}]. Current pending requests: {}", + requestId, + response, + pendingRequests); } else { try { - done.onDone.accept( - (i < response.getStatusCount()) ? response.getStatus(i) : CommitStatus.OK); + CommitStatus commitStatus = + i < response.getStatusCount() ? response.getStatus(i) : CommitStatus.OK; + pendingRequest.ackResponse(commitStatus); } catch (RuntimeException e) { // Catch possible exceptions to ensure that an exception for one commit does not prevent // other commits from being processed. @@ -183,9 +194,12 @@ protected void startThrottleTimer() { } private void flushInternal(Map requests) { - if (requests.isEmpty()) { + boolean isClosed = isClosed(); + if (requests.isEmpty() || isClosed) { + LOG.debug("Skipping commit stream flush. isClosed={}", isClosed); return; } + if (requests.size() == 1) { Map.Entry elem = requests.entrySet().iterator().next(); if (elem.getValue().request.getSerializedSize() @@ -199,7 +213,7 @@ private void flushInternal(Map requests) { } } - private void issueSingleRequest(final long id, PendingRequest pendingRequest) { + private void issueSingleRequest(long id, PendingRequest pendingRequest) { StreamingCommitWorkRequest.Builder requestBuilder = StreamingCommitWorkRequest.newBuilder(); requestBuilder .addCommitChunkBuilder() @@ -209,7 +223,7 @@ private void issueSingleRequest(final long id, PendingRequest pendingRequest) { .setSerializedWorkItemCommit(pendingRequest.request.toByteString()); StreamingCommitWorkRequest chunk = requestBuilder.build(); synchronized (this) { - pending.put(id, pendingRequest); + pendingRequests.put(id, pendingRequest); try { send(chunk); } catch (IllegalStateException e) { @@ -234,7 +248,7 @@ private void issueBatchedRequest(Map requests) { } StreamingCommitWorkRequest request = requestBuilder.build(); synchronized (this) { - pending.putAll(requests); + pendingRequests.putAll(requests); try { send(request); } catch (IllegalStateException e) { @@ -243,12 +257,12 @@ private void issueBatchedRequest(Map requests) { } } - private void issueMultiChunkRequest(final long id, PendingRequest pendingRequest) { + private void issueMultiChunkRequest(long id, PendingRequest pendingRequest) { checkNotNull(pendingRequest.computation); final ByteString serializedCommit = pendingRequest.request.toByteString(); synchronized (this) { - pending.put(id, pendingRequest); + pendingRequests.put(id, pendingRequest); for (int i = 0; i < serializedCommit.size(); i += AbstractWindmillStream.RPC_STREAM_CHUNK_SIZE) { @@ -279,7 +293,6 @@ private void issueMultiChunkRequest(final long id, PendingRequest pendingRequest } private static class PendingRequest { - private final String computation; private final WorkItemCommitRequest request; private final Consumer onDone; @@ -294,10 +307,13 @@ private static class PendingRequest { long getBytes() { return (long) request.getSerializedSize() + computation.length(); } + + private void ackResponse(CommitStatus status) { + onDone.accept(status); + } } private class Batcher implements CommitWorkStream.RequestBatcher { - private final Map queue; private long queuedBytes; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/observers/DirectStreamObserver.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/observers/DirectStreamObserver.java index 3aeff166ea491..5b6d2f9b2f5c8 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/observers/DirectStreamObserver.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/observers/DirectStreamObserver.java @@ -39,30 +39,30 @@ @ThreadSafe final class DirectStreamObserver implements StreamObserver { private static final Logger LOG = LoggerFactory.getLogger(DirectStreamObserver.class); - private final Phaser phaser; + private final Phaser isReadyNotifier; + private final long deadlineSeconds; + private final int messagesBetweenIsReadyChecks; private final Object lock = new Object(); @GuardedBy("lock") private final CallStreamObserver outboundObserver; - private final long deadlineSeconds; - private final int messagesBetweenIsReadyChecks; - @GuardedBy("lock") private int messagesSinceReady = 0; DirectStreamObserver( - Phaser phaser, + Phaser isReadyNotifier, CallStreamObserver outboundObserver, long deadlineSeconds, int messagesBetweenIsReadyChecks) { - this.phaser = phaser; + this.isReadyNotifier = isReadyNotifier; this.outboundObserver = outboundObserver; this.deadlineSeconds = deadlineSeconds; - // We always let the first message pass through without blocking because it is performed under - // the StreamPool synchronized block and single header message isn't going to cause memory - // issues due to excessive buffering within grpc. + // We always let the first message pass through without blocking because it is either performed + // under the StreamPool synchronized block or WindmillStreamSender on a single thread and + // the initial single header message isn't going to cause memory issues due to excessive + // buffering within grpc. this.messagesBetweenIsReadyChecks = Math.max(1, messagesBetweenIsReadyChecks); } @@ -84,7 +84,7 @@ public void onNext(T value) { // If we awaited previously and timed out, wait for the same phase. Otherwise we're // careful to observe the phase before observing isReady. if (awaitPhase < 0) { - awaitPhase = phaser.getPhase(); + awaitPhase = isReadyNotifier.getPhase(); } if (outboundObserver.isReady()) { messagesSinceReady = 0; @@ -98,7 +98,7 @@ public void onNext(T value) { // channel has become ready. This doesn't always seem to be the case (despite // documentation stating otherwise) so we poll periodically and enforce an overall // timeout related to the stream deadline. - phaser.awaitAdvanceInterruptibly(awaitPhase, waitSeconds, TimeUnit.SECONDS); + isReadyNotifier.awaitAdvanceInterruptibly(awaitPhase, waitSeconds, TimeUnit.SECONDS); // Exit early if the phaser was terminated. if (isTerminated()) { return; @@ -107,13 +107,22 @@ public void onNext(T value) { synchronized (lock) { messagesSinceReady = 0; outboundObserver.onNext(value); - return; } } catch (TimeoutException e) { + // Check to see if the stream observer was terminated while we were waiting for the + // isReadyNotifier to become ready. + if (isTerminated()) { + return; + } + totalSecondsWaited += waitSeconds; if (totalSecondsWaited > deadlineSeconds) { throw new StreamObserverCancelledException( - "Exceeded timeout waiting for the outboundObserver to become ready meaning " + "Waited " + + totalSecondsWaited + + "s which exceeds deadline of " + + deadlineSeconds + + "s for the outboundObserver to become ready meaning " + "that the stream deadline was not respected.", e); } @@ -132,13 +141,13 @@ public void onNext(T value) { } private boolean isTerminated() { - return phaser.isTerminated() || phaser.getRegisteredParties() == 0; + return isReadyNotifier.getRegisteredParties() == 0 || isReadyNotifier.isTerminated(); } @Override public void onError(Throwable t) { synchronized (lock) { - phaser.arriveAndDeregister(); + isReadyNotifier.arriveAndDeregister(); outboundObserver.onError(t); } } @@ -146,7 +155,7 @@ public void onError(Throwable t) { @Override public void onCompleted() { synchronized (lock) { - phaser.arriveAndDeregister(); + isReadyNotifier.arriveAndDeregister(); outboundObserver.onCompleted(); } } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/DirectHeartbeatSender.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/DirectHeartbeatSender.java index 2059039183451..c827cccdf88fe 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/DirectHeartbeatSender.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/DirectHeartbeatSender.java @@ -40,6 +40,7 @@ @Internal public final class DirectHeartbeatSender implements HeartbeatSender { private static final Logger LOG = LoggerFactory.getLogger(DirectHeartbeatSender.class); + private static final Runnable NO_OP_ON_STREAM_CLOSED_HANDLER = () -> {}; private final GetDataStream getDataStream; private final Runnable onStreamClosed; @@ -49,7 +50,7 @@ private DirectHeartbeatSender(GetDataStream getDataStream, Runnable onStreamClos } public static DirectHeartbeatSender create(GetDataStream getDataStream) { - return new DirectHeartbeatSender(getDataStream, () -> {}); + return new DirectHeartbeatSender(getDataStream, NO_OP_ON_STREAM_CLOSED_HANDLER); } @Override @@ -70,6 +71,10 @@ public HeartbeatSender withStreamClosedHandler(Runnable onStreamClosed) { return new DirectHeartbeatSender(getDataStream, onStreamClosed); } + public boolean hasStreamClosedHandler() { + return !onStreamClosed.equals(NO_OP_ON_STREAM_CLOSED_HANDLER); + } + @Override public int hashCode() { return getDataStream.hashCode(); diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/FakeWindmillServer.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/FakeWindmillServer.java index 2e98748d228f8..2188052e03642 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/FakeWindmillServer.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/FakeWindmillServer.java @@ -378,66 +378,69 @@ public CommitWorkStream commitWorkStream() { return new CommitWorkStream() { @Override - public RequestBatcher batcher() { - return new RequestBatcher() { - class RequestAndDone { - final Consumer onDone; - final WorkItemCommitRequest request; - - RequestAndDone(WorkItemCommitRequest request, Consumer onDone) { - this.request = request; - this.onDone = onDone; - } - } - - final List requests = new ArrayList<>(); - - @Override - public boolean commitWorkItem( - String computation, - WorkItemCommitRequest request, - Consumer onDone) { - LOG.debug("commitWorkStream::commitWorkItem: {}", request); - errorCollector.checkThat(request.hasWorkToken(), equalTo(true)); - errorCollector.checkThat( - request.getShardingKey(), allOf(greaterThan(0L), lessThan(Long.MAX_VALUE))); - errorCollector.checkThat(request.getCacheToken(), not(equalTo(0L))); - if (requests.size() > 5) return false; - - // Throws away the result, but allows to inject latency. - Windmill.CommitWorkRequest.Builder builder = Windmill.CommitWorkRequest.newBuilder(); - builder.addRequestsBuilder().setComputationId(computation).addRequests(request); - commitsToOffer.getOrDefault(builder.build()); - - requests.add(new RequestAndDone(request, onDone)); - flush(); - return true; - } + public Optional newBatcher() { + return Optional.of( + new RequestBatcher() { + class RequestAndDone { + final Consumer onDone; + final WorkItemCommitRequest request; + + RequestAndDone( + WorkItemCommitRequest request, Consumer onDone) { + this.request = request; + this.onDone = onDone; + } + } - @Override - public void flush() { - for (RequestAndDone elem : requests) { - if (dropStreamingCommits) { - droppedStreamingCommits.put(elem.request.getWorkToken(), elem.onDone); - // Return true to indicate the request was accepted even if we are dropping the - // commit to simulate a dropped commit. - continue; + final List requests = new ArrayList<>(); + + @Override + public boolean commitWorkItem( + String computation, + WorkItemCommitRequest request, + Consumer onDone) { + LOG.debug("commitWorkStream::commitWorkItem: {}", request); + errorCollector.checkThat(request.hasWorkToken(), equalTo(true)); + errorCollector.checkThat( + request.getShardingKey(), allOf(greaterThan(0L), lessThan(Long.MAX_VALUE))); + errorCollector.checkThat(request.getCacheToken(), not(equalTo(0L))); + if (requests.size() > 5) return false; + + // Throws away the result, but allows to inject latency. + Windmill.CommitWorkRequest.Builder builder = + Windmill.CommitWorkRequest.newBuilder(); + builder.addRequestsBuilder().setComputationId(computation).addRequests(request); + commitsToOffer.getOrDefault(builder.build()); + + requests.add(new RequestAndDone(request, onDone)); + flush(); + return true; } - commitsReceived.put(elem.request.getWorkToken(), elem.request); - elem.onDone.accept( - Optional.ofNullable( - streamingCommitsToOffer.remove( - WorkId.builder() - .setWorkToken(elem.request.getWorkToken()) - .setCacheToken(elem.request.getCacheToken()) - .build())) - // Default to CommitStatus.OK - .orElse(Windmill.CommitStatus.OK)); - } - requests.clear(); - } - }; + @Override + public void flush() { + for (RequestAndDone elem : requests) { + if (dropStreamingCommits) { + droppedStreamingCommits.put(elem.request.getWorkToken(), elem.onDone); + // Return true to indicate the request was accepted even if we are dropping the + // commit to simulate a dropped commit. + continue; + } + + commitsReceived.put(elem.request.getWorkToken(), elem.request); + elem.onDone.accept( + Optional.ofNullable( + streamingCommitsToOffer.remove( + WorkId.builder() + .setWorkToken(elem.request.getWorkToken()) + .setCacheToken(elem.request.getCacheToken()) + .build())) + // Default to CommitStatus.OK + .orElse(Windmill.CommitStatus.OK)); + } + requests.clear(); + } + }); } @Override diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/ActiveWorkStateTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/ActiveWorkStateTest.java index 58939b7f45201..f59060a00ff57 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/ActiveWorkStateTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/ActiveWorkStateTest.java @@ -25,6 +25,7 @@ import static org.mockito.Mockito.mock; import static org.mockito.Mockito.verify; +import com.google.common.truth.Correspondence; import java.util.Collections; import java.util.Deque; import java.util.HashMap; @@ -100,6 +101,17 @@ private static Windmill.WorkItem createWorkItem( .build(); } + private static Correspondence comparingActivatedWork() { + return Correspondence.from( + (ExecutableWork work, ExecutableWork activatedWork) -> + work.id().equals(activatedWork.id()) + && work.getWorkItem().equals(activatedWork.getWorkItem()) + && work.executeWorkFn() == activatedWork.executeWorkFn() + && work.work().getStartTime().equals(activatedWork.work().getStartTime()) + && work.work().getState().equals(activatedWork.work().getState()), + "Both work instances should have the same state."); + } + @Before public void setup() { Map> readWriteActiveWorkMap = new HashMap<>(); @@ -110,11 +122,12 @@ public void setup() { @Test public void testActivateWorkForKey_EXECUTE_unknownKey() { - ActivateWorkResult activateWorkResult = - activeWorkState.activateWorkForKey( - createWork(createWorkItem(1L, 1L, shardedKey("someKey", 1L)))); + ExecutableWork work = createWork(createWorkItem(1L, 1L, shardedKey("someKey", 1L))); + ActiveWorkState.ActivatedWork activateWorkResult = activeWorkState.activateWorkForKey(work); - assertEquals(ActivateWorkResult.EXECUTE, activateWorkResult); + assertEquals(ActivateWorkResult.EXECUTE, activateWorkResult.result()); + assertTrue(activateWorkResult.executableWork().isPresent()); + assertThat(activateWorkResult.executableWork().get()).isNotSameInstanceAs(work); } @Test @@ -123,14 +136,15 @@ public void testActivateWorkForKey_EXECUTE_emptyWorkQueueForKey() { long workToken = 1L; long cacheToken = 2L; - ActivateWorkResult activateWorkResult = - activeWorkState.activateWorkForKey( - createWork(createWorkItem(workToken, cacheToken, shardedKey))); + ExecutableWork work = createWork(createWorkItem(workToken, cacheToken, shardedKey)); + ActiveWorkState.ActivatedWork activateWorkResult = activeWorkState.activateWorkForKey(work); Optional nextWorkForKey = activeWorkState.completeWorkAndGetNextWorkForKey(shardedKey, workId(workToken, cacheToken)); - assertEquals(ActivateWorkResult.EXECUTE, activateWorkResult); + assertEquals(ActivateWorkResult.EXECUTE, activateWorkResult.result()); + assertTrue(activateWorkResult.executableWork().isPresent()); + assertThat(activateWorkResult.executableWork().get()).isNotSameInstanceAs(work); assertEquals(Optional.empty(), nextWorkForKey); assertThat(readOnlyActiveWork).doesNotContainKey(shardedKey); } @@ -142,10 +156,11 @@ public void testActivateWorkForKey_DUPLICATE() { // ActivateWork with the same shardedKey, and the same workTokens. activeWorkState.activateWorkForKey(createWork(createWorkItem(workToken, 1L, shardedKey))); - ActivateWorkResult activateWorkResult = + ActiveWorkState.ActivatedWork activateWorkResult = activeWorkState.activateWorkForKey(createWork(createWorkItem(workToken, 1L, shardedKey))); - assertEquals(ActivateWorkResult.DUPLICATE, activateWorkResult); + assertEquals(ActivateWorkResult.DUPLICATE, activateWorkResult.result()); + assertFalse(activateWorkResult.executableWork().isPresent()); } @Test @@ -154,10 +169,11 @@ public void testActivateWorkForKey_QUEUED() { // ActivateWork with the same shardedKey, but different workTokens. activeWorkState.activateWorkForKey(createWork(createWorkItem(1L, 1L, shardedKey))); - ActivateWorkResult activateWorkResult = + ActiveWorkState.ActivatedWork activateWorkResult = activeWorkState.activateWorkForKey(createWork(createWorkItem(2L, 1L, shardedKey))); - assertEquals(ActivateWorkResult.QUEUED, activateWorkResult); + assertEquals(ActivateWorkResult.QUEUED, activateWorkResult.result()); + assertFalse(activateWorkResult.executableWork().isPresent()); } @Test @@ -230,7 +246,7 @@ public void testCompleteWorkAndGetNextWorkForKey_returnsWorkIfPresent() { activeWorkState.completeWorkAndGetNextWorkForKey(shardedKey, workToBeCompleted.id()); assertTrue(nextWorkOpt.isPresent()); - assertSame(nextWork, nextWorkOpt.get()); + assertSame(nextWork.executeWorkFn(), nextWorkOpt.get().executeWorkFn()); Optional endOfWorkQueue = activeWorkState.completeWorkAndGetNextWorkForKey(shardedKey, nextWork.id()); @@ -344,18 +360,20 @@ public void testInvalidateStuckCommits() { activeWorkState.activateWorkForKey(differentWorkTokenWork); // ActivateWork with the same shardedKey, and the same workTokens, but different cacheTokens. activeWorkState.activateWorkForKey(firstWork); - ActivateWorkResult activateWorkResult = activeWorkState.activateWorkForKey(secondWork); + ActiveWorkState.ActivatedWork activateWorkResult = + activeWorkState.activateWorkForKey(secondWork); - assertEquals(ActivateWorkResult.QUEUED, activateWorkResult); + assertEquals(ActivateWorkResult.QUEUED, activateWorkResult.result()); + assertFalse(activateWorkResult.executableWork().isPresent()); assertTrue(readOnlyActiveWork.get(shardedKey).contains(secondWork)); Optional nextWork = activeWorkState.completeWorkAndGetNextWorkForKey(shardedKey, differentWorkTokenWork.id()); assertTrue(nextWork.isPresent()); - assertSame(firstWork, nextWork.get()); + assertSame(firstWork.executeWorkFn(), nextWork.get().executeWorkFn()); nextWork = activeWorkState.completeWorkAndGetNextWorkForKey(shardedKey, firstWork.id()); assertTrue(nextWork.isPresent()); - assertSame(secondWork, nextWork.get()); + assertSame(secondWork.executeWorkFn(), nextWork.get().executeWorkFn()); } @Test @@ -371,15 +389,17 @@ public void testInvalidateStuckCommits() { // ActivateWork with the same shardedKey, and the same workTokens, but different cacheTokens. activeWorkState.activateWorkForKey(firstWork); - ActivateWorkResult activateWorkResult = activeWorkState.activateWorkForKey(secondWork); + ActiveWorkState.ActivatedWork activateWorkResult = + activeWorkState.activateWorkForKey(secondWork); - assertEquals(ActivateWorkResult.QUEUED, activateWorkResult); + assertEquals(ActivateWorkResult.QUEUED, activateWorkResult.result()); + assertFalse(activateWorkResult.executableWork().isPresent()); assertEquals(firstWork, readOnlyActiveWork.get(shardedKey).peek()); assertTrue(readOnlyActiveWork.get(shardedKey).contains(secondWork)); Optional nextWork = activeWorkState.completeWorkAndGetNextWorkForKey(shardedKey, firstWork.id()); assertTrue(nextWork.isPresent()); - assertSame(secondWork, nextWork.get()); + assertSame(secondWork.executeWorkFn(), nextWork.get().executeWorkFn()); } @Test @@ -394,11 +414,14 @@ public void testInvalidateStuckCommits() { ExecutableWork newWork = createWork(createWorkItem(newWorkToken, cacheToken, shardedKey)); activeWorkState.activateWorkForKey(queuedWork); - ActivateWorkResult activateWorkResult = activeWorkState.activateWorkForKey(newWork); + ActiveWorkState.ActivatedWork activateWorkResult = activeWorkState.activateWorkForKey(newWork); // newWork should be queued and queuedWork should not be removed since it is currently active. - assertEquals(ActivateWorkResult.QUEUED, activateWorkResult); - assertTrue(readOnlyActiveWork.get(shardedKey).contains(newWork)); + assertEquals(ActivateWorkResult.QUEUED, activateWorkResult.result()); + assertFalse(activateWorkResult.executableWork().isPresent()); + assertThat(readOnlyActiveWork.get(shardedKey)) + .comparingElementsUsing(comparingActivatedWork()) + .contains(newWork); assertEquals(queuedWork, readOnlyActiveWork.get(shardedKey).peek()); } @@ -418,9 +441,10 @@ public void testInvalidateStuckCommits() { activeWorkState.activateWorkForKey(differentWorkTokenWork); activeWorkState.activateWorkForKey(queuedWork); - ActivateWorkResult activateWorkResult = activeWorkState.activateWorkForKey(newWork); + ActiveWorkState.ActivatedWork activateWorkResult = activeWorkState.activateWorkForKey(newWork); - assertEquals(ActivateWorkResult.QUEUED, activateWorkResult); + assertEquals(ActivateWorkResult.QUEUED, activateWorkResult.result()); + assertFalse(activateWorkResult.executableWork().isPresent()); assertTrue(readOnlyActiveWork.get(shardedKey).contains(newWork)); assertFalse(readOnlyActiveWork.get(shardedKey).contains(queuedWork)); assertEquals(differentWorkTokenWork, readOnlyActiveWork.get(shardedKey).peek()); @@ -437,9 +461,10 @@ public void testActivateWorkForKey_matchingCacheTokens_newWorkTokenLesser_STALE( ExecutableWork newWork = createWork(createWorkItem(newWorkToken, cacheToken, shardedKey)); activeWorkState.activateWorkForKey(queuedWork); - ActivateWorkResult activateWorkResult = activeWorkState.activateWorkForKey(newWork); + ActiveWorkState.ActivatedWork activateWorkResult = activeWorkState.activateWorkForKey(newWork); - assertEquals(ActivateWorkResult.STALE, activateWorkResult); + assertEquals(ActivateWorkResult.STALE, activateWorkResult.result()); + assertFalse(activateWorkResult.executableWork().isPresent()); assertFalse(readOnlyActiveWork.get(shardedKey).contains(newWork)); assertEquals(queuedWork, readOnlyActiveWork.get(shardedKey).peek()); } diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/commits/StreamingEngineWorkCommitterTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/commits/StreamingEngineWorkCommitterTest.java index 7ede4c8b8242d..053cbc9573c1b 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/commits/StreamingEngineWorkCommitterTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/commits/StreamingEngineWorkCommitterTest.java @@ -260,19 +260,22 @@ public void testStop_drainsCommitQueue() { private boolean closed = false; @Override - public RequestBatcher batcher() { - return new RequestBatcher() { - @Override - public boolean commitWorkItem( - String computation, - WorkItemCommitRequest request, - Consumer onDone) { - return false; - } - - @Override - public void flush() {} - }; + public Optional newBatcher() { + return isClosed() + ? Optional.empty() + : Optional.of( + new RequestBatcher() { + @Override + public boolean commitWorkItem( + String computation, + WorkItemCommitRequest request, + Consumer onDone) { + return false; + } + + @Override + public void flush() {} + }); } @Override diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcWindmillServerTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcWindmillServerTest.java index b1d5309e12d9c..4666339043e09 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcWindmillServerTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcWindmillServerTest.java @@ -110,14 +110,13 @@ "rawtypes", // TODO(https://github.com/apache/beam/issues/20447) }) public class GrpcWindmillServerTest { - @Rule public transient Timeout globalTimeout = Timeout.seconds(600); - @Rule public GrpcCleanupRule grpcCleanup = new GrpcCleanupRule(); - @Rule public ErrorCollector errorCollector = new ErrorCollector(); - private static final Logger LOG = LoggerFactory.getLogger(GrpcWindmillServerTest.class); private static final int STREAM_CHUNK_SIZE = 2 << 20; private final long clientId = 10L; private final MutableHandlerRegistry serviceRegistry = new MutableHandlerRegistry(); + @Rule public transient Timeout globalTimeout = Timeout.seconds(600); + @Rule public GrpcCleanupRule grpcCleanup = new GrpcCleanupRule(); + @Rule public ErrorCollector errorCollector = new ErrorCollector(); private Server server; private GrpcWindmillServer client; private int remainingErrors = 20; @@ -649,7 +648,8 @@ private void commitWorkTestHelper( latches.add(new CountDownLatch(1)); } Collections.shuffle(commitRequestList); - try (CommitWorkStream.RequestBatcher batcher = stream.batcher()) { + try (CommitWorkStream.RequestBatcher batcher = + stream.newBatcher().orElseThrow(IllegalStateException::new)) { for (int i = 0; i < commitRequestList.size(); ) { final CountDownLatch latch = latches.get(i); if (batcher.commitWorkItem( @@ -787,7 +787,8 @@ public void onCompleted() { // Make the commit requests, waiting for each of them to be verified and acknowledged. CommitWorkStream stream = client.commitWorkStream(); - try (CommitWorkStream.RequestBatcher batcher = stream.batcher()) { + try (CommitWorkStream.RequestBatcher batcher = + stream.newBatcher().orElseThrow(IllegalStateException::new)) { for (int i = 0; i < commitRequestList.size(); ) { final CountDownLatch latch = latches.get(i); if (batcher.commitWorkItem( From af44fdee13c20578281a56a035459c694e88057f Mon Sep 17 00:00:00 2001 From: Martin Trieu Date: Fri, 21 Jun 2024 18:02:08 -0700 Subject: [PATCH 08/13] fix stream observer not returning after calling onNext --- .../dataflow/worker/MetricTrackingWindmillServerStub.java | 2 +- .../dataflow/worker/windmill/client/AbstractWindmillStream.java | 2 ++ .../windmill/client/grpc/observers/DirectStreamObserver.java | 1 + 3 files changed, 4 insertions(+), 1 deletion(-) diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/MetricTrackingWindmillServerStub.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/MetricTrackingWindmillServerStub.java index aea6fddc4ddfc..995b074ffe17a 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/MetricTrackingWindmillServerStub.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/MetricTrackingWindmillServerStub.java @@ -245,7 +245,7 @@ public Windmill.KeyedGetDataResponse getStateData( return getDataStream.requestKeyedData(computation, request); } catch (Exception e) { if (WindmillStreamClosedException.wasCauseOf(e)) { - LOG.error("Tried to fetch keyed data from a closed stream. Work has been cancelled", e); + LOG.debug("Tried to fetch keyed data from a closed stream. Work has been cancelled.", e); throw new WorkItemCancelledException(request.getShardingKey()); } throw new RuntimeException(e); diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/AbstractWindmillStream.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/AbstractWindmillStream.java index 7693deb1206de..6104f20f71f14 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/AbstractWindmillStream.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/AbstractWindmillStream.java @@ -162,7 +162,9 @@ protected final void send(RequestT request) { } catch (StreamObserverCancelledException e) { if (isClosed()) { LOG.warn("Stream was closed during send.", e); + return; } + LOG.error("StreamObserver was unexpectedly cancelled.", e); throw e; } } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/observers/DirectStreamObserver.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/observers/DirectStreamObserver.java index 5b6d2f9b2f5c8..1a69215695e16 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/observers/DirectStreamObserver.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/observers/DirectStreamObserver.java @@ -107,6 +107,7 @@ public void onNext(T value) { synchronized (lock) { messagesSinceReady = 0; outboundObserver.onNext(value); + return; } } catch (TimeoutException e) { // Check to see if the stream observer was terminated while we were waiting for the From 3205784eccb147a7d524539cc57c5943b30feab0 Mon Sep 17 00:00:00 2001 From: Martin Trieu Date: Fri, 21 Jun 2024 18:29:45 -0700 Subject: [PATCH 09/13] add stream id --- .../client/AbstractWindmillStream.java | 11 ++++---- .../client/grpc/GrpcCommitWorkStream.java | 6 +++- .../client/grpc/GrpcDirectGetWorkStream.java | 12 ++++++-- .../client/grpc/GrpcGetDataStream.java | 10 ++++++- .../client/grpc/GrpcGetWorkStream.java | 10 ++++++- .../grpc/GrpcGetWorkerMetadataStream.java | 3 +- .../grpc/GrpcWindmillStreamFactory.java | 21 +++++++++++--- .../client/grpc/StreamingEngineClient.java | 1 + .../client/grpc/WindmillStreamSender.java | 13 +++++++-- .../client/grpc/GrpcGetDataStreamTest.java | 1 + .../grpc/StreamingEngineClientTest.java | 5 ++-- .../client/grpc/WindmillStreamSenderTest.java | 28 +++++++++++-------- .../EvenGetWorkBudgetDistributorTest.java | 1 + 13 files changed, 91 insertions(+), 31 deletions(-) diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/AbstractWindmillStream.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/AbstractWindmillStream.java index 6104f20f71f14..7ba241377ac5d 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/AbstractWindmillStream.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/AbstractWindmillStream.java @@ -69,6 +69,7 @@ public abstract class AbstractWindmillStream implements Win protected static final int RPC_STREAM_CHUNK_SIZE = 2 << 20; private static final Logger LOG = LoggerFactory.getLogger(AbstractWindmillStream.class); protected final AtomicBoolean clientClosed; + private final String streamId; private final AtomicLong lastSendTimeMs; private final Executor executor; private final BackOff backoff; @@ -91,13 +92,12 @@ protected AbstractWindmillStream( BackOff backoff, StreamObserverFactory streamObserverFactory, Set> streamRegistry, - int logEveryNStreamFailures) { + int logEveryNStreamFailures, + String streamId) { + this.streamId = streamId; this.executor = Executors.newSingleThreadExecutor( - new ThreadFactoryBuilder() - .setDaemon(true) - .setNameFormat("WindmillStream-" + getClass() + "-thread") - .build()); + new ThreadFactoryBuilder().setDaemon(true).setNameFormat(streamId + "-thread").build()); this.backoff = backoff; this.streamRegistry = streamRegistry; this.logEveryNStreamFailures = logEveryNStreamFailures; @@ -222,6 +222,7 @@ public final synchronized void maybeSendHealthCheck(Instant lastSendThreshold) { // rendering. public final void appendSummaryHtml(PrintWriter writer) { appendSpecificHtml(writer); + writer.format("id: %s; ", streamId); if (errorCount.get() > 0) { writer.format( ", %d errors, last error [ %s ] at [%s]", diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcCommitWorkStream.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcCommitWorkStream.java index fefcc13be65c8..c2bea55e3a4b8 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcCommitWorkStream.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcCommitWorkStream.java @@ -62,6 +62,7 @@ public final class GrpcCommitWorkStream private final int streamingRpcBatchLimit; private GrpcCommitWorkStream( + String streamId, Function, StreamObserver> startCommitWorkRpcFn, BackOff backoff, @@ -77,7 +78,8 @@ private GrpcCommitWorkStream( backoff, streamObserverFactory, streamRegistry, - logEveryNStreamFailures); + logEveryNStreamFailures, + streamId); pendingRequests = new ConcurrentHashMap<>(); this.idGenerator = idGenerator; this.jobHeader = jobHeader; @@ -86,6 +88,7 @@ private GrpcCommitWorkStream( } public static GrpcCommitWorkStream create( + String streamId, Function, StreamObserver> startCommitWorkRpcFn, BackOff backoff, @@ -98,6 +101,7 @@ public static GrpcCommitWorkStream create( int streamingRpcBatchLimit) { GrpcCommitWorkStream commitWorkStream = new GrpcCommitWorkStream( + "CommitWorkStream-" + streamId, startCommitWorkRpcFn, backoff, streamObserverFactory, diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcDirectGetWorkStream.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcDirectGetWorkStream.java index 641e3e323bb7c..83bbf3eac7455 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcDirectGetWorkStream.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcDirectGetWorkStream.java @@ -100,6 +100,7 @@ public final class GrpcDirectGetWorkStream private final ConcurrentMap workItemBuffers; private GrpcDirectGetWorkStream( + String streamId, Function< StreamObserver, StreamObserver> @@ -116,7 +117,12 @@ private GrpcDirectGetWorkStream( keyedGetDataFn, WorkItemScheduler workItemScheduler) { super( - startGetWorkRpcFn, backoff, streamObserverFactory, streamRegistry, logEveryNStreamFailures); + startGetWorkRpcFn, + backoff, + streamObserverFactory, + streamRegistry, + logEveryNStreamFailures, + streamId); this.request = request; this.getWorkThrottleTimer = getWorkThrottleTimer; this.workItemScheduler = workItemScheduler; @@ -132,6 +138,7 @@ private GrpcDirectGetWorkStream( } public static GrpcDirectGetWorkStream create( + String streamId, Function< StreamObserver, StreamObserver> @@ -149,6 +156,7 @@ public static GrpcDirectGetWorkStream create( WorkItemScheduler workItemScheduler) { GrpcDirectGetWorkStream getWorkStream = new GrpcDirectGetWorkStream( + "DirectGetWorkStream-" + streamId, startGetWorkRpcFn, request, backoff, @@ -240,7 +248,7 @@ protected boolean hasPendingRequests() { public void appendSpecificHtml(PrintWriter writer) { // Number of buffers is same as distinct workers that sent work on this stream. writer.format( - "GetWorkStream: %d buffers, %s inflight budget allowed.", + "DirectGetWorkStream: %d buffers, %s inflight budget allowed.", workItemBuffers.size(), inFlightBudget.get()); } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetDataStream.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetDataStream.java index 1734a73586572..98e3a9f207634 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetDataStream.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetDataStream.java @@ -76,6 +76,7 @@ public final class GrpcGetDataStream private final Consumer> processHeartbeatResponses; private GrpcGetDataStream( + String streamId, Function, StreamObserver> startGetDataRpcFn, BackOff backoff, @@ -89,7 +90,12 @@ private GrpcGetDataStream( boolean sendKeyedGetDataRequests, Consumer> processHeartbeatResponses) { super( - startGetDataRpcFn, backoff, streamObserverFactory, streamRegistry, logEveryNStreamFailures); + startGetDataRpcFn, + backoff, + streamObserverFactory, + streamRegistry, + logEveryNStreamFailures, + streamId); this.idGenerator = idGenerator; this.getDataThrottleTimer = getDataThrottleTimer; this.jobHeader = jobHeader; @@ -101,6 +107,7 @@ private GrpcGetDataStream( } public static GrpcGetDataStream create( + String streamId, Function, StreamObserver> startGetDataRpcFn, BackOff backoff, @@ -115,6 +122,7 @@ public static GrpcGetDataStream create( Consumer> processHeartbeatResponses) { GrpcGetDataStream getDataStream = new GrpcGetDataStream( + "GetDataStream" + streamId, startGetDataRpcFn, backoff, streamObserverFactory, diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetWorkStream.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetWorkStream.java index 867180fb0d31c..5c19ef61629ae 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetWorkStream.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetWorkStream.java @@ -60,6 +60,7 @@ public final class GrpcGetWorkStream private final AtomicLong inflightBytes; private GrpcGetWorkStream( + String streamId, Function< StreamObserver, StreamObserver> @@ -72,7 +73,12 @@ private GrpcGetWorkStream( ThrottleTimer getWorkThrottleTimer, WorkItemReceiver receiver) { super( - startGetWorkRpcFn, backoff, streamObserverFactory, streamRegistry, logEveryNStreamFailures); + startGetWorkRpcFn, + backoff, + streamObserverFactory, + streamRegistry, + logEveryNStreamFailures, + streamId); this.request = request; this.getWorkThrottleTimer = getWorkThrottleTimer; this.receiver = receiver; @@ -82,6 +88,7 @@ private GrpcGetWorkStream( } public static GrpcGetWorkStream create( + String streamId, Function< StreamObserver, StreamObserver> @@ -95,6 +102,7 @@ public static GrpcGetWorkStream create( WorkItemReceiver receiver) { GrpcGetWorkStream getWorkStream = new GrpcGetWorkStream( + streamId, startGetWorkRpcFn, request, backoff, diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetWorkerMetadataStream.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetWorkerMetadataStream.java index 3672f02c813f2..0459aa503d55c 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetWorkerMetadataStream.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetWorkerMetadataStream.java @@ -69,7 +69,8 @@ private GrpcGetWorkerMetadataStream( backoff, streamObserverFactory, streamRegistry, - logEveryNStreamFailures); + logEveryNStreamFailures, + "GetWorkerMetadata"); this.workerMetadataRequest = WorkerMetadataRequest.newBuilder().setHeader(jobHeader).build(); this.metadataVersion = metadataVersion; this.getWorkerMetadataThrottleTimer = getWorkerMetadataThrottleTimer; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcWindmillStreamFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcWindmillStreamFactory.java index 7c7122d7c37d8..5b4f4191f6b07 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcWindmillStreamFactory.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcWindmillStreamFactory.java @@ -72,6 +72,7 @@ public class GrpcWindmillStreamFactory implements StatusDataProvider { private static final int DEFAULT_STREAMING_RPC_BATCH_LIMIT = Integer.MAX_VALUE; private static final int DEFAULT_WINDMILL_MESSAGES_BETWEEN_IS_READY_CHECKS = 1; private static final int NO_HEALTH_CHECKS = -1; + private static final String DISPATCHER_STREAM_ID = "Dispatcher"; private final JobHeader jobHeader; private final int logEveryNStreamFailures; @@ -184,6 +185,7 @@ public GetWorkStream createGetWorkStream( ThrottleTimer getWorkThrottleTimer, WorkItemReceiver processWorkItem) { return GrpcGetWorkStream.create( + DISPATCHER_STREAM_ID, responseObserver -> withDefaultDeadline(stub).getWorkStream(responseObserver), request, grpcBackOff.get(), @@ -195,6 +197,7 @@ public GetWorkStream createGetWorkStream( } public GetWorkStream createDirectGetWorkStream( + String streamId, CloudWindmillServiceV1Alpha1Stub stub, GetWorkRequest request, ThrottleTimer getWorkThrottleTimer, @@ -206,6 +209,7 @@ public GetWorkStream createDirectGetWorkStream( keyedGetDataFn, WorkItemScheduler workItemScheduler) { return GrpcDirectGetWorkStream.create( + streamId, stub::getWorkStream, request, grpcBackOff.get(), @@ -220,8 +224,9 @@ public GetWorkStream createDirectGetWorkStream( } public GetDataStream createGetDataStream( - CloudWindmillServiceV1Alpha1Stub stub, ThrottleTimer getDataThrottleTimer) { + String streamId, CloudWindmillServiceV1Alpha1Stub stub, ThrottleTimer getDataThrottleTimer) { return GrpcGetDataStream.create( + streamId, stub::getDataStream, grpcBackOff.get(), newStreamObserverFactory(), @@ -238,19 +243,27 @@ public GetDataStream createGetDataStream( public CommitWorkStream createCommitWorkStream( CloudWindmillServiceV1Alpha1Stub stub, ThrottleTimer commitWorkThrottleTimer) { return createCommitWorkStream( - () -> withDefaultDeadline(stub), commitWorkThrottleTimer, newStreamObserverFactory()); + DISPATCHER_STREAM_ID, + () -> withDefaultDeadline(stub), + commitWorkThrottleTimer, + newStreamObserverFactory()); } public CommitWorkStream createDirectCommitWorkStream( - CloudWindmillServiceV1Alpha1Stub stub, ThrottleTimer commitWorkThrottleTimer) { - return createCommitWorkStream(() -> stub, commitWorkThrottleTimer, newStreamObserverFactory()); + String streamId, + CloudWindmillServiceV1Alpha1Stub stub, + ThrottleTimer commitWorkThrottleTimer) { + return createCommitWorkStream( + streamId, () -> stub, commitWorkThrottleTimer, newStreamObserverFactory()); } private CommitWorkStream createCommitWorkStream( + String streamId, Supplier stub, ThrottleTimer commitWorkThrottleTimer, StreamObserverFactory streamObserverFactory) { return GrpcCommitWorkStream.create( + streamId, responseObserver -> stub.get().commitWorkStream(responseObserver), grpcBackOff.get(), streamObserverFactory, diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/StreamingEngineClient.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/StreamingEngineClient.java index 1aed95d74a467..9ab1bb55d1a0b 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/StreamingEngineClient.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/StreamingEngineClient.java @@ -446,6 +446,7 @@ private WindmillStreamSender createAndStartWindmillStreamSenderFor( // GetWorkBudgetDistributor. WindmillStreamSender windmillStreamSender = WindmillStreamSender.create( + connection.backendWorkerToken().orElseGet(() -> ""), connection.stub(), GetWorkRequest.newBuilder() .setClientId(clientId) diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/WindmillStreamSender.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/WindmillStreamSender.java index 9ed78560dfb81..95b9f28a9dc9f 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/WindmillStreamSender.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/WindmillStreamSender.java @@ -67,6 +67,7 @@ public class WindmillStreamSender { private final StreamingEngineThrottleTimers streamingEngineThrottleTimers; private WindmillStreamSender( + String backendWorkerToken, CloudWindmillServiceV1Alpha1Stub stub, GetWorkRequest getWorkRequest, AtomicReference getWorkBudget, @@ -89,19 +90,23 @@ private WindmillStreamSender( Suppliers.memoize( () -> streamingEngineStreamFactory.createGetDataStream( - stub, streamingEngineThrottleTimers.getDataThrottleTimer())); - + backendWorkerToken, + stub, + streamingEngineThrottleTimers.getDataThrottleTimer())); this.commitWorkStream = Suppliers.memoize( () -> streamingEngineStreamFactory.createDirectCommitWorkStream( - stub, streamingEngineThrottleTimers.commitWorkThrottleTimer())); + backendWorkerToken, + stub, + streamingEngineThrottleTimers.commitWorkThrottleTimer())); this.workCommitter = Suppliers.memoize(() -> workCommitterFactory.apply(commitWorkStream.get())); this.getWorkStream = Suppliers.memoize( () -> streamingEngineStreamFactory.createDirectGetWorkStream( + backendWorkerToken, stub, withRequestBudget(getWorkRequest, getWorkBudget.get()), streamingEngineThrottleTimers.getWorkThrottleTimer(), @@ -112,6 +117,7 @@ private WindmillStreamSender( } public static WindmillStreamSender create( + String backendWorkerId, CloudWindmillServiceV1Alpha1Stub stub, GetWorkRequest getWorkRequest, GetWorkBudget getWorkBudget, @@ -123,6 +129,7 @@ public static WindmillStreamSender create( BiFunction> keyedGetDataFn) { return new WindmillStreamSender( + backendWorkerId, stub, getWorkRequest, new AtomicReference<>(getWorkBudget), diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetDataStreamTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetDataStreamTest.java index 414d1e39c1f42..70ad364ac2f2b 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetDataStreamTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetDataStreamTest.java @@ -99,6 +99,7 @@ public void cleanUp() { private GrpcGetDataStream createGetDataStream(GetDataStreamTestStub testStub) { serviceRegistry.addService(testStub); return GrpcGetDataStream.create( + "streamId", responseObserver -> CloudWindmillServiceV1Alpha1Grpc.newStub(inProcessChannel) .getDataStream(responseObserver), diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/StreamingEngineClientTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/StreamingEngineClientTest.java index 23f4d3edaf6d4..5fbbbd3129cb9 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/StreamingEngineClientTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/StreamingEngineClientTest.java @@ -21,6 +21,7 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyString; import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Mockito.atLeast; import static org.mockito.Mockito.mock; @@ -237,6 +238,7 @@ public void testStreamsStartCorrectly() throws InterruptedException { verify(streamFactory, times(2)) .createDirectGetWorkStream( + any(), any(), eq(getWorkRequest(0, 0)), any(), @@ -244,8 +246,7 @@ public void testStreamsStartCorrectly() throws InterruptedException { any(), any(), eq(noOpProcessWorkItemFn())); - - verify(streamFactory, times(2)).createGetDataStream(any(), any()); + verify(streamFactory, times(2)).createGetDataStream(anyString(), any(), any()); verify(streamFactory, times(2)).createCommitWorkStream(any(), any()); } diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/WindmillStreamSenderTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/WindmillStreamSenderTest.java index 8eb62297dbc1a..1d0c6a1de2c30 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/WindmillStreamSenderTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/WindmillStreamSenderTest.java @@ -18,6 +18,7 @@ package org.apache.beam.runners.dataflow.worker.windmill.client.grpc; import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyString; import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; @@ -96,10 +97,10 @@ public void testStartStream_startsAllStreams() { verify(streamFactory) .createDirectGetWorkStream( + anyString(), eq(stub), eq( - GET_WORK_REQUEST - .toBuilder() + GET_WORK_REQUEST.toBuilder() .setMaxItems(itemBudget) .setMaxBytes(byteBudget) .build()), @@ -109,7 +110,7 @@ public void testStartStream_startsAllStreams() { any(), eq(workItemScheduler)); - verify(streamFactory).createGetDataStream(eq(stub), any(ThrottleTimer.class)); + verify(streamFactory).createGetDataStream(anyString(), eq(stub), any(ThrottleTimer.class)); verify(streamFactory).createCommitWorkStream(eq(stub), any(ThrottleTimer.class)); } @@ -128,10 +129,10 @@ public void testStartStream_onlyStartsStreamsOnce() { verify(streamFactory, times(1)) .createDirectGetWorkStream( + anyString(), eq(stub), eq( - GET_WORK_REQUEST - .toBuilder() + GET_WORK_REQUEST.toBuilder() .setMaxItems(itemBudget) .setMaxBytes(byteBudget) .build()), @@ -141,7 +142,8 @@ public void testStartStream_onlyStartsStreamsOnce() { any(), eq(workItemScheduler)); - verify(streamFactory, times(1)).createGetDataStream(eq(stub), any(ThrottleTimer.class)); + verify(streamFactory, times(1)) + .createGetDataStream(anyString(), eq(stub), any(ThrottleTimer.class)); verify(streamFactory, times(1)).createCommitWorkStream(eq(stub), any(ThrottleTimer.class)); } @@ -163,10 +165,10 @@ public void testStartStream_onlyStartsStreamsOnceConcurrent() throws Interrupted verify(streamFactory, times(1)) .createDirectGetWorkStream( + anyString(), eq(stub), eq( - GET_WORK_REQUEST - .toBuilder() + GET_WORK_REQUEST.toBuilder() .setMaxItems(itemBudget) .setMaxBytes(byteBudget) .build()), @@ -176,7 +178,8 @@ public void testStartStream_onlyStartsStreamsOnceConcurrent() throws Interrupted any(), eq(workItemScheduler)); - verify(streamFactory, times(1)).createGetDataStream(eq(stub), any(ThrottleTimer.class)); + verify(streamFactory, times(1)) + .createGetDataStream(anyString(), eq(stub), any(ThrottleTimer.class)); verify(streamFactory, times(1)).createCommitWorkStream(eq(stub), any(ThrottleTimer.class)); } @@ -202,6 +205,7 @@ public void testCloseAllStreams_closesAllStreams() { CommitWorkStream mockCommitWorkStream = mock(CommitWorkStream.class); when(mockStreamFactory.createDirectGetWorkStream( + anyString(), eq(stub), eq(getWorkRequestWithBudget), any(ThrottleTimer.class), @@ -210,9 +214,10 @@ public void testCloseAllStreams_closesAllStreams() { any(), eq(workItemScheduler))) .thenReturn(mockGetWorkStream); - when(mockStreamFactory.createGetDataStream(eq(stub), any(ThrottleTimer.class))) + when(mockStreamFactory.createGetDataStream(anyString(), eq(stub), any(ThrottleTimer.class))) .thenReturn(mockGetDataStream); - when(mockStreamFactory.createDirectCommitWorkStream(eq(stub), any(ThrottleTimer.class))) + when(mockStreamFactory.createDirectCommitWorkStream( + anyString(), eq(stub), any(ThrottleTimer.class))) .thenReturn(mockCommitWorkStream); WindmillStreamSender windmillStreamSender = @@ -235,6 +240,7 @@ private WindmillStreamSender newWindmillStreamSender(GetWorkBudget budget) { private WindmillStreamSender newWindmillStreamSender( GetWorkBudget budget, GrpcWindmillStreamFactory streamFactory) { return WindmillStreamSender.create( + "backendWorkerId", stub, GET_WORK_REQUEST, budget, diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/work/budget/EvenGetWorkBudgetDistributorTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/work/budget/EvenGetWorkBudgetDistributorTest.java index 021d2651e66dc..78ab2881ef0ba 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/work/budget/EvenGetWorkBudgetDistributorTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/work/budget/EvenGetWorkBudgetDistributorTest.java @@ -244,6 +244,7 @@ public void testDistributeBudget_distributesFairlyWhenNotEven() { private WindmillStreamSender createWindmillStreamSender(GetWorkBudget getWorkBudget) { return WindmillStreamSender.create( + "backendWorkerId", stub, Windmill.GetWorkRequest.newBuilder() .setClientId(1L) From e480e82ca13a026a73e53b30a5b32e3c84c8467a Mon Sep 17 00:00:00 2001 From: Martin Trieu Date: Mon, 24 Jun 2024 17:49:26 -0700 Subject: [PATCH 10/13] add stream id, change synchronization in AbstractWindmillStream --- .../client/AbstractWindmillStream.java | 109 +++++++++++++----- .../windmill/client/WindmillStream.java | 51 +++++++- .../client/grpc/GrpcCommitWorkStream.java | 8 +- .../client/grpc/GrpcDirectGetWorkStream.java | 14 ++- .../client/grpc/GrpcGetDataStream.java | 4 +- .../client/grpc/GrpcGetWorkStream.java | 8 +- 6 files changed, 143 insertions(+), 51 deletions(-) diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/AbstractWindmillStream.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/AbstractWindmillStream.java index 7ba241377ac5d..dea3539237b8e 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/AbstractWindmillStream.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/AbstractWindmillStream.java @@ -22,6 +22,7 @@ import java.util.Set; import java.util.concurrent.CountDownLatch; import java.util.concurrent.Executor; +import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; @@ -30,6 +31,7 @@ import java.util.concurrent.atomic.AtomicReference; import java.util.function.Function; import java.util.function.Supplier; +import javax.annotation.concurrent.ThreadSafe; import org.apache.beam.runners.dataflow.worker.windmill.client.grpc.observers.StreamObserverCancelledException; import org.apache.beam.runners.dataflow.worker.windmill.client.grpc.observers.StreamObserverFactory; import org.apache.beam.sdk.util.BackOff; @@ -72,6 +74,7 @@ public abstract class AbstractWindmillStream implements Win private final String streamId; private final AtomicLong lastSendTimeMs; private final Executor executor; + private final ExecutorService sendExecutor; private final BackOff backoff; private final AtomicLong startTimeMs; private final AtomicLong lastResponseTimeMs; @@ -82,10 +85,9 @@ public abstract class AbstractWindmillStream implements Win private final CountDownLatch finishLatch; private final Set> streamRegistry; private final int logEveryNStreamFailures; - private final Supplier> requestObserverSupplier; + private final UpdatableDelegateRequestObserver requestObserver; // Indicates if the current stream in requestObserver is closed by calling close() method private final AtomicBoolean streamClosed; - private @Nullable StreamObserver requestObserver; protected AbstractWindmillStream( Function, StreamObserver> clientFactory, @@ -93,11 +95,18 @@ protected AbstractWindmillStream( StreamObserverFactory streamObserverFactory, Set> streamRegistry, int logEveryNStreamFailures, - String streamId) { - this.streamId = streamId; + String backendWorkerToken) { + this.streamId = + WindmillStream.Id.create(this, backendWorkerToken, backendWorkerToken.isEmpty()).toString(); this.executor = Executors.newSingleThreadExecutor( new ThreadFactoryBuilder().setDaemon(true).setNameFormat(streamId + "-thread").build()); + this.sendExecutor = + Executors.newSingleThreadExecutor( + new ThreadFactoryBuilder() + .setDaemon(true) + .setNameFormat(streamId + "-RequestThread") + .build()); this.backoff = backoff; this.streamRegistry = streamRegistry; this.logEveryNStreamFailures = logEveryNStreamFailures; @@ -111,10 +120,12 @@ protected AbstractWindmillStream( this.lastErrorTime = new AtomicReference<>(); this.sleepUntil = new AtomicLong(); this.finishLatch = new CountDownLatch(1); - this.requestObserverSupplier = - () -> - streamObserverFactory.from( - clientFactory, new AbstractWindmillStream.ResponseObserver()); + this.requestObserver = + new UpdatableDelegateRequestObserver<>( + () -> + streamObserverFactory.from( + clientFactory, + new AbstractWindmillStream.ResponseObserver())); } private static long debugDuration(long nowMs, long startMs) { @@ -140,15 +151,6 @@ private static long debugDuration(long nowMs, long startMs) { */ protected abstract void startThrottleTimer(); - private StreamObserver requestObserver() { - if (requestObserver == null) { - throw new NullPointerException( - "requestObserver cannot be null. Missing a call to startStream() to initialize."); - } - - return requestObserver; - } - /** Send a request to the server. */ protected final void send(RequestT request) { lastSendTimeMs.set(Instant.now().getMillis()); @@ -157,16 +159,22 @@ protected final void send(RequestT request) { throw new IllegalStateException("Send called on a client closed stream."); } - try { - requestObserver().onNext(request); - } catch (StreamObserverCancelledException e) { - if (isClosed()) { - LOG.warn("Stream was closed during send.", e); - return; - } - LOG.error("StreamObserver was unexpectedly cancelled.", e); - throw e; - } + sendExecutor.submit( + () -> { + if (isClosed()) { + return; + } + try { + requestObserver.onNext(request); + } catch (StreamObserverCancelledException e) { + if (isClosed()) { + LOG.warn("Stream was closed during send.", e); + return; + } + LOG.error("StreamObserver was unexpectedly cancelled.", e); + throw e; + } + }); } } @@ -181,7 +189,7 @@ protected final void startStream() { lastResponseTimeMs.set(0); streamClosed.set(false); // lazily initialize the requestObserver. Gets reset whenever the stream is reopened. - requestObserver = requestObserverSupplier.get(); + requestObserver.reset(); onNewStream(); if (clientClosed.get()) { close(); @@ -253,7 +261,7 @@ public final void appendSummaryHtml(PrintWriter writer) { public synchronized void close() { // Synchronization of close and onCompleted necessary for correct retry logic in onNewStream. clientClosed.set(true); - requestObserver().onCompleted(); + requestObserver.onCompleted(); streamClosed.set(true); } @@ -277,6 +285,48 @@ private void setLastError(String error) { lastErrorTime.set(DateTime.now()); } + /** Request observer that allows updating its internal delegate. */ + @ThreadSafe + private static class UpdatableDelegateRequestObserver + implements StreamObserver { + private final Supplier> requestObserverSupplier; + private final AtomicReference> delegateRequestObserver; + + private UpdatableDelegateRequestObserver( + Supplier> requestObserverSupplier) { + this.requestObserverSupplier = requestObserverSupplier; + this.delegateRequestObserver = new AtomicReference<>(); + } + + private synchronized StreamObserver delegate() { + if (delegateRequestObserver.get() == null) { + throw new NullPointerException( + "requestObserver cannot be null. Missing a call to startStream() to initialize."); + } + + return delegateRequestObserver.get(); + } + + private synchronized void reset() { + delegateRequestObserver.set(requestObserverSupplier.get()); + } + + @Override + public void onNext(RequestT requestT) { + delegate().onNext(requestT); + } + + @Override + public void onError(Throwable throwable) { + delegate().onError(throwable); + } + + @Override + public void onCompleted() { + delegate().onCompleted(); + } + } + private class ResponseObserver implements StreamObserver { @Override public void onNext(ResponseT response) { @@ -304,6 +354,7 @@ private void onStreamFinished(@Nullable Throwable t) { if (clientClosed.get() && !hasPendingRequests()) { streamRegistry.remove(AbstractWindmillStream.this); finishLatch.countDown(); + sendExecutor.shutdownNow(); return; } } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/WindmillStream.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/WindmillStream.java index 531586881221c..60b3d3a9bef85 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/WindmillStream.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/WindmillStream.java @@ -17,6 +17,7 @@ */ package org.apache.beam.runners.dataflow.worker.windmill.client; +import com.google.auto.value.AutoValue; import java.io.Closeable; import java.util.List; import java.util.Map; @@ -74,6 +75,12 @@ Windmill.KeyedGetDataResponse requestKeyedData( /** Interface for streaming CommitWorkRequests to Windmill. */ @ThreadSafe interface CommitWorkStream extends WindmillStream { + /** + * Returns a builder that can be used for sending requests. Each builder is not thread-safe but + * different builders for the same stream may be used simultaneously. + */ + Optional newBatcher(); + @NotThreadSafe interface RequestBatcher extends Closeable { /** @@ -96,15 +103,47 @@ default void close() { flush(); } } - - /** - * Returns a builder that can be used for sending requests. Each builder is not thread-safe but - * different builders for the same stream may be used simultaneously. - */ - Optional newBatcher(); } /** Interface for streaming GetWorkerMetadata requests to Windmill. */ @ThreadSafe interface GetWorkerMetadataStream extends WindmillStream {} + + @AutoValue + abstract class Id { + public static Id create(WindmillStream stream, String backendWorkerToken, boolean isDirect) { + return new AutoValue_WindmillStream_Id( + Id.getStreamType(stream), backendWorkerToken, isDirect); + } + + private static String getStreamType(WindmillStream windmillStream) { + if (windmillStream instanceof GetWorkStream) { + return "GetWork-"; + } else if (windmillStream instanceof GetWorkerMetadataStream) { + return "GetWorkerMetadata-"; + } else if (windmillStream instanceof GetDataStream) { + return "GetData-"; + } else if (windmillStream instanceof CommitWorkStream) { + return "CommitWork-"; + } + + // Should not happen conditions above are exhaustive. + throw new IllegalArgumentException("Unknown stream type."); + } + + abstract String streamType(); + + abstract String backendWorkerToken(); + + abstract boolean isDirect(); + + @Override + public String toString() { + return String.format( + "[%s]-[%s]-[%s]", + streamType(), + isDirect() ? "direct" : "dispatched", + backendWorkerToken().isEmpty() ? "no_worker_token" : backendWorkerToken()); + } + } } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcCommitWorkStream.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcCommitWorkStream.java index c2bea55e3a4b8..7450ab58f9425 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcCommitWorkStream.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcCommitWorkStream.java @@ -62,7 +62,7 @@ public final class GrpcCommitWorkStream private final int streamingRpcBatchLimit; private GrpcCommitWorkStream( - String streamId, + String backendWorkerToken, Function, StreamObserver> startCommitWorkRpcFn, BackOff backoff, @@ -79,7 +79,7 @@ private GrpcCommitWorkStream( streamObserverFactory, streamRegistry, logEveryNStreamFailures, - streamId); + backendWorkerToken); pendingRequests = new ConcurrentHashMap<>(); this.idGenerator = idGenerator; this.jobHeader = jobHeader; @@ -88,7 +88,7 @@ private GrpcCommitWorkStream( } public static GrpcCommitWorkStream create( - String streamId, + String backendWorkerToken, Function, StreamObserver> startCommitWorkRpcFn, BackOff backoff, @@ -101,7 +101,7 @@ public static GrpcCommitWorkStream create( int streamingRpcBatchLimit) { GrpcCommitWorkStream commitWorkStream = new GrpcCommitWorkStream( - "CommitWorkStream-" + streamId, + backendWorkerToken, startCommitWorkRpcFn, backoff, streamObserverFactory, diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcDirectGetWorkStream.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcDirectGetWorkStream.java index 83bbf3eac7455..24b53adc6d3df 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcDirectGetWorkStream.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcDirectGetWorkStream.java @@ -100,7 +100,7 @@ public final class GrpcDirectGetWorkStream private final ConcurrentMap workItemBuffers; private GrpcDirectGetWorkStream( - String streamId, + String backendWorkerToken, Function< StreamObserver, StreamObserver> @@ -122,7 +122,7 @@ private GrpcDirectGetWorkStream( streamObserverFactory, streamRegistry, logEveryNStreamFailures, - streamId); + backendWorkerToken); this.request = request; this.getWorkThrottleTimer = getWorkThrottleTimer; this.workItemScheduler = workItemScheduler; @@ -138,7 +138,7 @@ private GrpcDirectGetWorkStream( } public static GrpcDirectGetWorkStream create( - String streamId, + String backendWorkerToken, Function< StreamObserver, StreamObserver> @@ -156,7 +156,7 @@ public static GrpcDirectGetWorkStream create( WorkItemScheduler workItemScheduler) { GrpcDirectGetWorkStream getWorkStream = new GrpcDirectGetWorkStream( - "DirectGetWorkStream-" + streamId, + backendWorkerToken, startGetWorkRpcFn, request, backoff, @@ -278,8 +278,10 @@ protected void startThrottleTimer() { } @Override - public synchronized void adjustBudget(long itemsDelta, long bytesDelta) { - nextBudgetAdjustment.set(nextBudgetAdjustment.get().apply(itemsDelta, bytesDelta)); + public void adjustBudget(long itemsDelta, long bytesDelta) { + synchronized (this) { + nextBudgetAdjustment.set(nextBudgetAdjustment.get().apply(itemsDelta, bytesDelta)); + } sendRequestExtension(); } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetDataStream.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetDataStream.java index 98e3a9f207634..2242f40bc84f6 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetDataStream.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetDataStream.java @@ -107,7 +107,7 @@ private GrpcGetDataStream( } public static GrpcGetDataStream create( - String streamId, + String backendWorkerToken, Function, StreamObserver> startGetDataRpcFn, BackOff backoff, @@ -122,7 +122,7 @@ public static GrpcGetDataStream create( Consumer> processHeartbeatResponses) { GrpcGetDataStream getDataStream = new GrpcGetDataStream( - "GetDataStream" + streamId, + backendWorkerToken, startGetDataRpcFn, backoff, streamObserverFactory, diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetWorkStream.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetWorkStream.java index 5c19ef61629ae..5fc093ee32aa9 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetWorkStream.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetWorkStream.java @@ -60,7 +60,7 @@ public final class GrpcGetWorkStream private final AtomicLong inflightBytes; private GrpcGetWorkStream( - String streamId, + String backendWorkerToken, Function< StreamObserver, StreamObserver> @@ -78,7 +78,7 @@ private GrpcGetWorkStream( streamObserverFactory, streamRegistry, logEveryNStreamFailures, - streamId); + backendWorkerToken); this.request = request; this.getWorkThrottleTimer = getWorkThrottleTimer; this.receiver = receiver; @@ -88,7 +88,7 @@ private GrpcGetWorkStream( } public static GrpcGetWorkStream create( - String streamId, + String backendWorkerToken, Function< StreamObserver, StreamObserver> @@ -102,7 +102,7 @@ public static GrpcGetWorkStream create( WorkItemReceiver receiver) { GrpcGetWorkStream getWorkStream = new GrpcGetWorkStream( - streamId, + backendWorkerToken, startGetWorkRpcFn, request, backoff, From ba1d134595589f4ff155212810434d8a9ce01f34 Mon Sep 17 00:00:00 2001 From: Martin Trieu Date: Tue, 25 Jun 2024 16:28:30 -0700 Subject: [PATCH 11/13] add more visibility via debug capture pages --- .../worker/streaming/ActiveWorkState.java | 12 +- .../dataflow/worker/streaming/Work.java | 24 ++- .../harness/StreamingWorkerStatusPages.java | 6 +- .../client/AbstractWindmillStream.java | 91 ++++++---- .../windmill/client/WindmillStream.java | 28 +-- .../grpc/GetWorkTimingInfosTracker.java | 24 ++- .../client/grpc/GrpcCommitWorkStream.java | 25 ++- .../client/grpc/GrpcDirectGetWorkStream.java | 101 ++++++++++- .../client/grpc/GrpcGetDataStream.java | 71 ++++++-- .../grpc/GrpcGetWorkerMetadataStream.java | 28 ++- .../grpc/GrpcWindmillStreamFactory.java | 2 +- .../grpc/GrpcGetWorkerMetadataStreamTest.java | 21 ++- .../client/grpc/GrpcWindmillServerTest.java | 169 ++---------------- 13 files changed, 336 insertions(+), 266 deletions(-) diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/ActiveWorkState.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/ActiveWorkState.java index 0597bdf6c836c..e677c80e098c0 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/ActiveWorkState.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/ActiveWorkState.java @@ -327,7 +327,15 @@ synchronized void printActiveWork(PrintWriter writer, Instant now) { ""); writer.println( - ""); + "" + + "" + + "" + + "" + + "" + + "" + + "" + + "" + + ""); // Use StringBuilder because we are appending in loop. StringBuilder activeWorkStatus = new StringBuilder(); int commitsPendingCount = 0; @@ -353,6 +361,8 @@ synchronized void printActiveWork(PrintWriter writer, Instant now) { activeWorkStatus.append(activeWork.getState()); activeWorkStatus.append("\n"); } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/Work.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/Work.java index 92f681071d6b0..d374171086403 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/Work.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/Work.java @@ -147,7 +147,8 @@ public static ProcessingContext createProcessingContext( BiFunction getKeyedDataFn, Consumer workCommitter, HeartbeatSender heartbeatSender) { - return ProcessingContext.create(computationId, getKeyedDataFn, workCommitter, heartbeatSender); + return ProcessingContext.create(computationId, getKeyedDataFn, workCommitter, heartbeatSender) + .build(); } private static LatencyAttribution.Builder createLatencyAttributionWithActiveLatencyBreakdown( @@ -344,6 +345,10 @@ public boolean isFailed() { return isFailed; } + public String backendWorkerToken() { + return processingContext.backendWorkerToken(); + } + boolean isStuckCommittingAt(Instant stuckCommitDeadline) { return currentState.state() == Work.State.COMMITTING && currentState.startTime().isBefore(stuckCommitDeadline); @@ -435,21 +440,24 @@ private boolean isCommitPending() { @AutoValue public abstract static class ProcessingContext { + private static final String UNKNOWN_BACKEND_WORKER_TOKEN = "UNKNOWN"; - private static ProcessingContext create( + private static ProcessingContext.Builder create( String computationId, BiFunction getKeyedDataFn, Consumer workCommitter, HeartbeatSender heartbeatSender) { return new AutoValue_Work_ProcessingContext.Builder() + .setBackendWorkerToken(UNKNOWN_BACKEND_WORKER_TOKEN) .setComputationId(computationId) .setHeartbeatSender(heartbeatSender) .setWorkCommitter(workCommitter) .setKeyedDataFetcher( - request -> Optional.ofNullable(getKeyedDataFn.apply(computationId, request))) - .build(); + request -> Optional.ofNullable(getKeyedDataFn.apply(computationId, request))); } + abstract String backendWorkerToken(); + /** Computation that the {@link Work} belongs to. */ public abstract String computationId(); @@ -465,10 +473,12 @@ private static ProcessingContext create( public abstract HeartbeatSender heartbeatSender(); - abstract Builder toBuilder(); + public abstract Builder toBuilder(); @AutoValue.Builder - abstract static class Builder { + public abstract static class Builder { + public abstract Builder setBackendWorkerToken(String value); + abstract Builder setComputationId(String value); abstract Builder setKeyedDataFetcher( @@ -478,7 +488,7 @@ abstract Builder setKeyedDataFetcher( abstract Builder setHeartbeatSender(HeartbeatSender value); - abstract ProcessingContext build(); + public abstract ProcessingContext build(); } } } 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 1788479856b44..0b876378b102b 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 @@ -135,7 +135,11 @@ public void start(DataflowWorkerHarnessOptions options) { statusPages.addStatusDataProvider( "exception", "Last Exception", new LastExceptionDataProvider()); statusPages.addStatusDataProvider("cache", "State Cache", stateCache); - statusPages.addStatusDataProvider("activeGetWorkBudget", "Active GetWork Budget", writer -> {}); + statusPages.addStatusDataProvider( + "activeGetWorkBudget", + "Active GetWork Budget", + writer -> + writer.format("%s", computationStateCache.totalCurrentActiveGetWorkBudget())); if (isStreamingEngine()) { addStreamingEngineStatusPages(); diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/AbstractWindmillStream.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/AbstractWindmillStream.java index dea3539237b8e..9089587e0173f 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/AbstractWindmillStream.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/AbstractWindmillStream.java @@ -22,7 +22,6 @@ import java.util.Set; import java.util.concurrent.CountDownLatch; import java.util.concurrent.Executor; -import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; @@ -71,10 +70,10 @@ public abstract class AbstractWindmillStream implements Win protected static final int RPC_STREAM_CHUNK_SIZE = 2 << 20; private static final Logger LOG = LoggerFactory.getLogger(AbstractWindmillStream.class); protected final AtomicBoolean clientClosed; - private final String streamId; + protected final WindmillStream.Id streamId; private final AtomicLong lastSendTimeMs; private final Executor executor; - private final ExecutorService sendExecutor; + private final Executor requestSender; private final BackOff backoff; private final AtomicLong startTimeMs; private final AtomicLong lastResponseTimeMs; @@ -97,11 +96,11 @@ protected AbstractWindmillStream( int logEveryNStreamFailures, String backendWorkerToken) { this.streamId = - WindmillStream.Id.create(this, backendWorkerToken, backendWorkerToken.isEmpty()).toString(); + WindmillStream.Id.create(this, backendWorkerToken, !backendWorkerToken.isEmpty()); this.executor = Executors.newSingleThreadExecutor( new ThreadFactoryBuilder().setDaemon(true).setNameFormat(streamId + "-thread").build()); - this.sendExecutor = + this.requestSender = Executors.newSingleThreadExecutor( new ThreadFactoryBuilder() .setDaemon(true) @@ -128,11 +127,8 @@ protected AbstractWindmillStream( new AbstractWindmillStream.ResponseObserver())); } - private static long debugDuration(long nowMs, long startMs) { - if (startMs <= 0) { - return -1; - } - return Math.max(0, nowMs - startMs); + private static String debugDuration(long nowMs, long startMs) { + return (startMs <= 0 ? -1 : Math.max(0, nowMs - startMs)) + "ms"; } /** Called on each response from the server. */ @@ -159,7 +155,7 @@ protected final void send(RequestT request) { throw new IllegalStateException("Send called on a client closed stream."); } - sendExecutor.submit( + requestSender.execute( () -> { if (isClosed()) { return; @@ -225,33 +221,65 @@ public final synchronized void maybeSendHealthCheck(Instant lastSendThreshold) { protected abstract void sendHealthCheck(); - // Care is taken that synchronization on this is unnecessary for all status page information. - // Blocking sends are made beneath this stream object's lock which could block status page - // rendering. + /** + * @implNote Care is taken that synchronization on this is unnecessary for all status page + * information.Blocking sends are made beneath this stream object's lock which could block + * status page rendering. + */ public final void appendSummaryHtml(PrintWriter writer) { + writer.format("

%s:

", streamId); appendSpecificHtml(writer); - writer.format("id: %s; ", streamId); + writer.println("Status:"); + writer.println( + "
KeyTokenQueuedActive ForStateState Active For
KeyTokenQueuedActive ForStateState Active ForBackend Worker Token
"); activeWorkStatus.append(elapsedString(activeWork.getStateStartTime(), now)); + activeWorkStatus.append(""); + activeWorkStatus.append(activeWork.backendWorkerToken()); activeWorkStatus.append("
"); + writer.println( + "" + + "" + + "" + + "" + + "" + + "" + + "" + + "" + + "" + + "" + + ""); + + StringBuilder statusString = new StringBuilder(); + statusString.append(""); + statusString.append("\n"); + writer.print(statusString); + writer.println("
Error CountLast ErrorLast Error Received TimeIs Client ClosedIs ClosedBackOff RemainingCurrent Stream Age MillisLast Request Sent TimeLast Received Response Time
"); if (errorCount.get() > 0) { - writer.format( - ", %d errors, last error [ %s ] at [%s]", - errorCount.get(), lastError.get(), lastErrorTime.get()); - } - if (clientClosed.get()) { - writer.write(", client closed"); + statusString.append(errorCount.get()); + statusString.append(""); + statusString.append(lastError.get()); + statusString.append(""); + statusString.append(lastErrorTime.get()); + } else { + statusString.append(0); + statusString.append(""); + statusString.append("N/A"); + statusString.append(""); + statusString.append("N/A"); } + statusString.append(""); - writer.format(", isClosed=[%s]", isClosed()); + statusString.append(clientClosed.get()); + statusString.append(""); + statusString.append(isClosed()); + statusString.append(""); long nowMs = Instant.now().getMillis(); long sleepLeft = sleepUntil.get() - nowMs; - if (sleepLeft > 0) { - writer.format(", %dms backoff remaining", sleepLeft); - } - writer.format( - ", current stream is %dms old, last send %dms, last response %dms, closed: %s", - debugDuration(nowMs, startTimeMs.get()), - debugDuration(nowMs, lastSendTimeMs.get()), - debugDuration(nowMs, lastResponseTimeMs.get()), - streamClosed.get()); + statusString.append(Math.max(sleepLeft, 0)); + statusString.append(""); + statusString.append(debugDuration(nowMs, startTimeMs.get())); + statusString.append(""); + statusString.append(debugDuration(nowMs, lastSendTimeMs.get())); + statusString.append(""); + statusString.append(debugDuration(nowMs, lastResponseTimeMs.get())); + statusString.append("
"); } // Don't require synchronization on stream, see the appendSummaryHtml comment. @@ -354,7 +382,6 @@ private void onStreamFinished(@Nullable Throwable t) { if (clientClosed.get() && !hasPendingRequests()) { streamRegistry.remove(AbstractWindmillStream.this); finishLatch.countDown(); - sendExecutor.shutdownNow(); return; } } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/WindmillStream.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/WindmillStream.java index 60b3d3a9bef85..76df0df08e469 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/WindmillStream.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/WindmillStream.java @@ -111,20 +111,25 @@ interface GetWorkerMetadataStream extends WindmillStream {} @AutoValue abstract class Id { - public static Id create(WindmillStream stream, String backendWorkerToken, boolean isDirect) { + private static final String GET_WORK_STREAM_TYPE = "GetWorkStream"; + private static final String GET_DATA_STREAM_TYPE = "GetDataStream"; + private static final String GET_WORKER_METADATA_STREAM_TYPE = "GetWorkerMetadataStream"; + private static final String COMMIT_WORK_STREAM_TYPE = "CommitWorkStream"; + + static Id create(WindmillStream stream, String backendWorkerToken, boolean isDirect) { return new AutoValue_WindmillStream_Id( Id.getStreamType(stream), backendWorkerToken, isDirect); } private static String getStreamType(WindmillStream windmillStream) { if (windmillStream instanceof GetWorkStream) { - return "GetWork-"; + return GET_WORK_STREAM_TYPE; } else if (windmillStream instanceof GetWorkerMetadataStream) { - return "GetWorkerMetadata-"; + return GET_WORKER_METADATA_STREAM_TYPE; } else if (windmillStream instanceof GetDataStream) { - return "GetData-"; + return GET_DATA_STREAM_TYPE; } else if (windmillStream instanceof CommitWorkStream) { - return "CommitWork-"; + return COMMIT_WORK_STREAM_TYPE; } // Should not happen conditions above are exhaustive. @@ -133,17 +138,16 @@ private static String getStreamType(WindmillStream windmillStream) { abstract String streamType(); - abstract String backendWorkerToken(); + public abstract String backendWorkerToken(); abstract boolean isDirect(); @Override - public String toString() { - return String.format( - "[%s]-[%s]-[%s]", - streamType(), - isDirect() ? "direct" : "dispatched", - backendWorkerToken().isEmpty() ? "no_worker_token" : backendWorkerToken()); + public final String toString() { + String id = String.format("%s-%s", streamType(), isDirect() ? "Direct" : "Dispatched"); + return backendWorkerToken().isEmpty() + ? id + String.format("-[%s]", backendWorkerToken()) + : id; } } } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GetWorkTimingInfosTracker.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GetWorkTimingInfosTracker.java index 8e70ef03158ec..e3c2ec886640f 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GetWorkTimingInfosTracker.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GetWorkTimingInfosTracker.java @@ -24,6 +24,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Optional; import javax.annotation.Nullable; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.GetWorkStreamTimingInfo; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.GetWorkStreamTimingInfo.Event; @@ -170,6 +171,22 @@ List getLatencyAttributions() { return latencyAttributions; } + Map getAggregatedGetWorkStreamLatencies() { + return Collections.unmodifiableMap(aggregatedGetWorkStreamLatencies); + } + + Instant getWorkItemCreationEndTime() { + return workItemCreationEndTime; + } + + Instant getWorkItemLastChunkReceivedByWorkerTime() { + return workItemLastChunkReceivedByWorkerTime; + } + + Optional getWorkItemCreationLatency() { + return Optional.ofNullable(workItemCreationLatency); + } + void reset() { this.aggregatedGetWorkStreamLatencies.clear(); this.workItemCreationEndTime = Instant.EPOCH; @@ -177,7 +194,7 @@ void reset() { this.workItemCreationLatency = null; } - private static class SumAndMaxDurations { + static class SumAndMaxDurations { private Duration sum; private Duration max; @@ -185,5 +202,10 @@ private SumAndMaxDurations(Duration sum, Duration max) { this.sum = sum; this.max = max; } + + @Override + public String toString() { + return "Latency{" + "sum=" + sum + ", max=" + max + '}'; + } } } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcCommitWorkStream.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcCommitWorkStream.java index 7450ab58f9425..83d23ba6ad33f 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcCommitWorkStream.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcCommitWorkStream.java @@ -117,7 +117,7 @@ public static GrpcCommitWorkStream create( @Override public void appendSpecificHtml(PrintWriter writer) { - writer.format("CommitWorkStream: %d pending", pendingRequests.size()); + writer.format("
%d Pending Requests
", pendingRequests.size()); } @Override @@ -161,7 +161,7 @@ public void sendHealthCheck() { @Override protected void onResponse(StreamingCommitResponse response) { commitWorkThrottleTimer.stop(); - @Nullable RuntimeException finalException = null; + @Nullable RuntimeException failure = null; for (int i = 0; i < response.getRequestIdCount() && !isClosed(); ++i) { long requestId = response.getRequestId(i); if (requestId == HEARTBEAT_REQUEST_ID) { @@ -169,26 +169,23 @@ protected void onResponse(StreamingCommitResponse response) { } @Nullable PendingRequest pendingRequest = pendingRequests.remove(requestId); if (pendingRequest == null) { - LOG.warn( - "Got unknown commit request ID: [{}] in response: [{}]. Current pending requests: {}", - requestId, - response, - pendingRequests); + LOG.warn("Got unknown commit request ID: [{}].", requestId); } else { try { - CommitStatus commitStatus = - i < response.getStatusCount() ? response.getStatus(i) : CommitStatus.OK; - pendingRequest.ackResponse(commitStatus); + pendingRequest.ackResponse( + i < response.getStatusCount() ? response.getStatus(i) : CommitStatus.OK); } catch (RuntimeException e) { // Catch possible exceptions to ensure that an exception for one commit does not prevent - // other commits from being processed. + // other commits from being processed. Aggregate all the failures to throw after + // processing the response if they exist. LOG.warn("Exception while processing commit response.", e); - finalException = e; + if (failure == null) failure = e; + else failure.addSuppressed(e); } } } - if (finalException != null) { - throw finalException; + if (failure != null) { + throw failure; } } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcDirectGetWorkStream.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcDirectGetWorkStream.java index 24b53adc6d3df..fc536f9c51715 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcDirectGetWorkStream.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcDirectGetWorkStream.java @@ -21,6 +21,7 @@ import java.io.IOException; import java.io.InputStream; import java.io.PrintWriter; +import java.util.Map; import java.util.Optional; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; @@ -38,6 +39,7 @@ import org.apache.beam.runners.dataflow.worker.windmill.Windmill.GetWorkRequest; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.KeyedGetDataRequest; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.KeyedGetDataResponse; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill.LatencyAttribution.State; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.StreamingGetWorkRequest; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.StreamingGetWorkResponseChunk; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.WorkItem; @@ -189,6 +191,11 @@ private static Optional parseWorkItem(InputStream serializedWorkItem) } } + private static String newHtmlTableHeader() { + return ""; + } + private synchronized GetWorkBudget getThenResetBudgetAdjustment() { return nextBudgetAdjustment.getAndUpdate(unused -> GetWorkBudget.noBudget()); } @@ -246,10 +253,83 @@ protected boolean hasPendingRequests() { @Override public void appendSpecificHtml(PrintWriter writer) { - // Number of buffers is same as distinct workers that sent work on this stream. - writer.format( - "DirectGetWorkStream: %d buffers, %s inflight budget allowed.", - workItemBuffers.size(), inFlightBudget.get()); + writer.format("In-flight Budget: %s
", inFlightBudget.get()); + writer.format("Next Budget Adjustment: %s
", inFlightBudget.get()); + writer.format("Pending Response Budget: %s
", pendingResponseBudget.get()); + writer.println("WorkItemBuffers:"); + writer.println(newHtmlTableHeader()); + writer.println( + "" + + "" + + "" + + "" + + "" + + "" + + "" + + "" + + "" + + ""); + StringBuilder statusString = new StringBuilder(); + for (Map.Entry buffer : workItemBuffers.entrySet()) { + statusString.append(""); + statusString.append("" + + "" + + "" + + ""); + statusString.append(""); + statusString.append("\n"); + } + statusString.append("
Stream IDComputationInput Data WatermarkSynchronized Processing TimeWorkItem Create End TimeLast WorkItem Chunk Received TimeWorkItem Creation LatencyAggregate Latency Table
"); + statusString.append(buffer.getKey()); + statusString.append(""); + @Nullable ComputationMetadata metadata = buffer.getValue().metadata; + if (metadata != null) { + statusString.append(metadata.computationId()); + statusString.append(""); + statusString.append(metadata.inputDataWatermark()); + statusString.append(""); + statusString.append(metadata.synchronizedProcessingTime()); + } else { + statusString.append("N/A"); + statusString.append(""); + statusString.append("N/A"); + statusString.append(""); + statusString.append("N/A"); + } + statusString.append(""); + statusString.append(buffer.getValue().workTimingInfosTracker.getWorkItemCreationEndTime()); + statusString.append(""); + statusString.append( + buffer.getValue().workTimingInfosTracker.getWorkItemLastChunkReceivedByWorkerTime()); + statusString.append(""); + statusString.append( + buffer + .getValue() + .workTimingInfosTracker + .getWorkItemCreationLatency() + .map(Windmill.LatencyAttribution::toString) + .orElse("N/A")); + statusString.append(""); + Map aggregatedGetWorkStreamLatencies = + buffer.getValue().workTimingInfosTracker.getAggregatedGetWorkStreamLatencies(); + if (!aggregatedGetWorkStreamLatencies.isEmpty()) { + statusString.append(newHtmlTableHeader()); + for (Map.Entry latencyEntry : + aggregatedGetWorkStreamLatencies.entrySet()) { + statusString.append( + "
Latency StateLatency Sum and Max Duration
"); + statusString.append(latencyEntry.getKey()); + statusString.append(""); + statusString.append(latencyEntry.getValue()); + statusString.append("
"); + } else { + statusString.append("N/A"); + } + statusString.append("\n"); + } + + writer.print(statusString); + writer.println(""); } @Override @@ -355,11 +435,14 @@ private void scheduleWorkItem(WorkItem workItem) { workItem, createWatermarks(workItem, metadata), Work.createProcessingContext( - metadata.computationId(), - (computation, request) -> - keyedGetDataFn.apply(getDataStream.get()).apply(computation, request), - workCommitter.get()::commit, - DirectHeartbeatSender.create(getDataStream.get())), + metadata.computationId(), + (computation, request) -> + keyedGetDataFn.apply(getDataStream.get()).apply(computation, request), + workCommitter.get()::commit, + DirectHeartbeatSender.create(getDataStream.get())) + .toBuilder() + .setBackendWorkerToken(streamId.backendWorkerToken()) + .build(), // After the work item is successfully queued or dropped by ActiveWorkState, remove it // from the pendingResponseBudget. queuedWorkItem -> updatePendingResponseBudget(-1, -workItem.getSerializedSize()), diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetDataStream.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetDataStream.java index 2242f40bc84f6..7c88b3d29243e 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetDataStream.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetDataStream.java @@ -55,6 +55,7 @@ import org.apache.beam.runners.dataflow.worker.windmill.client.throttling.ThrottleTimer; import org.apache.beam.sdk.util.BackOff; import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.StreamObserver; +import org.joda.time.Duration; import org.joda.time.Instant; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -138,6 +139,12 @@ public static GrpcGetDataStream create( return getDataStream; } + private static String elapsedString(Instant start, Instant end) { + Duration activeFor = new Duration(start, end); + // Duration's toString always starts with "PT"; remove that here. + return activeFor.toString().substring(2); + } + @Override protected synchronized void onNewStream() { send(StreamingGetDataRequest.newBuilder().setHeader(jobHeader).build()); @@ -290,26 +297,54 @@ public void sendHealthCheck() { @Override public void appendSpecificHtml(PrintWriter writer) { - writer.format( - "GetDataStream: %d queued batches, %d pending requests [", batches.size(), pending.size()); - for (Map.Entry entry : pending.entrySet()) { - writer.format("Stream %d ", entry.getKey()); - if (entry.getValue().isCancelled()) { - writer.append("cancelled "); - } - if (entry.getValue().isComplete()) { - writer.append("complete "); - } - int queueSize = entry.getValue().size(); - if (queueSize > 0) { - writer.format("%d queued responses ", queueSize); - } - long blockedMs = entry.getValue().getBlockedStartMs(); - if (blockedMs > 0) { - writer.format("blocked for %dms", Instant.now().getMillis() - blockedMs); + writer.format("
%d queued batches, %d pending requests", batches.size(), pending.size()); + writer.println( + ""); + writer.println( + "" + + "" + + "" + + "" + + "" + + "" + + ""); + + StringBuilder statusString = new StringBuilder(); + if (pending.isEmpty()) { + statusString.append(""); + statusString.append("\n"); + } else { + for (Map.Entry entry : pending.entrySet()) { + statusString.append(""); + statusString.append("\n"); } } - writer.append("]"); + + writer.print(statusString); + writer.println("
Request IDIs Cancelled?Is Complete?Queued ResponsesBlocked For
"); + statusString.append("N/A"); + statusString.append(""); + statusString.append("N/A"); + statusString.append(""); + statusString.append("N/A"); + statusString.append(""); + statusString.append("N/A"); + statusString.append(""); + statusString.append("N/A"); + statusString.append("
"); + statusString.append(entry.getKey()); + statusString.append(""); + statusString.append(entry.getValue().isCancelled()); + statusString.append(""); + statusString.append(entry.getValue().isComplete()); + statusString.append(""); + statusString.append(entry.getValue().size()); + statusString.append(""); + statusString.append( + elapsedString( + Instant.ofEpochMilli(entry.getValue().getBlockedStartMs()), Instant.now())); + statusString.append("
"); } private ResponseT issueRequest(QueuedRequest request, ParseFn parseFn) { diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetWorkerMetadataStream.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetWorkerMetadataStream.java index 0459aa503d55c..f8593ee5a5d7e 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetWorkerMetadataStream.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetWorkerMetadataStream.java @@ -48,10 +48,10 @@ public final class GrpcGetWorkerMetadataStream private final Object metadataLock; @GuardedBy("metadataLock") - private long metadataVersion; + private volatile long metadataVersion; @GuardedBy("metadataLock") - private WorkerMetadataResponse latestResponse; + private volatile WorkerMetadataResponse latestResponse; private GrpcGetWorkerMetadataStream( Function, StreamObserver> @@ -70,7 +70,7 @@ private GrpcGetWorkerMetadataStream( streamObserverFactory, streamRegistry, logEveryNStreamFailures, - "GetWorkerMetadata"); + ""); this.workerMetadataRequest = WorkerMetadataRequest.newBuilder().setHeader(jobHeader).build(); this.metadataVersion = metadataVersion; this.getWorkerMetadataThrottleTimer = getWorkerMetadataThrottleTimer; @@ -162,12 +162,24 @@ protected void sendHealthCheck() { send(HEALTH_CHECK_REQUEST); } + @SuppressWarnings("GuardedBy") // This is for debugging purposes, don't block processing threads. @Override protected void appendSpecificHtml(PrintWriter writer) { - synchronized (metadataLock) { - writer.format( - "GetWorkerMetadataStream: version=[%d] , job_header=[%s], latest_response=[%s]", - this.metadataVersion, workerMetadataRequest.getHeader(), this.latestResponse); - } + writer.println( + ""); + writer.println( + "" + "" + "" + "" + ""); + String statusString = + "" + + "\n"; + writer.print(statusString); + writer.println("
VersionJobLatest Response
" + + metadataVersion + + "" + + workerMetadataRequest.getHeader() + + "" + + latestResponse + + "
"); } } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcWindmillStreamFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcWindmillStreamFactory.java index 5b4f4191f6b07..df18965f512ac 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcWindmillStreamFactory.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcWindmillStreamFactory.java @@ -298,7 +298,7 @@ private StreamObserverFactory newStreamObserverFactory() { @Override public void appendSummaryHtml(PrintWriter writer) { - writer.write("Active Streams:
"); + writer.write("

Active Streams:


"); for (AbstractWindmillStream stream : streamRegistry) { stream.appendSummaryHtml(writer); writer.write("
"); diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetWorkerMetadataStreamTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetWorkerMetadataStreamTest.java index 515beba0c88d4..e33a0a531c792 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetWorkerMetadataStreamTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetWorkerMetadataStreamTest.java @@ -30,6 +30,7 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.concurrent.CountDownLatch; import java.util.function.Consumer; import java.util.stream.Collectors; import javax.annotation.Nullable; @@ -266,13 +267,14 @@ public void testGetWorkerMetadata_correctlyAddsAndRemovesStreamFromRegistry() { } @Test - public void testSendHealthCheck() { + public void testSendHealthCheck() throws InterruptedException { + CountDownLatch onRequestReceived = new CountDownLatch(1); TestGetWorkMetadataRequestObserver requestObserver = - Mockito.spy(new TestGetWorkMetadataRequestObserver()); + Mockito.spy(new TestGetWorkMetadataRequestObserver(onRequestReceived::countDown)); GetWorkerMetadataTestStub testStub = new GetWorkerMetadataTestStub(requestObserver); stream = getWorkerMetadataTestStream(testStub, 0, new TestWindmillEndpointsConsumer()); stream.sendHealthCheck(); - + onRequestReceived.await(); verify(requestObserver).onNext(WorkerMetadataRequest.getDefaultInstance()); } @@ -308,9 +310,20 @@ private void injectWorkerMetadata(WorkerMetadataResponse response) { private static class TestGetWorkMetadataRequestObserver implements StreamObserver { private @Nullable StreamObserver responseObserver; + private final Runnable onRequestReceived; + + private TestGetWorkMetadataRequestObserver(Runnable onRequestReceived) { + this.onRequestReceived = onRequestReceived; + } + + private TestGetWorkMetadataRequestObserver() { + this.onRequestReceived = () -> {}; + } @Override - public void onNext(WorkerMetadataRequest workerMetadataRequest) {} + public void onNext(WorkerMetadataRequest workerMetadataRequest) { + onRequestReceived.run(); + } @Override public void onError(Throwable throwable) {} diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcWindmillServerTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcWindmillServerTest.java index 4666339043e09..b2540a63d2649 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcWindmillServerTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcWindmillServerTest.java @@ -36,8 +36,6 @@ import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicInteger; import org.apache.beam.runners.dataflow.worker.windmill.CloudWindmillServiceV1Alpha1Grpc.CloudWindmillServiceV1Alpha1ImplBase; import org.apache.beam.runners.dataflow.worker.windmill.Windmill; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.CommitStatus; @@ -725,132 +723,6 @@ public StreamObserver commitWorkStream( executor.shutdown(); } - @Test - // Tests stream retries on server errors before and after `close()` - public void testStreamingCommitClosedStream() throws Exception { - List commitRequestList = new ArrayList<>(); - List latches = new ArrayList<>(); - Map commitRequests = new ConcurrentHashMap<>(); - AtomicBoolean shouldServerReturnError = new AtomicBoolean(true); - AtomicBoolean isClientClosed = new AtomicBoolean(false); - AtomicInteger errorsBeforeClose = new AtomicInteger(); - AtomicInteger errorsAfterClose = new AtomicInteger(); - for (int i = 0; i < 500; ++i) { - // Build some requests of varying size with a few big ones. - WorkItemCommitRequest request = makeCommitRequest(i, i * (i < 480 ? 8 : 128)); - commitRequestList.add(request); - commitRequests.put((long) i, request); - latches.add(new CountDownLatch(1)); - } - Collections.shuffle(commitRequestList); - - // This server returns errors if shouldServerReturnError is true, else returns valid responses. - serviceRegistry.addService( - new CloudWindmillServiceV1Alpha1ImplBase() { - @Override - public StreamObserver commitWorkStream( - StreamObserver responseObserver) { - StreamObserver testCommitStreamObserver = - getTestCommitStreamObserver(responseObserver, commitRequests); - return new StreamObserver() { - @Override - public void onNext(StreamingCommitWorkRequest request) { - if (shouldServerReturnError.get()) { - try { - responseObserver.onError( - new RuntimeException("shouldServerReturnError = true")); - if (isClientClosed.get()) { - errorsAfterClose.incrementAndGet(); - } else { - errorsBeforeClose.incrementAndGet(); - } - } catch (IllegalStateException e) { - // The stream is already closed. - } - } else { - testCommitStreamObserver.onNext(request); - } - } - - @Override - public void onError(Throwable throwable) { - testCommitStreamObserver.onError(throwable); - } - - @Override - public void onCompleted() { - testCommitStreamObserver.onCompleted(); - } - }; - } - }); - - // Make the commit requests, waiting for each of them to be verified and acknowledged. - CommitWorkStream stream = client.commitWorkStream(); - try (CommitWorkStream.RequestBatcher batcher = - stream.newBatcher().orElseThrow(IllegalStateException::new)) { - for (int i = 0; i < commitRequestList.size(); ) { - final CountDownLatch latch = latches.get(i); - if (batcher.commitWorkItem( - "computation", - commitRequestList.get(i), - (CommitStatus status) -> { - assertEquals(status, CommitStatus.OK); - latch.countDown(); - })) { - i++; - } else { - batcher.flush(); - } - } - } - - long deadline = System.currentTimeMillis() + 60_000; // 1 min - while (true) { - Thread.sleep(100); - int tmpErrorsBeforeClose = errorsBeforeClose.get(); - // wait for at least 1 error before close - if (tmpErrorsBeforeClose > 0) { - break; - } - if (System.currentTimeMillis() > deadline) { - // Control should not reach here if the test is working as expected - fail( - String.format( - "Expected errors not sent by server errorsBeforeClose: %s" - + " \n Should not reach here if the test is working as expected.", - tmpErrorsBeforeClose)); - } - } - - stream.close(); - isClientClosed.set(true); - - deadline = System.currentTimeMillis() + 60_000; // 1 min - while (true) { - Thread.sleep(100); - int tmpErrorsAfterClose = errorsAfterClose.get(); - // wait for at least 1 error after close - if (tmpErrorsAfterClose > 0) { - break; - } - if (System.currentTimeMillis() > deadline) { - // Control should not reach here if the test is working as expected - fail( - String.format( - "Expected errors not sent by server errorsAfterClose: %s" - + " \n Should not reach here if the test is working as expected.", - tmpErrorsAfterClose)); - } - } - - shouldServerReturnError.set(false); - for (CountDownLatch latch : latches) { - assertTrue(latch.await(1, TimeUnit.MINUTES)); - } - assertTrue(stream.awaitTermination(30, TimeUnit.SECONDS)); - } - private List makeGetDataHeartbeatRequest(List keys) { List result = new ArrayList<>(); for (String key : keys) { @@ -880,7 +752,8 @@ private List makeHeartbeatRequest(List keys) { @Test public void testStreamingGetDataHeartbeatsAsKeyedGetDataRequests() throws Exception { // This server records the heartbeats observed but doesn't respond. - final Map> getDataHeartbeats = new HashMap<>(); + Map> getDataHeartbeats = new HashMap<>(); + CountDownLatch heartbeatReceived = new CountDownLatch(1); serviceRegistry.addService( new CloudWindmillServiceV1Alpha1ImplBase() { @@ -921,6 +794,8 @@ public void onNext(StreamingGetDataRequest chunk) { .add(request.getRequestsList().get(0)); } } + + heartbeatReceived.countDown(); } } catch (Exception e) { errorCollector.addError(e); @@ -956,20 +831,10 @@ public void onCompleted() { GetDataStream stream = client.getDataStream(); stream.refreshActiveWork(heartbeatsToRefresh); + heartbeatReceived.await(); stream.close(); assertTrue(stream.awaitTermination(60, TimeUnit.SECONDS)); - - boolean receivedAllGetDataHeartbeats = false; - while (!receivedAllGetDataHeartbeats) { - Thread.sleep(100); - synchronized (getDataHeartbeats) { - if (getDataHeartbeats.size() != expectedKeyedGetDataRequests.size()) { - continue; - } - assertEquals(expectedKeyedGetDataRequests, getDataHeartbeats); - receivedAllGetDataHeartbeats = true; - } - } + assertEquals(expectedKeyedGetDataRequests, getDataHeartbeats); } @Test @@ -991,7 +856,7 @@ public void testStreamingGetDataHeartbeatsAsHeartbeatRequests() throws Exception () -> WindmillChannelFactory.inProcessChannel("TestServer"))); // This server records the heartbeats observed but doesn't respond. final List receivedHeartbeats = new ArrayList<>(); - + CountDownLatch heartbeatsReceieved = new CountDownLatch(1); serviceRegistry.addService( new CloudWindmillServiceV1Alpha1ImplBase() { @Override @@ -1022,10 +887,8 @@ public void onNext(StreamingGetDataRequest chunk) { errorCollector.checkThat( chunk.getSerializedSize(), Matchers.lessThanOrEqualTo(STREAM_CHUNK_SIZE)); errorCollector.checkThat(chunk.getRequestIdCount(), Matchers.is(0)); - - synchronized (receivedHeartbeats) { - receivedHeartbeats.addAll(chunk.getComputationHeartbeatRequestList()); - } + receivedHeartbeats.addAll(chunk.getComputationHeartbeatRequestList()); + heartbeatsReceieved.countDown(); } } catch (Exception e) { errorCollector.addError(e); @@ -1072,20 +935,10 @@ public void onCompleted() { GetDataStream stream = client.getDataStream(); stream.refreshActiveWork(heartbeatRequestMap); + heartbeatsReceieved.await(); stream.close(); assertTrue(stream.awaitTermination(60, TimeUnit.SECONDS)); - - boolean receivedAllHeartbeatRequests = false; - while (!receivedAllHeartbeatRequests) { - Thread.sleep(100); - synchronized (receivedHeartbeats) { - if (receivedHeartbeats.size() != expectedHeartbeats.size()) { - continue; - } - assertEquals(expectedHeartbeats, receivedHeartbeats); - receivedAllHeartbeatRequests = true; - } - } + assertEquals(expectedHeartbeats, receivedHeartbeats); } @Test From 50b5da69a993fb1938b4bb246ad8c1d250e474c3 Mon Sep 17 00:00:00 2001 From: Martin Trieu Date: Sat, 29 Jun 2024 23:14:02 -0700 Subject: [PATCH 12/13] add shutdown boolean to handle the stream being explicitly closed. once the stream is shutdown, no more interactions should be performed on the stream. --- .../MetricTrackingWindmillServerStub.java | 2 +- .../worker/streaming/ActiveWorkState.java | 7 +- .../dataflow/worker/streaming/Work.java | 2 +- .../client/AbstractWindmillStream.java | 78 ++++++++--- .../windmill/client/WindmillStream.java | 15 +- .../client/grpc/GrpcCommitWorkStream.java | 10 +- .../client/grpc/GrpcDirectGetWorkStream.java | 4 +- .../client/grpc/GrpcGetDataStream.java | 10 +- .../grpc/GrpcWindmillStreamFactory.java | 45 ++++-- .../client/grpc/StreamingEngineClient.java | 3 +- .../client/grpc/WindmillStreamSender.java | 8 +- .../grpc/observers/DirectStreamObserver.java | 12 +- .../work/refresh/ActiveWorkRefresher.java | 12 +- .../work/refresh/DirectHeartbeatSender.java | 10 +- .../dataflow/worker/FakeWindmillServer.java | 42 ++++++ .../client/WindmillStreamPoolTest.java | 18 ++- .../StreamingEngineWorkCommitterTest.java | 18 ++- .../client/grpc/GrpcWindmillServerTest.java | 128 ++++++++++++++++++ .../grpc/StreamingEngineClientTest.java | 4 +- .../client/grpc/WindmillStreamSenderTest.java | 48 ++++--- 20 files changed, 381 insertions(+), 95 deletions(-) diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/MetricTrackingWindmillServerStub.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/MetricTrackingWindmillServerStub.java index 995b074ffe17a..fbe61a3fb662e 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/MetricTrackingWindmillServerStub.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/MetricTrackingWindmillServerStub.java @@ -236,7 +236,7 @@ public Windmill.KeyedGetDataResponse getStateData( public Windmill.KeyedGetDataResponse getStateData( GetDataStream getDataStream, String computation, Windmill.KeyedGetDataRequest request) { gcThrashingMonitor.waitForResources("GetStateData"); - if (getDataStream.isClosed()) { + if (getDataStream.isShutdown()) { throw new WorkItemCancelledException(request.getShardingKey()); } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/ActiveWorkState.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/ActiveWorkState.java index e677c80e098c0..0a8ad1a99c0d5 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/ActiveWorkState.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/ActiveWorkState.java @@ -286,10 +286,11 @@ private synchronized ImmutableMap getStuckCommitsAt( if (executableWork != null) { Work work = executableWork.work(); if (work.isStuckCommittingAt(stuckCommitDeadline)) { - LOG.error( - "Detected key {} stuck in COMMITTING state since {}, completing it with error.", + LOG.warn( + "Detected key {} stuck in COMMITTING state since {} originating from {} worker, invalidating.", shardedKey, - work.getStateStartTime()); + work.getStateStartTime(), + work.backendWorkerToken()); stuckCommits.put(shardedKey, work.id()); } } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/Work.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/Work.java index d374171086403..d69371ee03a9d 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/Work.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/Work.java @@ -440,7 +440,7 @@ private boolean isCommitPending() { @AutoValue public abstract static class ProcessingContext { - private static final String UNKNOWN_BACKEND_WORKER_TOKEN = "UNKNOWN"; + private static final String UNKNOWN_BACKEND_WORKER_TOKEN = ""; private static ProcessingContext.Builder create( String computationId, diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/AbstractWindmillStream.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/AbstractWindmillStream.java index 9089587e0173f..2782741ecf679 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/AbstractWindmillStream.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/AbstractWindmillStream.java @@ -22,7 +22,9 @@ import java.util.Set; import java.util.concurrent.CountDownLatch; import java.util.concurrent.Executor; +import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; +import java.util.concurrent.RejectedExecutionException; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; @@ -70,10 +72,10 @@ public abstract class AbstractWindmillStream implements Win protected static final int RPC_STREAM_CHUNK_SIZE = 2 << 20; private static final Logger LOG = LoggerFactory.getLogger(AbstractWindmillStream.class); protected final AtomicBoolean clientClosed; - protected final WindmillStream.Id streamId; + private final WindmillStream.Id streamId; private final AtomicLong lastSendTimeMs; private final Executor executor; - private final Executor requestSender; + private final ExecutorService requestSender; private final BackOff backoff; private final AtomicLong startTimeMs; private final AtomicLong lastResponseTimeMs; @@ -87,6 +89,7 @@ public abstract class AbstractWindmillStream implements Win private final UpdatableDelegateRequestObserver requestObserver; // Indicates if the current stream in requestObserver is closed by calling close() method private final AtomicBoolean streamClosed; + private final AtomicBoolean isShutdown; protected AbstractWindmillStream( Function, StreamObserver> clientFactory, @@ -119,6 +122,7 @@ protected AbstractWindmillStream( this.lastErrorTime = new AtomicReference<>(); this.sleepUntil = new AtomicLong(); this.finishLatch = new CountDownLatch(1); + this.isShutdown = new AtomicBoolean(false); this.requestObserver = new UpdatableDelegateRequestObserver<>( () -> @@ -131,6 +135,11 @@ private static String debugDuration(long nowMs, long startMs) { return (startMs <= 0 ? -1 : Math.max(0, nowMs - startMs)) + "ms"; } + @Override + public final Id id() { + return streamId; + } + /** Called on each response from the server. */ protected abstract void onResponse(ResponseT response); @@ -147,7 +156,13 @@ private static String debugDuration(long nowMs, long startMs) { */ protected abstract void startThrottleTimer(); - /** Send a request to the server. */ + /** + * Send a request to the server. + * + * @implNote Requests are sent on a dedicated RequestThread via {@link #requestSender} to not + * block external callers from closing/shutting down the stream. + */ + @SuppressWarnings("FutureReturnValueIgnored") protected final void send(RequestT request) { lastSendTimeMs.set(Instant.now().getMillis()); synchronized (this) { @@ -155,22 +170,29 @@ protected final void send(RequestT request) { throw new IllegalStateException("Send called on a client closed stream."); } - requestSender.execute( - () -> { - if (isClosed()) { - return; - } - try { - requestObserver.onNext(request); - } catch (StreamObserverCancelledException e) { - if (isClosed()) { - LOG.warn("Stream was closed during send.", e); + try { + requestSender.submit( + () -> { + if (isShutdown()) { return; } - LOG.error("StreamObserver was unexpectedly cancelled.", e); - throw e; - } - }); + try { + requestObserver.onNext(request); + } catch (StreamObserverCancelledException e) { + if (isClosed() || isShutdown()) { + LOG.warn("Stream was closed or shutdown during send.", e); + return; + } + LOG.error("StreamObserver was unexpectedly cancelled.", e); + throw e; + } + }); + } catch (RejectedExecutionException e) { + LOG.warn( + "{} is shutdown and will not send or receive any more requests or responses.", + streamId, + e); + } } } @@ -178,7 +200,7 @@ protected final void send(RequestT request) { protected final void startStream() { // Add the stream to the registry after it has been fully constructed. streamRegistry.add(this); - while (true) { + while (!isShutdown()) { try { synchronized (this) { startTimeMs.set(Instant.now().getMillis()); @@ -236,6 +258,7 @@ public final void appendSummaryHtml(PrintWriter writer) { writer.println( "" + "Error Count" + + "Is Shutdown" + "Last Error" + "Last Error Received Time" + "Is Client Closed" @@ -252,6 +275,8 @@ public final void appendSummaryHtml(PrintWriter writer) { if (errorCount.get() > 0) { statusString.append(errorCount.get()); statusString.append(""); + statusString.append(isShutdown.get()); + statusString.append(""); statusString.append(lastError.get()); statusString.append(""); statusString.append(lastErrorTime.get()); @@ -286,13 +311,26 @@ public final void appendSummaryHtml(PrintWriter writer) { protected abstract void appendSpecificHtml(PrintWriter writer); @Override - public synchronized void close() { + public final synchronized void close() { // Synchronization of close and onCompleted necessary for correct retry logic in onNewStream. clientClosed.set(true); requestObserver.onCompleted(); streamClosed.set(true); } + @Override + public synchronized void shutdown() { + if (isShutdown.compareAndSet(false, true)) { + close(); + requestSender.shutdownNow(); + } + } + + @Override + public boolean isShutdown() { + return isShutdown.get(); + } + @Override public final boolean awaitTermination(int time, TimeUnit unit) throws InterruptedException { return finishLatch.await(time, unit); @@ -379,7 +417,7 @@ public void onCompleted() { private void onStreamFinished(@Nullable Throwable t) { synchronized (this) { - if (clientClosed.get() && !hasPendingRequests()) { + if (isShutdown() || (clientClosed.get() && !hasPendingRequests())) { streamRegistry.remove(AbstractWindmillStream.this); finishLatch.countDown(); return; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/WindmillStream.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/WindmillStream.java index 76df0df08e469..1614e468f8e36 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/WindmillStream.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/WindmillStream.java @@ -34,6 +34,8 @@ /** Superclass for streams returned by streaming Windmill methods. */ @ThreadSafe public interface WindmillStream { + Id id(); + /** Indicates that no more requests will be sent. */ void close(); @@ -46,6 +48,15 @@ public interface WindmillStream { /** Reflects that {@link #close()} was explicitly called. */ boolean isClosed(); + /** + * Shutdown the stream. Logically closes the stream and terminates the stream. The stream instance + * should not have any interactions after this point. + */ + void shutdown(); + + /** Indicates that the stream is shutdown and should not be used. */ + boolean isShutdown(); + /** Handle representing a stream of GetWork responses. */ @ThreadSafe interface GetWorkStream extends WindmillStream { @@ -116,7 +127,7 @@ abstract class Id { private static final String GET_WORKER_METADATA_STREAM_TYPE = "GetWorkerMetadataStream"; private static final String COMMIT_WORK_STREAM_TYPE = "CommitWorkStream"; - static Id create(WindmillStream stream, String backendWorkerToken, boolean isDirect) { + public static Id create(WindmillStream stream, String backendWorkerToken, boolean isDirect) { return new AutoValue_WindmillStream_Id( Id.getStreamType(stream), backendWorkerToken, isDirect); } @@ -145,7 +156,7 @@ private static String getStreamType(WindmillStream windmillStream) { @Override public final String toString() { String id = String.format("%s-%s", streamType(), isDirect() ? "Direct" : "Dispatched"); - return backendWorkerToken().isEmpty() + return !backendWorkerToken().isEmpty() ? id + String.format("-[%s]", backendWorkerToken()) : id; } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcCommitWorkStream.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcCommitWorkStream.java index 83d23ba6ad33f..a9e053a9feacd 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcCommitWorkStream.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcCommitWorkStream.java @@ -141,7 +141,7 @@ protected synchronized void onNewStream() { */ @Override public Optional newBatcher() { - return isClosed() ? Optional.empty() : Optional.of(new Batcher()); + return isShutdown() ? Optional.empty() : Optional.of(new Batcher()); } @Override @@ -162,7 +162,7 @@ public void sendHealthCheck() { protected void onResponse(StreamingCommitResponse response) { commitWorkThrottleTimer.stop(); @Nullable RuntimeException failure = null; - for (int i = 0; i < response.getRequestIdCount() && !isClosed(); ++i) { + for (int i = 0; i < response.getRequestIdCount() && !isShutdown(); ++i) { long requestId = response.getRequestId(i); if (requestId == HEARTBEAT_REQUEST_ID) { continue; @@ -195,9 +195,9 @@ protected void startThrottleTimer() { } private void flushInternal(Map requests) { - boolean isClosed = isClosed(); - if (requests.isEmpty() || isClosed) { - LOG.debug("Skipping commit stream flush. isClosed={}", isClosed); + boolean isShutdown = isShutdown(); + if (requests.isEmpty() || isShutdown) { + LOG.debug("Skipping commit stream flush. isShutdown={}", isShutdown); return; } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcDirectGetWorkStream.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcDirectGetWorkStream.java index fc536f9c51715..ef3eb668f4694 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcDirectGetWorkStream.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcDirectGetWorkStream.java @@ -226,7 +226,7 @@ private void sendRequestExtension() { @Override protected synchronized void onNewStream() { workItemBuffers.clear(); - if (!isClosed()) { + if (!isShutdown()) { // Add the current in-flight budget to the next adjustment. Only positive values are allowed // here with negatives defaulting to 0, since GetWorkBudgets cannot be created with negative // values. @@ -441,7 +441,7 @@ private void scheduleWorkItem(WorkItem workItem) { workCommitter.get()::commit, DirectHeartbeatSender.create(getDataStream.get())) .toBuilder() - .setBackendWorkerToken(streamId.backendWorkerToken()) + .setBackendWorkerToken(id().backendWorkerToken()) .build(), // After the work item is successfully queued or dropped by ActiveWorkState, remove it // from the pendingResponseBudget. diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetDataStream.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetDataStream.java index 7c88b3d29243e..81fbef9f84e4f 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetDataStream.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetDataStream.java @@ -161,8 +161,8 @@ protected synchronized void onNewStream() { } @Override - public synchronized void close() { - super.close(); + public synchronized void shutdown() { + super.shutdown(); // Stream has been explicitly closed. Drain pending input streams and request batches. // Future calls to send RPCs will fail. @@ -350,7 +350,7 @@ public void appendSpecificHtml(PrintWriter writer) { private ResponseT issueRequest(QueuedRequest request, ParseFn parseFn) { while (true) { // Handle stream closure during loop. - if (isClosed()) { + if (isShutdown() || isClosed()) { throw new WindmillStreamClosedException( "Cannot send request=[" + request + "] on closed stream."); } @@ -360,13 +360,13 @@ private ResponseT issueRequest(QueuedRequest request, ParseFn withDefaultDeadline(stub).getWorkStream(responseObserver), request, grpcBackOff.get(), @@ -197,7 +195,7 @@ public GetWorkStream createGetWorkStream( } public GetWorkStream createDirectGetWorkStream( - String streamId, + String backendWorkerToken, CloudWindmillServiceV1Alpha1Stub stub, GetWorkRequest request, ThrottleTimer getWorkThrottleTimer, @@ -209,7 +207,7 @@ public GetWorkStream createDirectGetWorkStream( keyedGetDataFn, WorkItemScheduler workItemScheduler) { return GrpcDirectGetWorkStream.create( - streamId, + backendWorkerToken, stub::getWorkStream, request, grpcBackOff.get(), @@ -224,9 +222,28 @@ public GetWorkStream createDirectGetWorkStream( } public GetDataStream createGetDataStream( - String streamId, CloudWindmillServiceV1Alpha1Stub stub, ThrottleTimer getDataThrottleTimer) { + CloudWindmillServiceV1Alpha1Stub stub, ThrottleTimer getDataThrottleTimer) { + return GrpcGetDataStream.create( + NO_BACKEND_WORKER_TOKEN, + responseObserver -> withDefaultDeadline(stub).getDataStream(responseObserver), + grpcBackOff.get(), + newStreamObserverFactory(), + streamRegistry, + logEveryNStreamFailures, + getDataThrottleTimer, + jobHeader, + streamIdGenerator, + streamingRpcBatchLimit, + sendKeyedGetDataRequests, + processHeartbeatResponses); + } + + public GetDataStream createDirectGetDataStream( + String backendWorkerToken, + CloudWindmillServiceV1Alpha1Stub stub, + ThrottleTimer getDataThrottleTimer) { return GrpcGetDataStream.create( - streamId, + backendWorkerToken, stub::getDataStream, grpcBackOff.get(), newStreamObserverFactory(), @@ -243,27 +260,27 @@ public GetDataStream createGetDataStream( public CommitWorkStream createCommitWorkStream( CloudWindmillServiceV1Alpha1Stub stub, ThrottleTimer commitWorkThrottleTimer) { return createCommitWorkStream( - DISPATCHER_STREAM_ID, + NO_BACKEND_WORKER_TOKEN, () -> withDefaultDeadline(stub), commitWorkThrottleTimer, newStreamObserverFactory()); } public CommitWorkStream createDirectCommitWorkStream( - String streamId, + String backendWorkerToken, CloudWindmillServiceV1Alpha1Stub stub, ThrottleTimer commitWorkThrottleTimer) { return createCommitWorkStream( - streamId, () -> stub, commitWorkThrottleTimer, newStreamObserverFactory()); + backendWorkerToken, () -> stub, commitWorkThrottleTimer, newStreamObserverFactory()); } private CommitWorkStream createCommitWorkStream( - String streamId, + String backendWorkerToken, Supplier stub, ThrottleTimer commitWorkThrottleTimer, StreamObserverFactory streamObserverFactory) { return GrpcCommitWorkStream.create( - streamId, + backendWorkerToken, responseObserver -> stub.get().commitWorkStream(responseObserver), grpcBackOff.get(), streamObserverFactory, diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/StreamingEngineClient.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/StreamingEngineClient.java index 9ab1bb55d1a0b..0ae02606cb5e2 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/StreamingEngineClient.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/StreamingEngineClient.java @@ -33,7 +33,6 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; import java.util.function.BiFunction; -import java.util.function.Consumer; import java.util.function.Function; import java.util.function.Supplier; import java.util.stream.Collectors; @@ -287,7 +286,7 @@ private void startWorkerMetadataConsumer() { @VisibleForTesting public synchronized void finish() { Preconditions.checkState(started, "StreamingEngineClient never started."); - getWorkerMetadataStream.get().close(); + getWorkerMetadataStream.get().shutdown(); getWorkBudgetRefresher.stop(); newWorkerMetadataPublisher.shutdownNow(); newWorkerMetadataConsumer.shutdownNow(); diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/WindmillStreamSender.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/WindmillStreamSender.java index 95b9f28a9dc9f..22b2c804274e8 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/WindmillStreamSender.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/WindmillStreamSender.java @@ -89,7 +89,7 @@ private WindmillStreamSender( this.getDataStream = Suppliers.memoize( () -> - streamingEngineStreamFactory.createGetDataStream( + streamingEngineStreamFactory.createDirectGetDataStream( backendWorkerToken, stub, streamingEngineThrottleTimers.getDataThrottleTimer())); @@ -158,10 +158,10 @@ void closeAllStreams() { // streaming RPCs by possibly making calls over the network. Do not close the streams unless // they have already been started. if (started.get()) { - getWorkStream.get().close(); - getDataStream.get().close(); + getWorkStream.get().shutdown(); + getDataStream.get().shutdown(); workCommitter.get().stop(); - commitWorkStream.get().close(); + commitWorkStream.get().shutdown(); } } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/observers/DirectStreamObserver.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/observers/DirectStreamObserver.java index 1a69215695e16..09d518b86a21f 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/observers/DirectStreamObserver.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/observers/DirectStreamObserver.java @@ -148,7 +148,7 @@ private boolean isTerminated() { @Override public void onError(Throwable t) { synchronized (lock) { - isReadyNotifier.arriveAndDeregister(); + unblockIsReadyNotifier(); outboundObserver.onError(t); } } @@ -156,8 +156,16 @@ public void onError(Throwable t) { @Override public void onCompleted() { synchronized (lock) { - isReadyNotifier.arriveAndDeregister(); + unblockIsReadyNotifier(); outboundObserver.onCompleted(); } } + + private void unblockIsReadyNotifier() { + try { + isReadyNotifier.arriveAndDeregister(); + } catch (IllegalStateException e) { + // onCompleted or onError has already been called and waiting threads have been freed. + } + } } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/ActiveWorkRefresher.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/ActiveWorkRefresher.java index 9e235fa4e75c3..04bc4954d71e9 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/ActiveWorkRefresher.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/ActiveWorkRefresher.java @@ -30,6 +30,7 @@ import org.apache.beam.runners.dataflow.worker.DataflowExecutionStateSampler; import org.apache.beam.runners.dataflow.worker.streaming.ComputationState; import org.apache.beam.runners.dataflow.worker.windmill.Windmill; +import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableListMultimap; import org.joda.time.Duration; import org.joda.time.Instant; @@ -44,13 +45,14 @@ * threshold is determined by {@link #activeWorkRefreshPeriodMillis} */ @ThreadSafe -public class ActiveWorkRefresher { +@Internal +public final class ActiveWorkRefresher { private static final Logger LOG = LoggerFactory.getLogger(ActiveWorkRefresher.class); - protected final Supplier clock; - protected final int activeWorkRefreshPeriodMillis; - protected final Supplier> computations; - protected final DataflowExecutionStateSampler sampler; + private final Supplier clock; + private final int activeWorkRefreshPeriodMillis; + private final Supplier> computations; + private final DataflowExecutionStateSampler sampler; private final int stuckCommitDurationMillis; private final ScheduledExecutorService activeWorkRefreshExecutor; private final Consumer>>> diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/DirectHeartbeatSender.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/DirectHeartbeatSender.java index c827cccdf88fe..9c3e39ce69872 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/DirectHeartbeatSender.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/DirectHeartbeatSender.java @@ -55,12 +55,12 @@ public static DirectHeartbeatSender create(GetDataStream getDataStream) { @Override public void sendHeartbeats(Map> heartbeats) { - if (getDataStream.isClosed()) { + if (getDataStream.isShutdown() || getDataStream.isClosed()) { LOG.warn( - "Trying to refresh work on stream={} after work has moved off of worker." - + " heartbeats={}", - getDataStream, - heartbeats); + "Trying to refresh work w/ {} heartbeats on stream={} after work has moved off of worker." + + " heartbeats", + getDataStream.id(), + heartbeats.size()); onStreamClosed.run(); } else { getDataStream.refreshActiveWork(heartbeats); diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/FakeWindmillServer.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/FakeWindmillServer.java index 2188052e03642..6180bbe2e4e52 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/FakeWindmillServer.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/FakeWindmillServer.java @@ -61,6 +61,7 @@ import org.apache.beam.runners.dataflow.worker.windmill.Windmill.LatencyAttribution.State; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.WorkItemCommitRequest; import org.apache.beam.runners.dataflow.worker.windmill.WindmillServerStub; +import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStream; import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStream.CommitWorkStream; import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStream.GetDataStream; import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStream.GetWorkStream; @@ -231,6 +232,12 @@ public GetWorkStream getWorkStream(Windmill.GetWorkRequest request, WorkItemRece Instant startTime = Instant.now(); final CountDownLatch done = new CountDownLatch(1); return new GetWorkStream() { + + @Override + public Id id() { + return WindmillStream.Id.create(this, "backend_worker_token", false); + } + @Override public void close() { done.countDown(); @@ -292,6 +299,16 @@ public Instant startTime() { public boolean isClosed() { return done.getCount() == 0; } + + @Override + public void shutdown() { + close(); + } + + @Override + public boolean isShutdown() { + return isClosed(); + } }; } @@ -299,6 +316,11 @@ public boolean isClosed() { public GetDataStream getDataStream() { Instant startTime = Instant.now(); return new GetDataStream() { + @Override + public Id id() { + return WindmillStream.Id.create(this, "backend_worker_token", false); + } + @Override public Windmill.KeyedGetDataResponse requestKeyedData( String computation, KeyedGetDataRequest request) { @@ -369,6 +391,14 @@ public Instant startTime() { public boolean isClosed() { return false; } + + @Override + public void shutdown() {} + + @Override + public boolean isShutdown() { + return false; + } }; } @@ -376,6 +406,10 @@ public boolean isClosed() { public CommitWorkStream commitWorkStream() { Instant startTime = Instant.now(); return new CommitWorkStream() { + @Override + public Id id() { + return WindmillStream.Id.create(this, "backend_worker_token", false); + } @Override public Optional newBatcher() { @@ -460,6 +494,14 @@ public Instant startTime() { public boolean isClosed() { return false; } + + @Override + public void shutdown() {} + + @Override + public boolean isShutdown() { + return false; + } }; } diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/WindmillStreamPoolTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/WindmillStreamPoolTest.java index 01f7391dd67f3..e4dcec84cff1c 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/WindmillStreamPoolTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/WindmillStreamPoolTest.java @@ -22,6 +22,7 @@ import static org.junit.Assert.assertNotSame; import static org.junit.Assert.assertThrows; import static org.junit.Assert.assertTrue; +import static org.mockito.Mockito.mock; import java.util.List; import java.util.concurrent.ConcurrentHashMap; @@ -38,12 +39,12 @@ @RunWith(JUnit4.class) public class WindmillStreamPoolTest { - @Rule public transient Timeout globalTimeout = Timeout.seconds(600); private static final int DEFAULT_NUM_STREAMS = 10; private static final int NEW_STREAM_HOLDS = 2; private final ConcurrentHashMap< TestWindmillStream, WindmillStreamPool.StreamData> holds = new ConcurrentHashMap<>(); + @Rule public transient Timeout globalTimeout = Timeout.seconds(600); private List> streams; @Before @@ -236,6 +237,11 @@ private TestWindmillStream(Instant startTime) { this.closed = false; } + @Override + public Id id() { + return Id.create(mock(GetWorkStream.class), "backend_worker_token", false); + } + @Override public void close() { closed = true; @@ -255,5 +261,15 @@ public Instant startTime() { public boolean isClosed() { return closed; } + + @Override + public void shutdown() { + close(); + } + + @Override + public boolean isShutdown() { + return closed; + } } } diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/commits/StreamingEngineWorkCommitterTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/commits/StreamingEngineWorkCommitterTest.java index 053cbc9573c1b..5d89ea679ac50 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/commits/StreamingEngineWorkCommitterTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/commits/StreamingEngineWorkCommitterTest.java @@ -47,6 +47,7 @@ import org.apache.beam.runners.dataflow.worker.windmill.Windmill; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.WorkItemCommitRequest; import org.apache.beam.runners.dataflow.worker.windmill.client.CloseableStream; +import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStream; import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStream.CommitWorkStream; import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStreamPool; import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; @@ -259,9 +260,14 @@ public void testStop_drainsCommitQueue() { new CommitWorkStream() { private boolean closed = false; + @Override + public Id id() { + return WindmillStream.Id.create(this, "backend_worker_token", false); + } + @Override public Optional newBatcher() { - return isClosed() + return isShutdown() ? Optional.empty() : Optional.of( new RequestBatcher() { @@ -283,6 +289,16 @@ public void close() { closed = true; } + @Override + public void shutdown() { + close(); + } + + @Override + public boolean isShutdown() { + return closed; + } + @Override public boolean awaitTermination(int time, TimeUnit unit) { return false; diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcWindmillServerTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcWindmillServerTest.java index b2540a63d2649..7cb1b1aa6396f 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcWindmillServerTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcWindmillServerTest.java @@ -36,6 +36,8 @@ import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; import org.apache.beam.runners.dataflow.worker.windmill.CloudWindmillServiceV1Alpha1Grpc.CloudWindmillServiceV1Alpha1ImplBase; import org.apache.beam.runners.dataflow.worker.windmill.Windmill; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.CommitStatus; @@ -723,6 +725,132 @@ public StreamObserver commitWorkStream( executor.shutdown(); } + @Test + // Tests stream retries on server errors before and after `close()` + public void testStreamingCommitClosedStream() throws Exception { + List commitRequestList = new ArrayList<>(); + List latches = new ArrayList<>(); + Map commitRequests = new ConcurrentHashMap<>(); + AtomicBoolean shouldServerReturnError = new AtomicBoolean(true); + AtomicBoolean isClientClosed = new AtomicBoolean(false); + AtomicInteger errorsBeforeClose = new AtomicInteger(); + AtomicInteger errorsAfterClose = new AtomicInteger(); + for (int i = 0; i < 500; ++i) { + // Build some requests of varying size with a few big ones. + WorkItemCommitRequest request = makeCommitRequest(i, i * (i < 480 ? 8 : 128)); + commitRequestList.add(request); + commitRequests.put((long) i, request); + latches.add(new CountDownLatch(1)); + } + Collections.shuffle(commitRequestList); + + // This server returns errors if shouldServerReturnError is true, else returns valid responses. + serviceRegistry.addService( + new CloudWindmillServiceV1Alpha1ImplBase() { + @Override + public StreamObserver commitWorkStream( + StreamObserver responseObserver) { + StreamObserver testCommitStreamObserver = + getTestCommitStreamObserver(responseObserver, commitRequests); + return new StreamObserver() { + @Override + public void onNext(StreamingCommitWorkRequest request) { + if (shouldServerReturnError.get()) { + try { + responseObserver.onError( + new RuntimeException("shouldServerReturnError = true")); + if (isClientClosed.get()) { + errorsAfterClose.incrementAndGet(); + } else { + errorsBeforeClose.incrementAndGet(); + } + } catch (IllegalStateException e) { + // The stream is already closed. + } + } else { + testCommitStreamObserver.onNext(request); + } + } + + @Override + public void onError(Throwable throwable) { + testCommitStreamObserver.onError(throwable); + } + + @Override + public void onCompleted() { + testCommitStreamObserver.onCompleted(); + } + }; + } + }); + + // Make the commit requests, waiting for each of them to be verified and acknowledged. + CommitWorkStream stream = client.commitWorkStream(); + try (CommitWorkStream.RequestBatcher batcher = + stream.newBatcher().orElseThrow(IllegalStateException::new)) { + for (int i = 0; i < commitRequestList.size(); ) { + final CountDownLatch latch = latches.get(i); + if (batcher.commitWorkItem( + "computation", + commitRequestList.get(i), + (CommitStatus status) -> { + assertEquals(status, CommitStatus.OK); + latch.countDown(); + })) { + i++; + } else { + batcher.flush(); + } + } + } + + long deadline = System.currentTimeMillis() + 60_000; // 1 min + while (true) { + Thread.sleep(100); + int tmpErrorsBeforeClose = errorsBeforeClose.get(); + // wait for at least 1 error before close + if (tmpErrorsBeforeClose > 0) { + break; + } + if (System.currentTimeMillis() > deadline) { + // Control should not reach here if the test is working as expected + fail( + String.format( + "Expected errors not sent by server errorsBeforeClose: %s" + + " \n Should not reach here if the test is working as expected.", + tmpErrorsBeforeClose)); + } + } + + stream.close(); + isClientClosed.set(true); + + deadline = System.currentTimeMillis() + 60_000; // 1 min + while (true) { + Thread.sleep(100); + int tmpErrorsAfterClose = errorsAfterClose.get(); + // wait for at least 1 error after close + if (tmpErrorsAfterClose > 0) { + break; + } + if (System.currentTimeMillis() > deadline) { + // Control should not reach here if the test is working as expected + fail( + String.format( + "Expected errors not sent by server errorsAfterClose: %s" + + " \n Should not reach here if the test is working as expected.", + tmpErrorsAfterClose)); + } + } + + shouldServerReturnError.set(false); + for (CountDownLatch latch : latches) { + assertTrue(latch.await(1, TimeUnit.MINUTES)); + } + assertTrue(stream.awaitTermination(30, TimeUnit.SECONDS)); + } + private List makeGetDataHeartbeatRequest(List keys) { List result = new ArrayList<>(); for (String key : keys) { diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/StreamingEngineClientTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/StreamingEngineClientTest.java index 5fbbbd3129cb9..57ec547e25cc9 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/StreamingEngineClientTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/StreamingEngineClientTest.java @@ -246,8 +246,8 @@ public void testStreamsStartCorrectly() throws InterruptedException { any(), any(), eq(noOpProcessWorkItemFn())); - verify(streamFactory, times(2)).createGetDataStream(anyString(), any(), any()); - verify(streamFactory, times(2)).createCommitWorkStream(any(), any()); + verify(streamFactory, times(2)).createDirectGetDataStream(anyString(), any(), any()); + verify(streamFactory, times(2)).createDirectCommitWorkStream(anyString(), any(), any()); } @Test diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/WindmillStreamSenderTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/WindmillStreamSenderTest.java index 1d0c6a1de2c30..a1bef4f369773 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/WindmillStreamSenderTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/WindmillStreamSenderTest.java @@ -18,7 +18,6 @@ package org.apache.beam.runners.dataflow.worker.windmill.client.grpc; import static org.mockito.ArgumentMatchers.any; -import static org.mockito.ArgumentMatchers.anyString; import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; @@ -52,6 +51,7 @@ @RunWith(JUnit4.class) public class WindmillStreamSenderTest { + private static final String BACKEND_WORKER_TOKEN = "some_backend_worker"; private static final GetWorkRequest GET_WORK_REQUEST = GetWorkRequest.newBuilder().setClientId(1L).setJobId("job").setProjectId("project").build(); @Rule public final GrpcCleanupRule grpcCleanup = new GrpcCleanupRule(); @@ -97,10 +97,11 @@ public void testStartStream_startsAllStreams() { verify(streamFactory) .createDirectGetWorkStream( - anyString(), + eq(BACKEND_WORKER_TOKEN), eq(stub), eq( - GET_WORK_REQUEST.toBuilder() + GET_WORK_REQUEST + .toBuilder() .setMaxItems(itemBudget) .setMaxBytes(byteBudget) .build()), @@ -110,8 +111,10 @@ public void testStartStream_startsAllStreams() { any(), eq(workItemScheduler)); - verify(streamFactory).createGetDataStream(anyString(), eq(stub), any(ThrottleTimer.class)); - verify(streamFactory).createCommitWorkStream(eq(stub), any(ThrottleTimer.class)); + verify(streamFactory) + .createDirectGetDataStream(eq(BACKEND_WORKER_TOKEN), eq(stub), any(ThrottleTimer.class)); + verify(streamFactory) + .createDirectCommitWorkStream(eq(BACKEND_WORKER_TOKEN), eq(stub), any(ThrottleTimer.class)); } @Test @@ -129,10 +132,11 @@ public void testStartStream_onlyStartsStreamsOnce() { verify(streamFactory, times(1)) .createDirectGetWorkStream( - anyString(), + eq(BACKEND_WORKER_TOKEN), eq(stub), eq( - GET_WORK_REQUEST.toBuilder() + GET_WORK_REQUEST + .toBuilder() .setMaxItems(itemBudget) .setMaxBytes(byteBudget) .build()), @@ -143,8 +147,9 @@ public void testStartStream_onlyStartsStreamsOnce() { eq(workItemScheduler)); verify(streamFactory, times(1)) - .createGetDataStream(anyString(), eq(stub), any(ThrottleTimer.class)); - verify(streamFactory, times(1)).createCommitWorkStream(eq(stub), any(ThrottleTimer.class)); + .createDirectGetDataStream(eq(BACKEND_WORKER_TOKEN), eq(stub), any(ThrottleTimer.class)); + verify(streamFactory, times(1)) + .createDirectGetDataStream(eq(BACKEND_WORKER_TOKEN), eq(stub), any(ThrottleTimer.class)); } @Test @@ -165,10 +170,11 @@ public void testStartStream_onlyStartsStreamsOnceConcurrent() throws Interrupted verify(streamFactory, times(1)) .createDirectGetWorkStream( - anyString(), + eq(BACKEND_WORKER_TOKEN), eq(stub), eq( - GET_WORK_REQUEST.toBuilder() + GET_WORK_REQUEST + .toBuilder() .setMaxItems(itemBudget) .setMaxBytes(byteBudget) .build()), @@ -179,8 +185,9 @@ public void testStartStream_onlyStartsStreamsOnceConcurrent() throws Interrupted eq(workItemScheduler)); verify(streamFactory, times(1)) - .createGetDataStream(anyString(), eq(stub), any(ThrottleTimer.class)); - verify(streamFactory, times(1)).createCommitWorkStream(eq(stub), any(ThrottleTimer.class)); + .createDirectGetDataStream(eq(BACKEND_WORKER_TOKEN), eq(stub), any(ThrottleTimer.class)); + verify(streamFactory, times(1)) + .createDirectCommitWorkStream(eq(BACKEND_WORKER_TOKEN), eq(stub), any(ThrottleTimer.class)); } @Test @@ -205,7 +212,7 @@ public void testCloseAllStreams_closesAllStreams() { CommitWorkStream mockCommitWorkStream = mock(CommitWorkStream.class); when(mockStreamFactory.createDirectGetWorkStream( - anyString(), + eq(BACKEND_WORKER_TOKEN), eq(stub), eq(getWorkRequestWithBudget), any(ThrottleTimer.class), @@ -214,10 +221,11 @@ public void testCloseAllStreams_closesAllStreams() { any(), eq(workItemScheduler))) .thenReturn(mockGetWorkStream); - when(mockStreamFactory.createGetDataStream(anyString(), eq(stub), any(ThrottleTimer.class))) + when(mockStreamFactory.createDirectGetDataStream( + eq(BACKEND_WORKER_TOKEN), eq(stub), any(ThrottleTimer.class))) .thenReturn(mockGetDataStream); when(mockStreamFactory.createDirectCommitWorkStream( - anyString(), eq(stub), any(ThrottleTimer.class))) + eq(BACKEND_WORKER_TOKEN), eq(stub), any(ThrottleTimer.class))) .thenReturn(mockCommitWorkStream); WindmillStreamSender windmillStreamSender = @@ -228,9 +236,9 @@ public void testCloseAllStreams_closesAllStreams() { windmillStreamSender.startStreams(); windmillStreamSender.closeAllStreams(); - verify(mockGetWorkStream).close(); - verify(mockGetDataStream).close(); - verify(mockCommitWorkStream).close(); + verify(mockGetWorkStream).shutdown(); + verify(mockGetDataStream).shutdown(); + verify(mockCommitWorkStream).shutdown(); } private WindmillStreamSender newWindmillStreamSender(GetWorkBudget budget) { @@ -240,7 +248,7 @@ private WindmillStreamSender newWindmillStreamSender(GetWorkBudget budget) { private WindmillStreamSender newWindmillStreamSender( GetWorkBudget budget, GrpcWindmillStreamFactory streamFactory) { return WindmillStreamSender.create( - "backendWorkerId", + BACKEND_WORKER_TOKEN, stub, GET_WORK_REQUEST, budget, From c3ff2c33607eafad54eb1706de4e56cb767afa3c Mon Sep 17 00:00:00 2001 From: Martin Trieu Date: Tue, 2 Jul 2024 01:58:30 -0700 Subject: [PATCH 13/13] address PR comments --- .../MetricTrackingWindmillServerStub.java | 6 +- .../worker/StreamingDataflowWorker.java | 53 ++++++++-------- .../client/AbstractWindmillStream.java | 29 ++++++--- .../windmill/client/WindmillStream.java | 63 +++++++------------ .../client/grpc/GrpcDirectGetWorkStream.java | 2 +- .../grpc/observers/DirectStreamObserver.java | 22 +++++++ .../work/refresh/DirectHeartbeatSender.java | 2 +- .../work/refresh/HeartbeatSender.java | 4 ++ ...er.java => StreamPoolHeartbeatSender.java} | 6 +- .../dataflow/worker/FakeWindmillServer.java | 13 ++-- .../client/WindmillStreamPoolTest.java | 10 ++- .../StreamingEngineWorkCommitterTest.java | 5 +- 12 files changed, 117 insertions(+), 98 deletions(-) rename runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/{PoolBackedHeartbeatSender.java => StreamPoolHeartbeatSender.java} (91%) diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/MetricTrackingWindmillServerStub.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/MetricTrackingWindmillServerStub.java index fbe61a3fb662e..1a472184172aa 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/MetricTrackingWindmillServerStub.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/MetricTrackingWindmillServerStub.java @@ -296,7 +296,8 @@ public Windmill.GlobalData getSideInputData( } /** - * Attempts to refresh active work, fanning out to each {@link GetDataStream} in parallel. + * Attempts to refresh active work, fanning out to each {@link GetDataStream} in parallel based on + * {@link HeartbeatSender} implementation. * * @implNote Skips closed {@link GetDataStream}(s). */ @@ -306,11 +307,14 @@ public void refreshActiveWork( return; } + // TODO(m-trieu): Return CompleteableFuture to prevent per-destination blocked thread for + // heartbeats. try { if (heartbeats.size() == 1) { // There is 1 destination to send heartbeat requests. Map.Entry>> heartbeat = Iterables.getOnlyElement(heartbeats.entrySet()); + activeHeartbeats.getAndUpdate(existing -> existing + heartbeat.getValue().size()); HeartbeatSender sender = heartbeat.getKey(); sender.sendHeartbeats(heartbeat.getValue()); } else { 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 b9124753d0209..a947e239242e3 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 @@ -94,7 +94,7 @@ import org.apache.beam.runners.dataflow.worker.windmill.work.refresh.ActiveWorkRefresher; import org.apache.beam.runners.dataflow.worker.windmill.work.refresh.ApplianceHeartbeatSender; import org.apache.beam.runners.dataflow.worker.windmill.work.refresh.HeartbeatSender; -import org.apache.beam.runners.dataflow.worker.windmill.work.refresh.PoolBackedHeartbeatSender; +import org.apache.beam.runners.dataflow.worker.windmill.work.refresh.StreamPoolHeartbeatSender; import org.apache.beam.sdk.fn.IdGenerator; import org.apache.beam.sdk.fn.IdGenerators; import org.apache.beam.sdk.fn.JvmInitializers; @@ -171,7 +171,7 @@ public class StreamingDataflowWorker { private final StreamingWorkerStatusReporter workerStatusReporter; private final StreamingCounters streamingCounters; private final StreamingWorkScheduler streamingWorkScheduler; - private final HeartbeatSender heartbeatSender; + private final @Nullable HeartbeatSender heartbeatSender; private @Nullable StreamingEngineClient streamingEngineClient; private StreamingDataflowWorker( @@ -236,7 +236,7 @@ private StreamingDataflowWorker( if (windmillServiceEnabled) { streamingDispatchLoop(); } else { - dispatchLoop(); + applianceDispatchLoop(); } LOG.info("Dispatch done"); }); @@ -339,19 +339,19 @@ private StreamingDataflowWorker( (computationId, request) -> metricTrackingWindmillServer.getStateData( getDataStream, computationId, request)); + this.heartbeatSender = null; } else { this.streamingEngineClient = null; + this.heartbeatSender = + options.isEnableStreamingEngine() + ? new StreamPoolHeartbeatSender( + options.getUseSeparateWindmillHeartbeatStreams() + ? WindmillStreamPool.create( + 1, GET_DATA_STREAM_TIMEOUT, windmillServer::getDataStream) + : getDataStreamPool) + : new ApplianceHeartbeatSender(windmillServer::getData); } - this.heartbeatSender = - options.isEnableStreamingEngine() - ? new PoolBackedHeartbeatSender( - options.getUseSeparateWindmillHeartbeatStreams() - ? WindmillStreamPool.create( - 1, GET_DATA_STREAM_TIMEOUT, windmillServer::getDataStream) - : getDataStreamPool) - : new ApplianceHeartbeatSender(windmillServer::getData); - this.activeWorkRefresher = new ActiveWorkRefresher( clock, @@ -441,6 +441,7 @@ public static StreamingDataflowWorker fromOptions(DataflowWorkerHarnessOptions o configFetcherComputationStateCacheAndWindmillClient = createConfigFetcherComputationStateCacheAndWindmillClient( options, + dispatcherClient, dataflowServiceClient, maxWorkItemCommitBytes, windmillStreamFactoryBuilder, @@ -506,6 +507,7 @@ public static StreamingDataflowWorker fromOptions(DataflowWorkerHarnessOptions o private static ConfigFetcherComputationStateCacheAndWindmillClient createConfigFetcherComputationStateCacheAndWindmillClient( DataflowWorkerHarnessOptions options, + GrpcDispatcherClient dispatcherClient, WorkUnitClient dataflowServiceClient, AtomicInteger maxWorkItemCommitBytes, GrpcWindmillStreamFactory.Builder windmillStreamFactoryBuilder, @@ -513,7 +515,6 @@ public static StreamingDataflowWorker fromOptions(DataflowWorkerHarnessOptions o ComputationConfig.Fetcher configFetcher; WindmillServerStub windmillServer; ComputationStateCache computationStateCache; - GrpcDispatcherClient dispatcherClient = GrpcDispatcherClient.create(createStubFactory(options)); GrpcWindmillStreamFactory windmillStreamFactory; if (options.isEnableStreamingEngine()) { configFetcher = @@ -743,20 +744,18 @@ private static void validateWorkerOptions(DataflowWorkerHarnessOptions options) } private static boolean isDirectPathPipeline(DataflowWorkerHarnessOptions options) { - boolean isIpV6Enabled = - Optional.ofNullable(options.getDataflowServiceOptions()) - .map(serviceOptions -> serviceOptions.contains(ENABLE_IPV6_EXPERIMENT)) - .orElse(false); - if (options.isEnableStreamingEngine() - && options.getIsWindmillServiceDirectPathEnabled() - && !isIpV6Enabled) { + if (options.isEnableStreamingEngine() && options.getIsWindmillServiceDirectPathEnabled()) { + boolean isIpV6Enabled = + Optional.ofNullable(options.getDataflowServiceOptions()) + .map(serviceOptions -> serviceOptions.contains(ENABLE_IPV6_EXPERIMENT)) + .orElse(false); + if (isIpV6Enabled) { + return true; + } LOG.warn( "DirectPath is currently only supported with IPv6 networking stack. Defaulting to CloudPath."); } - - return options.isEnableStreamingEngine() - && options.getIsWindmillServiceDirectPathEnabled() - && isIpV6Enabled; + return false; } private static ChannelCachingStubFactory createStubFactory( @@ -895,7 +894,7 @@ void stop() { } } - private void dispatchLoop() { + private void applianceDispatchLoop() { while (running.get()) { memoryMonitor.waitForResources("GetWork"); @@ -939,7 +938,7 @@ private void dispatchLoop() { computationId, metricTrackingWindmillServer::getStateData, workCommitter::commit, - heartbeatSender), + Preconditions.checkNotNull(heartbeatSender)), /* getWorkStreamLatencies= */ Collections.emptyList()); } } @@ -977,7 +976,7 @@ void streamingDispatchLoop() { computationState.getComputationId(), metricTrackingWindmillServer::getStateData, workCommitter::commit, - heartbeatSender), + Preconditions.checkNotNull(heartbeatSender)), getWorkStreamLatencies); })); try { diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/AbstractWindmillStream.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/AbstractWindmillStream.java index 2782741ecf679..51958430aeb0d 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/AbstractWindmillStream.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/AbstractWindmillStream.java @@ -72,7 +72,7 @@ public abstract class AbstractWindmillStream implements Win protected static final int RPC_STREAM_CHUNK_SIZE = 2 << 20; private static final Logger LOG = LoggerFactory.getLogger(AbstractWindmillStream.class); protected final AtomicBoolean clientClosed; - private final WindmillStream.Id streamId; + private final String backendWorkerToken; private final AtomicLong lastSendTimeMs; private final Executor executor; private final ExecutorService requestSender; @@ -98,16 +98,18 @@ protected AbstractWindmillStream( Set> streamRegistry, int logEveryNStreamFailures, String backendWorkerToken) { - this.streamId = - WindmillStream.Id.create(this, backendWorkerToken, !backendWorkerToken.isEmpty()); + this.backendWorkerToken = backendWorkerToken; this.executor = Executors.newSingleThreadExecutor( - new ThreadFactoryBuilder().setDaemon(true).setNameFormat(streamId + "-thread").build()); + new ThreadFactoryBuilder() + .setDaemon(true) + .setNameFormat(streamType() + backendWorkerToken + "-thread") + .build()); this.requestSender = Executors.newSingleThreadExecutor( new ThreadFactoryBuilder() .setDaemon(true) - .setNameFormat(streamId + "-RequestThread") + .setNameFormat(streamType() + backendWorkerToken + "-RequestThread") .build()); this.backoff = backoff; this.streamRegistry = streamRegistry; @@ -136,8 +138,8 @@ private static String debugDuration(long nowMs, long startMs) { } @Override - public final Id id() { - return streamId; + public final String backendWorkerToken() { + return backendWorkerToken; } /** Called on each response from the server. */ @@ -189,8 +191,9 @@ protected final void send(RequestT request) { }); } catch (RejectedExecutionException e) { LOG.warn( - "{} is shutdown and will not send or receive any more requests or responses.", - streamId, + "{}-{} is shutdown and will not send or receive any more requests or responses.", + streamType(), + backendWorkerToken(), e); } } @@ -249,7 +252,7 @@ public final synchronized void maybeSendHealthCheck(Instant lastSendThreshold) { * status page rendering. */ public final void appendSummaryHtml(PrintWriter writer) { - writer.format("

%s:

", streamId); + writer.format("

%s-%s:

", streamType(), backendWorkerToken()); appendSpecificHtml(writer); writer.println("Status:"); writer.println( @@ -351,6 +354,12 @@ private void setLastError(String error) { lastErrorTime.set(DateTime.now()); } + @Override + public String toString() { + String id = streamType(); + return !backendWorkerToken().isEmpty() ? id + String.format("-[%s]", backendWorkerToken()) : id; + } + /** Request observer that allows updating its internal delegate. */ @ThreadSafe private static class UpdatableDelegateRequestObserver diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/WindmillStream.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/WindmillStream.java index 1614e468f8e36..dc58405821a33 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/WindmillStream.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/WindmillStream.java @@ -17,7 +17,6 @@ */ package org.apache.beam.runners.dataflow.worker.windmill.client; -import com.google.auto.value.AutoValue; import java.io.Closeable; import java.util.List; import java.util.Map; @@ -34,7 +33,7 @@ /** Superclass for streams returned by streaming Windmill methods. */ @ThreadSafe public interface WindmillStream { - Id id(); + String backendWorkerToken(); /** Indicates that no more requests will be sent. */ void close(); @@ -57,6 +56,8 @@ public interface WindmillStream { /** Indicates that the stream is shutdown and should not be used. */ boolean isShutdown(); + String streamType(); + /** Handle representing a stream of GetWork responses. */ @ThreadSafe interface GetWorkStream extends WindmillStream { @@ -65,6 +66,11 @@ interface GetWorkStream extends WindmillStream { /** Returns the remaining in-flight {@link GetWorkBudget}. */ GetWorkBudget remainingBudget(); + + @Override + default String streamType() { + return "GetWorkStream"; + } } /** Interface for streaming GetDataRequests to Windmill. */ @@ -81,6 +87,11 @@ Windmill.KeyedGetDataResponse requestKeyedData( void refreshActiveWork(Map> heartbeats); void onHeartbeatResponse(List responses); + + @Override + default String streamType() { + return "GetDataStream"; + } } /** Interface for streaming CommitWorkRequests to Windmill. */ @@ -92,6 +103,11 @@ interface CommitWorkStream extends WindmillStream { */ Optional newBatcher(); + @Override + default String streamType() { + return "CommitWorkStream"; + } + @NotThreadSafe interface RequestBatcher extends Closeable { /** @@ -118,47 +134,10 @@ default void close() { /** Interface for streaming GetWorkerMetadata requests to Windmill. */ @ThreadSafe - interface GetWorkerMetadataStream extends WindmillStream {} - - @AutoValue - abstract class Id { - private static final String GET_WORK_STREAM_TYPE = "GetWorkStream"; - private static final String GET_DATA_STREAM_TYPE = "GetDataStream"; - private static final String GET_WORKER_METADATA_STREAM_TYPE = "GetWorkerMetadataStream"; - private static final String COMMIT_WORK_STREAM_TYPE = "CommitWorkStream"; - - public static Id create(WindmillStream stream, String backendWorkerToken, boolean isDirect) { - return new AutoValue_WindmillStream_Id( - Id.getStreamType(stream), backendWorkerToken, isDirect); - } - - private static String getStreamType(WindmillStream windmillStream) { - if (windmillStream instanceof GetWorkStream) { - return GET_WORK_STREAM_TYPE; - } else if (windmillStream instanceof GetWorkerMetadataStream) { - return GET_WORKER_METADATA_STREAM_TYPE; - } else if (windmillStream instanceof GetDataStream) { - return GET_DATA_STREAM_TYPE; - } else if (windmillStream instanceof CommitWorkStream) { - return COMMIT_WORK_STREAM_TYPE; - } - - // Should not happen conditions above are exhaustive. - throw new IllegalArgumentException("Unknown stream type."); - } - - abstract String streamType(); - - public abstract String backendWorkerToken(); - - abstract boolean isDirect(); - + interface GetWorkerMetadataStream extends WindmillStream { @Override - public final String toString() { - String id = String.format("%s-%s", streamType(), isDirect() ? "Direct" : "Dispatched"); - return !backendWorkerToken().isEmpty() - ? id + String.format("-[%s]", backendWorkerToken()) - : id; + default String streamType() { + return "GetWorkerMetadataStream"; } } } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcDirectGetWorkStream.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcDirectGetWorkStream.java index ef3eb668f4694..27e2df0d33ebc 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcDirectGetWorkStream.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcDirectGetWorkStream.java @@ -441,7 +441,7 @@ private void scheduleWorkItem(WorkItem workItem) { workCommitter.get()::commit, DirectHeartbeatSender.create(getDataStream.get())) .toBuilder() - .setBackendWorkerToken(id().backendWorkerToken()) + .setBackendWorkerToken(backendWorkerToken().backendWorkerToken()) .build(), // After the work item is successfully queued or dropped by ActiveWorkState, remove it // from the pendingResponseBudget. diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/observers/DirectStreamObserver.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/observers/DirectStreamObserver.java index 09d518b86a21f..dc37c21fc90b6 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/observers/DirectStreamObserver.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/observers/DirectStreamObserver.java @@ -118,6 +118,12 @@ public void onNext(T value) { totalSecondsWaited += waitSeconds; if (totalSecondsWaited > deadlineSeconds) { + LOG.error( + "PhaserState: isTerminated={}, registeredParties={}, unarrived={}, arrived={}", + isReadyNotifier.isTerminated(), + isReadyNotifier.getRegisteredParties(), + isReadyNotifier.getUnarrivedParties(), + isReadyNotifier.getArrivedParties()); throw new StreamObserverCancelledException( "Waited " + totalSecondsWaited @@ -168,4 +174,20 @@ private void unblockIsReadyNotifier() { // onCompleted or onError has already been called and waiting threads have been freed. } } + + @Override + public String toString() { + return "DirectStreamObserver{" + + "isReadyNotifier=" + + isReadyNotifier + + ", deadlineSeconds=" + + deadlineSeconds + + ", messagesBetweenIsReadyChecks=" + + messagesBetweenIsReadyChecks + + ", outboundObserver=" + + outboundObserver + + ", messagesSinceReady=" + + messagesSinceReady + + '}'; + } } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/DirectHeartbeatSender.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/DirectHeartbeatSender.java index 9c3e39ce69872..9dfbae5985e3c 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/DirectHeartbeatSender.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/DirectHeartbeatSender.java @@ -59,7 +59,7 @@ public void sendHeartbeats(Map> heartbeats) { LOG.warn( "Trying to refresh work w/ {} heartbeats on stream={} after work has moved off of worker." + " heartbeats", - getDataStream.id(), + getDataStream.backendWorkerToken(), heartbeats.size()); onStreamClosed.run(); } else { diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/HeartbeatSender.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/HeartbeatSender.java index 85918353b3331..be093a32c8136 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/HeartbeatSender.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/HeartbeatSender.java @@ -24,5 +24,9 @@ /** Interface for sending heartbeats. */ @FunctionalInterface public interface HeartbeatSender { + /** + * Send heartbeats. Heartbeats are a map of computation id to a list of heartbeat requests that + * represent WorkItem that is actively being processed belonging to the computation. + */ void sendHeartbeats(Map> heartbeats); } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/PoolBackedHeartbeatSender.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/StreamPoolHeartbeatSender.java similarity index 91% rename from runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/PoolBackedHeartbeatSender.java rename to runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/StreamPoolHeartbeatSender.java index ae582dc02fad1..c2569a3667457 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/PoolBackedHeartbeatSender.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/StreamPoolHeartbeatSender.java @@ -29,12 +29,12 @@ /** StreamingEngine stream pool based implementation of {@link HeartbeatSender}. */ @Internal -public final class PoolBackedHeartbeatSender implements HeartbeatSender { - private static final Logger LOG = LoggerFactory.getLogger(PoolBackedHeartbeatSender.class); +public final class StreamPoolHeartbeatSender implements HeartbeatSender { + private static final Logger LOG = LoggerFactory.getLogger(StreamPoolHeartbeatSender.class); private final WindmillStreamPool heartbeatStreamPool; - public PoolBackedHeartbeatSender( + public StreamPoolHeartbeatSender( WindmillStreamPool heartbeatStreamPool) { this.heartbeatStreamPool = heartbeatStreamPool; } diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/FakeWindmillServer.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/FakeWindmillServer.java index 6180bbe2e4e52..61688e5ddfc59 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/FakeWindmillServer.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/FakeWindmillServer.java @@ -61,7 +61,6 @@ import org.apache.beam.runners.dataflow.worker.windmill.Windmill.LatencyAttribution.State; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.WorkItemCommitRequest; import org.apache.beam.runners.dataflow.worker.windmill.WindmillServerStub; -import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStream; import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStream.CommitWorkStream; import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStream.GetDataStream; import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStream.GetWorkStream; @@ -234,8 +233,8 @@ public GetWorkStream getWorkStream(Windmill.GetWorkRequest request, WorkItemRece return new GetWorkStream() { @Override - public Id id() { - return WindmillStream.Id.create(this, "backend_worker_token", false); + public String backendWorkerToken() { + return "backend_worker_token"; } @Override @@ -317,8 +316,8 @@ public GetDataStream getDataStream() { Instant startTime = Instant.now(); return new GetDataStream() { @Override - public Id id() { - return WindmillStream.Id.create(this, "backend_worker_token", false); + public String backendWorkerToken() { + return "backend_worker_token"; } @Override @@ -407,8 +406,8 @@ public CommitWorkStream commitWorkStream() { Instant startTime = Instant.now(); return new CommitWorkStream() { @Override - public Id id() { - return WindmillStream.Id.create(this, "backend_worker_token", false); + public String backendWorkerToken() { + return "backend_worker_token"; } @Override diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/WindmillStreamPoolTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/WindmillStreamPoolTest.java index e4dcec84cff1c..6f76aaee8d88f 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/WindmillStreamPoolTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/WindmillStreamPoolTest.java @@ -22,7 +22,6 @@ import static org.junit.Assert.assertNotSame; import static org.junit.Assert.assertThrows; import static org.junit.Assert.assertTrue; -import static org.mockito.Mockito.mock; import java.util.List; import java.util.concurrent.ConcurrentHashMap; @@ -238,8 +237,13 @@ private TestWindmillStream(Instant startTime) { } @Override - public Id id() { - return Id.create(mock(GetWorkStream.class), "backend_worker_token", false); + public String backendWorkerToken() { + return "backend_worker_token"; + } + + @Override + public String streamType() { + return "test_stream"; } @Override diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/commits/StreamingEngineWorkCommitterTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/commits/StreamingEngineWorkCommitterTest.java index 5d89ea679ac50..49aa7a56245df 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/commits/StreamingEngineWorkCommitterTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/commits/StreamingEngineWorkCommitterTest.java @@ -47,7 +47,6 @@ import org.apache.beam.runners.dataflow.worker.windmill.Windmill; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.WorkItemCommitRequest; import org.apache.beam.runners.dataflow.worker.windmill.client.CloseableStream; -import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStream; import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStream.CommitWorkStream; import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStreamPool; import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; @@ -261,8 +260,8 @@ public void testStop_drainsCommitQueue() { private boolean closed = false; @Override - public Id id() { - return WindmillStream.Id.create(this, "backend_worker_token", false); + public String backendWorkerToken() { + return "backend_worker_token"; } @Override