arunpandianp commented on code in PR #30695:
URL: https://github.com/apache/beam/pull/30695#discussion_r1538539063
##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java:
##########
@@ -489,26 +465,67 @@ static StreamingDataflowWorker forTesting(
boolean publishCounters,
HotKeyLogger hotKeyLogger,
Supplier<Instant> clock,
- Function<String, ScheduledExecutorService> executorSupplier) {
- BoundedQueueExecutor boundedQueueExecutor =
createWorkUnitExecutor(options);
+ Function<String, ScheduledExecutorService> executorSupplier,
+ int localRetryTimeoutMs) {
+ ConcurrentMap<String, StageInfo> stageInfo = new ConcurrentHashMap<>();
+ BoundedQueueExecutor workExecutor = createWorkUnitExecutor(options);
WindmillStateCache stateCache =
WindmillStateCache.ofSizeMbs(options.getWorkerCacheMb());
computationMap.putAll(
- createComputationMapForTesting(mapTasks, boundedQueueExecutor,
stateCache::forComputation));
+ createComputationMapForTesting(mapTasks, workExecutor,
stateCache::forComputation));
+ MemoryMonitor memoryMonitor = MemoryMonitor.fromOptions(options);
+ StreamingCounters streamingCounters = StreamingCounters.create();
+ FailureReporter failureReporter =
+ options.isEnableStreamingEngine()
+ ? StreamingEngineFailureReporter.create(
+ MAX_FAILURES_TO_REPORT_IN_UPDATE,
options.getMaxStackTraceDepthToReport())
+ : StreamingApplianceFailureReporter.create(
+ MAX_FAILURES_TO_REPORT_IN_UPDATE,
+ options.getMaxStackTraceDepthToReport(),
+ windmillServer::reportStats);
+ WorkFailureProcessor workFailureProcessor =
+ new WorkFailureProcessor(
+ workExecutor,
+ failureReporter,
+ () -> Optional.ofNullable(memoryMonitor.tryToDumpHeap()),
+ clock,
+ localRetryTimeoutMs);
+ StreamingWorkerStatusReporter workerStatusReporter =
+ StreamingWorkerStatusReporter.forTesting(
+ publishCounters,
+ workUnitClient,
+ windmillServer::getAndResetThrottleTime,
+ stageInfo::values,
+ failureReporter,
+ streamingCounters,
+ memoryMonitor,
+ workExecutor,
+ () -> chooseMaxThreads(options),
+ executorSupplier);
return new StreamingDataflowWorker(
windmillServer,
1L,
computationMap,
+ stageInfo,
stateCache,
- boundedQueueExecutor,
+ workExecutor,
mapTaskExecutorFactory,
workUnitClient,
options,
- publishCounters,
hotKeyLogger,
clock,
+ workerStatusReporter,
+ failureReporter,
+ workFailureProcessor,
+ streamingCounters,
+ memoryMonitor,
executorSupplier);
}
+ @VisibleForTesting
+ public final void reportPeriodicWorkerUpdatesForTest() {
Review Comment:
can we make this package private?
##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/harness/StreamingWorkerStatusReporter.java:
##########
@@ -0,0 +1,370 @@
+/*
+ * 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.harness;
+
+import static
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkNotNull;
+
+import com.google.api.services.dataflow.model.CounterUpdate;
+import com.google.api.services.dataflow.model.PerStepNamespaceMetrics;
+import com.google.api.services.dataflow.model.PerWorkerMetrics;
+import com.google.api.services.dataflow.model.Status;
+import com.google.api.services.dataflow.model.StreamingScalingReport;
+import com.google.api.services.dataflow.model.WorkItemStatus;
+import com.google.api.services.dataflow.model.WorkerMessage;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+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.AtomicLong;
+import java.util.function.Function;
+import java.util.function.Supplier;
+import javax.annotation.concurrent.ThreadSafe;
+import org.apache.beam.runners.dataflow.worker.DataflowSystemMetrics;
+import org.apache.beam.runners.dataflow.worker.StreamingStepMetricsContainer;
+import org.apache.beam.runners.dataflow.worker.WorkUnitClient;
+import org.apache.beam.runners.dataflow.worker.counters.CounterSet;
+import
org.apache.beam.runners.dataflow.worker.counters.DataflowCounterUpdateExtractor;
+import
org.apache.beam.runners.dataflow.worker.logging.DataflowWorkerLoggingMDC;
+import org.apache.beam.runners.dataflow.worker.streaming.StageInfo;
+import org.apache.beam.runners.dataflow.worker.util.BoundedQueueExecutor;
+import org.apache.beam.runners.dataflow.worker.util.MemoryMonitor;
+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.ImmutableList;
+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.util.concurrent.ThreadFactoryBuilder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Reports the status of the worker to Dataflow Service. */
+@Internal
+@ThreadSafe
+public final class StreamingWorkerStatusReporter {
+ private static final Logger LOG =
LoggerFactory.getLogger(StreamingWorkerStatusReporter.class);
+
+ // Reserved ID for counter updates.
+ // Matches kWindmillCounterUpdate in workflow_worker_service_multi_hubs.cc.
+ private static final String WINDMILL_COUNTER_UPDATE_WORK_ID = "3";
+ private static final int COUNTER_UPDATES_SIZE = 128;
+ 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 WorkUnitClient dataflowServiceClient;
+ private final Supplier<Long> windmillQuotaThrottleTime;
+ private final Supplier<Collection<StageInfo>> allStageInfo;
+ private final Supplier<ImmutableList<Status>> pendingErrorsToReport;
+ private final StreamingCounters streamingCounters;
+ private final MemoryMonitor memoryMonitor;
+ private final BoundedQueueExecutor workExecutor;
+ private final AtomicLong previousTimeAtMaxThreads;
+ private final Supplier<Integer> maxThreads;
+ private final ScheduledExecutorService globalWorkerUpdateReporter;
+ private final ScheduledExecutorService workerMessageReporter;
+
+ private StreamingWorkerStatusReporter(
+ boolean publishCounters,
+ WorkUnitClient dataflowServiceClient,
+ Supplier<Long> windmillQuotaThrottleTime,
+ Supplier<Collection<StageInfo>> allStageInfo,
+ Supplier<ImmutableList<Status>> pendingErrorsToReport,
+ StreamingCounters streamingCounters,
+ MemoryMonitor memoryMonitor,
+ BoundedQueueExecutor workExecutor,
+ AtomicLong previousTimeAtMaxThreads,
+ Supplier<Integer> maxThreads,
+ Function<String, ScheduledExecutorService> executorFactory) {
+ this.publishCounters = publishCounters;
+ this.dataflowServiceClient = dataflowServiceClient;
+ this.windmillQuotaThrottleTime = windmillQuotaThrottleTime;
+ this.allStageInfo = allStageInfo;
+ this.pendingErrorsToReport = pendingErrorsToReport;
+ this.streamingCounters = streamingCounters;
+ this.memoryMonitor = memoryMonitor;
+ this.workExecutor = workExecutor;
+ this.previousTimeAtMaxThreads = previousTimeAtMaxThreads;
+ this.maxThreads = maxThreads;
+ this.globalWorkerUpdateReporter =
executorFactory.apply(GLOBAL_WORKER_UPDATE_REPORTER_THREAD);
+ this.workerMessageReporter =
executorFactory.apply(WORKER_MESSAGE_REPORTER_THREAD);
+ }
+
+ public static StreamingWorkerStatusReporter create(
+ boolean publishCounters,
+ WorkUnitClient workUnitClient,
+ Supplier<Long> windmillQuotaThrottleTime,
+ Supplier<Collection<StageInfo>> allStageInfo,
+ Supplier<ImmutableList<Status>> pendingErrorsToReport,
+ StreamingCounters streamingCounters,
+ MemoryMonitor memoryMonitor,
+ BoundedQueueExecutor workExecutor,
+ Supplier<Integer> maxThreads) {
+ return new StreamingWorkerStatusReporter(
+ publishCounters,
+ workUnitClient,
+ windmillQuotaThrottleTime,
+ allStageInfo,
+ pendingErrorsToReport,
+ streamingCounters,
+ memoryMonitor,
+ workExecutor,
+ new AtomicLong(),
+ maxThreads,
+ threadName ->
+ Executors.newSingleThreadScheduledExecutor(
+ new ThreadFactoryBuilder().setNameFormat(threadName).build()));
+ }
+
+ @VisibleForTesting
+ public static StreamingWorkerStatusReporter forTesting(
+ boolean publishCounters,
+ WorkUnitClient workUnitClient,
+ Supplier<Long> windmillQuotaThrottleTime,
+ Supplier<Collection<StageInfo>> allStageInfo,
+ Supplier<ImmutableList<Status>> pendingErrorsToReport,
+ StreamingCounters streamingCounters,
+ MemoryMonitor memoryMonitor,
+ BoundedQueueExecutor workExecutor,
+ Supplier<Integer> maxThreads,
+ Function<String, ScheduledExecutorService> executorFactory) {
+ return new StreamingWorkerStatusReporter(
+ publishCounters,
+ workUnitClient,
+ windmillQuotaThrottleTime,
+ allStageInfo,
+ pendingErrorsToReport,
+ streamingCounters,
+ memoryMonitor,
+ workExecutor,
+ new AtomicLong(),
+ maxThreads,
+ executorFactory);
+ }
+
+ /**
+ * Returns key for a counter update. It is a String in case of legacy
counter and
+ * CounterStructuredName in the case of a structured counter.
+ */
+ private static Object getCounterUpdateKey(CounterUpdate counterUpdate) {
+ Object key = null;
+ if (counterUpdate.getNameAndKind() != null) {
+ key = counterUpdate.getNameAndKind().getName();
+ } else if (counterUpdate.getStructuredNameAndMetadata() != null) {
+ key = counterUpdate.getStructuredNameAndMetadata().getName();
+ }
+ return checkNotNull(key, "Could not find name for CounterUpdate: %s",
counterUpdate);
+ }
+
+ /**
+ * Clears counterUpdates and enqueues unique counters from counterMultimap.
If a counter appears
+ * more than once, one of them is extracted leaving the remaining in the map.
+ */
+ private static void extractUniqueCounters(
+ List<CounterUpdate> counterUpdates, ListMultimap<Object, CounterUpdate>
counterMultimap) {
+ counterUpdates.clear();
+ for (Iterator<Object> iter = counterMultimap.keySet().iterator();
iter.hasNext(); ) {
+ List<CounterUpdate> counters = counterMultimap.get(iter.next());
+ counterUpdates.add(counters.get(0));
+ if (counters.size() == 1) {
+ // There is single value. Remove the entry through the iterator.
+ iter.remove();
+ } else {
+ // Otherwise remove the first value.
+ counters.remove(0);
+ }
+ }
+ }
+
+ private static void shutdownExecutor(ScheduledExecutorService executor) {
+ executor.shutdown();
+ try {
+ executor.awaitTermination(10, TimeUnit.SECONDS);
+ } catch (InterruptedException e) {
+ LOG.warn("Error occurred trying to gracefully shutdown executor={}",
executor, e);
+ executor.shutdownNow();
+ }
+ }
+
+ @SuppressWarnings("FutureReturnValueIgnored")
+ public void start(long windmillHarnessUpdateReportingPeriod) {
Review Comment:
maybe take Duration instead of millis?
##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/harness/StreamingCounters.java:
##########
@@ -0,0 +1,134 @@
+/*
+ * 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.harness;
+
+import com.google.auto.value.AutoValue;
+import java.util.concurrent.atomic.AtomicLong;
+import
org.apache.beam.runners.dataflow.worker.DataflowSystemMetrics.StreamingSystemCounterNames;
+import org.apache.beam.runners.dataflow.worker.counters.Counter;
+import org.apache.beam.runners.dataflow.worker.counters.CounterSet;
+import org.apache.beam.runners.dataflow.worker.util.BoundedQueueExecutor;
+import org.apache.beam.sdk.annotations.Internal;
+
+/** Streaming pipeline counters to report pipeline processing metrics to
Dataflow backend. */
+@Internal
+@AutoValue
+public abstract class StreamingCounters {
+
+ public static StreamingCounters create() {
+ CounterSet pendingDeltaCounters = new CounterSet();
+ CounterSet pendingCumulativeCounters = new CounterSet();
+ return new AutoValue_StreamingCounters(
+ pendingDeltaCounters,
+ pendingCumulativeCounters,
+ pendingDeltaCounters.longSum(
+
StreamingSystemCounterNames.WINDMILL_SHUFFLE_BYTES_READ.counterName()),
+ pendingDeltaCounters.longSum(
+
StreamingSystemCounterNames.WINDMILL_STATE_BYTES_READ.counterName()),
+ pendingDeltaCounters.longSum(
+
StreamingSystemCounterNames.WINDMILL_STATE_BYTES_WRITTEN.counterName()),
+ pendingDeltaCounters.longSum(
+
StreamingSystemCounterNames.WINDMILL_QUOTA_THROTTLING.counterName()),
+ pendingDeltaCounters.longSum(
+
StreamingSystemCounterNames.TIME_AT_MAX_ACTIVE_THREADS.counterName()),
+ pendingCumulativeCounters.longSum(
+
StreamingSystemCounterNames.JAVA_HARNESS_USED_MEMORY.counterName()),
+ pendingCumulativeCounters.longSum(
+ StreamingSystemCounterNames.JAVA_HARNESS_MAX_MEMORY.counterName()),
+
pendingCumulativeCounters.intSum(StreamingSystemCounterNames.ACTIVE_THREADS.counterName()),
+ pendingCumulativeCounters.intSum(
+ StreamingSystemCounterNames.TOTAL_ALLOCATED_THREADS.counterName()),
+ pendingCumulativeCounters.longSum(
+ StreamingSystemCounterNames.OUTSTANDING_BYTES.counterName()),
+ pendingCumulativeCounters.longSum(
+ StreamingSystemCounterNames.MAX_OUTSTANDING_BYTES.counterName()),
+ pendingCumulativeCounters.longSum(
+ StreamingSystemCounterNames.OUTSTANDING_BUNDLES.counterName()),
+ pendingCumulativeCounters.longSum(
+ StreamingSystemCounterNames.MAX_OUTSTANDING_BUNDLES.counterName()),
+ pendingCumulativeCounters.intMax(
+
StreamingSystemCounterNames.WINDMILL_MAX_WORK_ITEM_COMMIT_BYTES.counterName()),
+ pendingCumulativeCounters.intSum(
+ StreamingSystemCounterNames.MEMORY_THRASHING.counterName()));
+ }
+
+ public abstract CounterSet pendingDeltaCounters();
+
+ public abstract CounterSet pendingCumulativeCounters();
+ // Built-in delta counters.
+ public abstract Counter<Long, Long> windmillShuffleBytesRead();
+
+ public abstract Counter<Long, Long> windmillStateBytesRead();
+
+ public abstract Counter<Long, Long> windmillStateBytesWritten();
+
+ public abstract Counter<Long, Long> windmillQuotaThrottling();
+
+ public abstract Counter<Long, Long> timeAtMaxActiveThreads();
+ // Built-in cumulative counters.
+ public abstract Counter<Long, Long> javaHarnessUsedMemory();
+
+ public abstract Counter<Long, Long> javaHarnessMaxMemory();
+
+ public abstract Counter<Integer, Integer> activeThreads();
+
+ public abstract Counter<Integer, Integer> totalAllocatedThreads();
+
+ public abstract Counter<Long, Long> outstandingBytes();
+
+ public abstract Counter<Long, Long> maxOutstandingBytes();
+
+ public abstract Counter<Long, Long> outstandingBundles();
+
+ public abstract Counter<Long, Long> maxOutstandingBundles();
+
+ public abstract Counter<Integer, Integer>
windmillMaxObservedWorkItemCommitBytes();
+
+ public abstract Counter<Integer, Integer> memoryThrashing();
+
+ final void updateVMMetrics() {
Review Comment:
can we move `updateVMMetrics`, `updateThreadMetrics` to
`StreamingWorkerStatusReporter`? then we won't need to have the computation
logic in a AutoValue class, which seems odd.
##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/harness/StreamingWorkerStatusReporter.java:
##########
@@ -0,0 +1,370 @@
+/*
+ * 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.harness;
+
+import static
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkNotNull;
+
+import com.google.api.services.dataflow.model.CounterUpdate;
+import com.google.api.services.dataflow.model.PerStepNamespaceMetrics;
+import com.google.api.services.dataflow.model.PerWorkerMetrics;
+import com.google.api.services.dataflow.model.Status;
+import com.google.api.services.dataflow.model.StreamingScalingReport;
+import com.google.api.services.dataflow.model.WorkItemStatus;
+import com.google.api.services.dataflow.model.WorkerMessage;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+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.AtomicLong;
+import java.util.function.Function;
+import java.util.function.Supplier;
+import javax.annotation.concurrent.ThreadSafe;
+import org.apache.beam.runners.dataflow.worker.DataflowSystemMetrics;
+import org.apache.beam.runners.dataflow.worker.StreamingStepMetricsContainer;
+import org.apache.beam.runners.dataflow.worker.WorkUnitClient;
+import org.apache.beam.runners.dataflow.worker.counters.CounterSet;
+import
org.apache.beam.runners.dataflow.worker.counters.DataflowCounterUpdateExtractor;
+import
org.apache.beam.runners.dataflow.worker.logging.DataflowWorkerLoggingMDC;
+import org.apache.beam.runners.dataflow.worker.streaming.StageInfo;
+import org.apache.beam.runners.dataflow.worker.util.BoundedQueueExecutor;
+import org.apache.beam.runners.dataflow.worker.util.MemoryMonitor;
+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.ImmutableList;
+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.util.concurrent.ThreadFactoryBuilder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Reports the status of the worker to Dataflow Service. */
+@Internal
+@ThreadSafe
+public final class StreamingWorkerStatusReporter {
+ private static final Logger LOG =
LoggerFactory.getLogger(StreamingWorkerStatusReporter.class);
+
+ // Reserved ID for counter updates.
+ // Matches kWindmillCounterUpdate in workflow_worker_service_multi_hubs.cc.
+ private static final String WINDMILL_COUNTER_UPDATE_WORK_ID = "3";
+ private static final int COUNTER_UPDATES_SIZE = 128;
+ 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 WorkUnitClient dataflowServiceClient;
+ private final Supplier<Long> windmillQuotaThrottleTime;
+ private final Supplier<Collection<StageInfo>> allStageInfo;
+ private final Supplier<ImmutableList<Status>> pendingErrorsToReport;
+ private final StreamingCounters streamingCounters;
+ private final MemoryMonitor memoryMonitor;
+ private final BoundedQueueExecutor workExecutor;
+ private final AtomicLong previousTimeAtMaxThreads;
+ private final Supplier<Integer> maxThreads;
+ private final ScheduledExecutorService globalWorkerUpdateReporter;
+ private final ScheduledExecutorService workerMessageReporter;
+
+ private StreamingWorkerStatusReporter(
+ boolean publishCounters,
+ WorkUnitClient dataflowServiceClient,
+ Supplier<Long> windmillQuotaThrottleTime,
+ Supplier<Collection<StageInfo>> allStageInfo,
+ Supplier<ImmutableList<Status>> pendingErrorsToReport,
+ StreamingCounters streamingCounters,
+ MemoryMonitor memoryMonitor,
+ BoundedQueueExecutor workExecutor,
+ AtomicLong previousTimeAtMaxThreads,
+ Supplier<Integer> maxThreads,
+ Function<String, ScheduledExecutorService> executorFactory) {
+ this.publishCounters = publishCounters;
+ this.dataflowServiceClient = dataflowServiceClient;
+ this.windmillQuotaThrottleTime = windmillQuotaThrottleTime;
+ this.allStageInfo = allStageInfo;
+ this.pendingErrorsToReport = pendingErrorsToReport;
+ this.streamingCounters = streamingCounters;
+ this.memoryMonitor = memoryMonitor;
+ this.workExecutor = workExecutor;
+ this.previousTimeAtMaxThreads = previousTimeAtMaxThreads;
+ this.maxThreads = maxThreads;
+ this.globalWorkerUpdateReporter =
executorFactory.apply(GLOBAL_WORKER_UPDATE_REPORTER_THREAD);
+ this.workerMessageReporter =
executorFactory.apply(WORKER_MESSAGE_REPORTER_THREAD);
+ }
+
+ public static StreamingWorkerStatusReporter create(
+ boolean publishCounters,
+ WorkUnitClient workUnitClient,
+ Supplier<Long> windmillQuotaThrottleTime,
+ Supplier<Collection<StageInfo>> allStageInfo,
+ Supplier<ImmutableList<Status>> pendingErrorsToReport,
+ StreamingCounters streamingCounters,
+ MemoryMonitor memoryMonitor,
+ BoundedQueueExecutor workExecutor,
+ Supplier<Integer> maxThreads) {
+ return new StreamingWorkerStatusReporter(
+ publishCounters,
+ workUnitClient,
+ windmillQuotaThrottleTime,
+ allStageInfo,
+ pendingErrorsToReport,
+ streamingCounters,
+ memoryMonitor,
+ workExecutor,
+ new AtomicLong(),
+ maxThreads,
+ threadName ->
+ Executors.newSingleThreadScheduledExecutor(
+ new ThreadFactoryBuilder().setNameFormat(threadName).build()));
+ }
+
+ @VisibleForTesting
+ public static StreamingWorkerStatusReporter forTesting(
+ boolean publishCounters,
+ WorkUnitClient workUnitClient,
+ Supplier<Long> windmillQuotaThrottleTime,
+ Supplier<Collection<StageInfo>> allStageInfo,
+ Supplier<ImmutableList<Status>> pendingErrorsToReport,
+ StreamingCounters streamingCounters,
+ MemoryMonitor memoryMonitor,
+ BoundedQueueExecutor workExecutor,
+ Supplier<Integer> maxThreads,
+ Function<String, ScheduledExecutorService> executorFactory) {
+ return new StreamingWorkerStatusReporter(
+ publishCounters,
+ workUnitClient,
+ windmillQuotaThrottleTime,
+ allStageInfo,
+ pendingErrorsToReport,
+ streamingCounters,
+ memoryMonitor,
+ workExecutor,
+ new AtomicLong(),
+ maxThreads,
+ executorFactory);
+ }
+
+ /**
+ * Returns key for a counter update. It is a String in case of legacy
counter and
+ * CounterStructuredName in the case of a structured counter.
+ */
+ private static Object getCounterUpdateKey(CounterUpdate counterUpdate) {
+ Object key = null;
+ if (counterUpdate.getNameAndKind() != null) {
+ key = counterUpdate.getNameAndKind().getName();
+ } else if (counterUpdate.getStructuredNameAndMetadata() != null) {
+ key = counterUpdate.getStructuredNameAndMetadata().getName();
+ }
+ return checkNotNull(key, "Could not find name for CounterUpdate: %s",
counterUpdate);
+ }
+
+ /**
+ * Clears counterUpdates and enqueues unique counters from counterMultimap.
If a counter appears
+ * more than once, one of them is extracted leaving the remaining in the map.
+ */
+ private static void extractUniqueCounters(
+ List<CounterUpdate> counterUpdates, ListMultimap<Object, CounterUpdate>
counterMultimap) {
+ counterUpdates.clear();
+ for (Iterator<Object> iter = counterMultimap.keySet().iterator();
iter.hasNext(); ) {
+ List<CounterUpdate> counters = counterMultimap.get(iter.next());
+ counterUpdates.add(counters.get(0));
+ if (counters.size() == 1) {
+ // There is single value. Remove the entry through the iterator.
+ iter.remove();
+ } else {
+ // Otherwise remove the first value.
+ counters.remove(0);
+ }
+ }
+ }
+
+ private static void shutdownExecutor(ScheduledExecutorService executor) {
+ executor.shutdown();
+ try {
+ executor.awaitTermination(10, TimeUnit.SECONDS);
+ } catch (InterruptedException e) {
+ LOG.warn("Error occurred trying to gracefully shutdown executor={}",
executor, e);
+ executor.shutdownNow();
+ }
+ }
+
+ @SuppressWarnings("FutureReturnValueIgnored")
+ public void start(long windmillHarnessUpdateReportingPeriod) {
+ reportHarnessStartup();
+ // Periodically report workers counters and other updates.
+ globalWorkerUpdateReporter.scheduleWithFixedDelay(
+ this::reportPeriodicWorkerUpdates,
+ 0,
+ windmillHarnessUpdateReportingPeriod,
+ TimeUnit.MILLISECONDS);
+
+ if (windmillHarnessUpdateReportingPeriod > 0) {
+ workerMessageReporter.scheduleWithFixedDelay(
+ this::reportPeriodicWorkerMessage,
+ 0,
+ windmillHarnessUpdateReportingPeriod,
+ TimeUnit.MILLISECONDS);
+ }
+ }
+
+ public void stop() {
+ shutdownExecutor(globalWorkerUpdateReporter);
+ shutdownExecutor(workerMessageReporter);
+ // one last send
+ reportPeriodicWorkerUpdates();
+ reportPeriodicWorkerMessage();
+ }
+
+ private void reportHarnessStartup() {
+ DataflowWorkerLoggingMDC.setStageName("startup");
+ CounterSet restartCounter = new CounterSet();
+ restartCounter
+ .longSum(
+
DataflowSystemMetrics.StreamingSystemCounterNames.JAVA_HARNESS_RESTARTS.counterName())
+ .addValue(1L);
+ try {
+ // Sending a one time update. Use empty counter set for
cumulativeCounters (2nd arg).
+ sendWorkerUpdatesToDataflowService(restartCounter, new CounterSet());
Review Comment:
I think we should make this not block start. It doesn't look like it is
blocking anything critical now. Could you file an issue to track?
##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/processing/failures/WorkFailureProcessor.java:
##########
@@ -0,0 +1,160 @@
+/*
+ * 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.processing.failures;
+
+import java.io.File;
+import java.util.Optional;
+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.KeyTokenInvalidException;
+import org.apache.beam.runners.dataflow.worker.WorkItemCancelledException;
+import
org.apache.beam.runners.dataflow.worker.status.LastExceptionDataProvider;
+import org.apache.beam.runners.dataflow.worker.streaming.Work;
+import org.apache.beam.runners.dataflow.worker.util.BoundedQueueExecutor;
+import org.apache.beam.sdk.annotations.Internal;
+import org.apache.beam.sdk.util.UserCodeException;
+import
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.Uninterruptibles;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Processes a failure that occurs during user processing of {@link Work}. */
+@ThreadSafe
+@Internal
+public final class WorkFailureProcessor {
+ private static final Logger LOG =
LoggerFactory.getLogger(WorkFailureProcessor.class);
+ private static final Duration MAX_LOCAL_PROCESSING_RETRY_DURATION =
Duration.standardMinutes(5);
+
+ private final BoundedQueueExecutor workUnitExecutor;
+ private final FailureReporter failureReporter;
+ private final Supplier<Optional<File>> heapDumpFetcher;
+ private final Supplier<Instant> clock;
+ private final int retryLocallyDelayMs;
+
+ public WorkFailureProcessor(
+ BoundedQueueExecutor workUnitExecutor,
+ FailureReporter failureReporter,
+ Supplier<Optional<File>> heapDumpFetcher,
+ Supplier<Instant> clock,
+ int retryLocallyDelayMs) {
+ this.workUnitExecutor = workUnitExecutor;
+ this.failureReporter = failureReporter;
+ this.heapDumpFetcher = heapDumpFetcher;
+ this.clock = clock;
+ this.retryLocallyDelayMs = retryLocallyDelayMs;
+ }
+
+ /** Returns whether an exception was caused by a {@link OutOfMemoryError}. */
+ private static boolean isOutOfMemoryError(Throwable t) {
+ while (t != null) {
+ if (t instanceof OutOfMemoryError) {
+ return true;
+ }
+ t = t.getCause();
+ }
+ return false;
+ }
+
+ /**
+ * Processes failures caused by thrown exceptions that occur during
execution of {@link Work}. May
+ * attempt to retry execution of the {@link Work} or drop it if it is
invalid.
+ */
+ public void logAndProcessFailure(
+ String computationId, Work work, Throwable t, Consumer<Work>
onInvalidWork) {
+ if (shouldRetryLocally(
+ computationId, work, t instanceof UserCodeException ? t.getCause() :
t)) {
+ // Try again after some delay and at the end of the queue to avoid a
tight loop.
+ executeWithDelay(retryLocallyDelayMs, work);
+ } else {
+ // Consider the item invalid. It will eventually be retried by Windmill
if it still needs to
+ // be processed.
+ onInvalidWork.accept(work);
+ }
+ }
+
+ private String getHeapDumpLog() {
+ return heapDumpFetcher
+ .get()
+ .map(heapDump -> "written to '" + heapDump + "'")
+ .orElseGet(() -> "not written");
+ }
+
+ private void executeWithDelay(long delayMs, Work work) {
+ Uninterruptibles.sleepUninterruptibly(delayMs, TimeUnit.MILLISECONDS);
+ workUnitExecutor.forceExecute(work,
work.getWorkItem().getSerializedSize());
+ }
+
+ private boolean shouldRetryLocally(String computationId, Work work,
Throwable t) {
+ if (KeyTokenInvalidException.isKeyTokenInvalidException(t)) {
+ LOG.debug(
+ "Execution of work for computation '{}' on key '{}' failed due to
token expiration. "
+ + "Work will not be retried locally.",
+ computationId,
+ work.getWorkItem().getKey().toStringUtf8());
+ } else if (WorkItemCancelledException.isWorkItemCancelledException(t)) {
+ LOG.debug(
+ "Execution of work for computation '{}' on key '{}' failed. "
+ + "Work will not be retried locally.",
+ computationId,
+ work.getWorkItem().getShardingKey());
+ } else {
+ LastExceptionDataProvider.reportException(t);
+ LOG.debug("Failed work: {}", work);
+ Duration elapsedTimeSinceStart = new Duration(work.getStartTime(),
clock.get());
+ if (!failureReporter.reportFailure(computationId, work.getWorkItem(),
t)) {
+ LOG.error(
+ "Execution of work for computation '{}' on key '{}' failed with
uncaught exception, "
+ + "and Windmill indicated not to retry locally.",
+ computationId,
+ work.getWorkItem().getKey().toStringUtf8(),
+ t);
+ } else if (isOutOfMemoryError(t)) {
+ LOG.error(
Review Comment:
`getHeapDumpLog` doesn't convey that is is creating the heap dump internally
I think the existing logic is clearer.
```
File heapDump = memoryMonitor.tryToDumpHeap();
LOG.error(
"Execution of work for computation '{}' for key '{}' failed
with out-of-memory. "
+ "Work will not be retried locally. Heap dump {}.",
computationId,
key.toStringUtf8(),
heapDump == null ? "not written" : ("written to '" + heapDump
+ "'"),
t);
```
##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/processing/failures/StreamingApplianceFailureReporter.java:
##########
@@ -0,0 +1,64 @@
+/*
+ * 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.processing.failures;
+
+import java.util.function.Function;
+import javax.annotation.concurrent.ThreadSafe;
+import org.apache.beam.runners.dataflow.worker.windmill.Windmill;
+import
org.apache.beam.runners.dataflow.worker.windmill.Windmill.ReportStatsRequest;
+import
org.apache.beam.runners.dataflow.worker.windmill.Windmill.ReportStatsResponse;
+import org.apache.beam.sdk.annotations.Internal;
+import
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.EvictingQueue;
+
+/** Implementation of {@link FailureReporter} that reports failures to
Streaming Appliance. */
+@ThreadSafe
+@Internal
+public final class StreamingApplianceFailureReporter extends FailureReporter {
+ private final Function<ReportStatsRequest, ReportStatsResponse>
reportStatsFn;
+
+ private StreamingApplianceFailureReporter(
+ int maxStackTraceDepthToReport,
+ EvictingQueue<String> pendingFailuresToReport,
+ Function<ReportStatsRequest, ReportStatsResponse> reportStatsFn) {
Review Comment:
Taking specific classes also helps with editor IntelliSense.
##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/harness/StreamingWorkerStatusReporter.java:
##########
@@ -0,0 +1,370 @@
+/*
+ * 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.harness;
+
+import static
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkNotNull;
+
+import com.google.api.services.dataflow.model.CounterUpdate;
+import com.google.api.services.dataflow.model.PerStepNamespaceMetrics;
+import com.google.api.services.dataflow.model.PerWorkerMetrics;
+import com.google.api.services.dataflow.model.Status;
+import com.google.api.services.dataflow.model.StreamingScalingReport;
+import com.google.api.services.dataflow.model.WorkItemStatus;
+import com.google.api.services.dataflow.model.WorkerMessage;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+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.AtomicLong;
+import java.util.function.Function;
+import java.util.function.Supplier;
+import javax.annotation.concurrent.ThreadSafe;
+import org.apache.beam.runners.dataflow.worker.DataflowSystemMetrics;
+import org.apache.beam.runners.dataflow.worker.StreamingStepMetricsContainer;
+import org.apache.beam.runners.dataflow.worker.WorkUnitClient;
+import org.apache.beam.runners.dataflow.worker.counters.CounterSet;
+import
org.apache.beam.runners.dataflow.worker.counters.DataflowCounterUpdateExtractor;
+import
org.apache.beam.runners.dataflow.worker.logging.DataflowWorkerLoggingMDC;
+import org.apache.beam.runners.dataflow.worker.streaming.StageInfo;
+import org.apache.beam.runners.dataflow.worker.util.BoundedQueueExecutor;
+import org.apache.beam.runners.dataflow.worker.util.MemoryMonitor;
+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.ImmutableList;
+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.util.concurrent.ThreadFactoryBuilder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Reports the status of the worker to Dataflow Service. */
+@Internal
+@ThreadSafe
+public final class StreamingWorkerStatusReporter {
+ private static final Logger LOG =
LoggerFactory.getLogger(StreamingWorkerStatusReporter.class);
+
+ // Reserved ID for counter updates.
+ // Matches kWindmillCounterUpdate in workflow_worker_service_multi_hubs.cc.
+ private static final String WINDMILL_COUNTER_UPDATE_WORK_ID = "3";
+ private static final int COUNTER_UPDATES_SIZE = 128;
+ 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 WorkUnitClient dataflowServiceClient;
+ private final Supplier<Long> windmillQuotaThrottleTime;
+ private final Supplier<Collection<StageInfo>> allStageInfo;
+ private final Supplier<ImmutableList<Status>> pendingErrorsToReport;
+ private final StreamingCounters streamingCounters;
+ private final MemoryMonitor memoryMonitor;
+ private final BoundedQueueExecutor workExecutor;
+ private final AtomicLong previousTimeAtMaxThreads;
+ private final Supplier<Integer> maxThreads;
+ private final ScheduledExecutorService globalWorkerUpdateReporter;
+ private final ScheduledExecutorService workerMessageReporter;
+
+ private StreamingWorkerStatusReporter(
+ boolean publishCounters,
+ WorkUnitClient dataflowServiceClient,
+ Supplier<Long> windmillQuotaThrottleTime,
+ Supplier<Collection<StageInfo>> allStageInfo,
+ Supplier<ImmutableList<Status>> pendingErrorsToReport,
+ StreamingCounters streamingCounters,
+ MemoryMonitor memoryMonitor,
+ BoundedQueueExecutor workExecutor,
+ AtomicLong previousTimeAtMaxThreads,
+ Supplier<Integer> maxThreads,
+ Function<String, ScheduledExecutorService> executorFactory) {
+ this.publishCounters = publishCounters;
+ this.dataflowServiceClient = dataflowServiceClient;
+ this.windmillQuotaThrottleTime = windmillQuotaThrottleTime;
+ this.allStageInfo = allStageInfo;
+ this.pendingErrorsToReport = pendingErrorsToReport;
+ this.streamingCounters = streamingCounters;
+ this.memoryMonitor = memoryMonitor;
+ this.workExecutor = workExecutor;
+ this.previousTimeAtMaxThreads = previousTimeAtMaxThreads;
+ this.maxThreads = maxThreads;
+ this.globalWorkerUpdateReporter =
executorFactory.apply(GLOBAL_WORKER_UPDATE_REPORTER_THREAD);
+ this.workerMessageReporter =
executorFactory.apply(WORKER_MESSAGE_REPORTER_THREAD);
+ }
+
+ public static StreamingWorkerStatusReporter create(
+ boolean publishCounters,
Review Comment:
This is used only in some tests to set publishCounters=false.
It doesn't look like we are doing using any dependencies under `if
(publishCounters)`, can we remove `publishCounters` and always set it to true?
##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/processing/failures/FailureReporter.java:
##########
@@ -0,0 +1,103 @@
+/*
+ * 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.processing.failures;
+
+import static
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList.toImmutableList;
+
+import com.google.api.services.dataflow.model.Status;
+import com.google.rpc.Code;
+import java.util.function.Supplier;
+import javax.annotation.concurrent.GuardedBy;
+import javax.annotation.concurrent.ThreadSafe;
+import org.apache.beam.runners.dataflow.worker.windmill.Windmill;
+import org.apache.beam.runners.dataflow.worker.windmill.Windmill.WorkItem;
+import org.apache.beam.sdk.annotations.Internal;
+import
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.EvictingQueue;
+import
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList;
+
+/** Reports failures that occur during user processing. */
+@ThreadSafe
+@Internal
+public abstract class FailureReporter implements
Supplier<ImmutableList<Status>> {
+
+ private final int maxStackTraceDepthToReport;
+
+ @GuardedBy("pendingFailuresToReport")
+ private final EvictingQueue<String> pendingFailuresToReport;
+
+ protected FailureReporter(
+ int maxStackTraceDepthToReport, EvictingQueue<String>
pendingFailuresToReport) {
Review Comment:
Consider changing this to `FailureReporter(int maxFailuresToReportInUpdate,
int maxStackTraceDepthToReport)` and create the `EvictingQueue` in the
constructor.
We don't need to expose `EvictingQueue` in the constructor. Not exposing it
outside the class also allows changing it without needing to modify the derived
classes.
##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/harness/StreamingWorkerStatusReporter.java:
##########
@@ -0,0 +1,370 @@
+/*
+ * 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.harness;
+
+import static
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkNotNull;
+
+import com.google.api.services.dataflow.model.CounterUpdate;
+import com.google.api.services.dataflow.model.PerStepNamespaceMetrics;
+import com.google.api.services.dataflow.model.PerWorkerMetrics;
+import com.google.api.services.dataflow.model.Status;
+import com.google.api.services.dataflow.model.StreamingScalingReport;
+import com.google.api.services.dataflow.model.WorkItemStatus;
+import com.google.api.services.dataflow.model.WorkerMessage;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+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.AtomicLong;
+import java.util.function.Function;
+import java.util.function.Supplier;
+import javax.annotation.concurrent.ThreadSafe;
+import org.apache.beam.runners.dataflow.worker.DataflowSystemMetrics;
+import org.apache.beam.runners.dataflow.worker.StreamingStepMetricsContainer;
+import org.apache.beam.runners.dataflow.worker.WorkUnitClient;
+import org.apache.beam.runners.dataflow.worker.counters.CounterSet;
+import
org.apache.beam.runners.dataflow.worker.counters.DataflowCounterUpdateExtractor;
+import
org.apache.beam.runners.dataflow.worker.logging.DataflowWorkerLoggingMDC;
+import org.apache.beam.runners.dataflow.worker.streaming.StageInfo;
+import org.apache.beam.runners.dataflow.worker.util.BoundedQueueExecutor;
+import org.apache.beam.runners.dataflow.worker.util.MemoryMonitor;
+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.ImmutableList;
+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.util.concurrent.ThreadFactoryBuilder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Reports the status of the worker to Dataflow Service. */
+@Internal
+@ThreadSafe
+public final class StreamingWorkerStatusReporter {
+ private static final Logger LOG =
LoggerFactory.getLogger(StreamingWorkerStatusReporter.class);
+
+ // Reserved ID for counter updates.
+ // Matches kWindmillCounterUpdate in workflow_worker_service_multi_hubs.cc.
+ private static final String WINDMILL_COUNTER_UPDATE_WORK_ID = "3";
+ private static final int COUNTER_UPDATES_SIZE = 128;
+ 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 WorkUnitClient dataflowServiceClient;
+ private final Supplier<Long> windmillQuotaThrottleTime;
+ private final Supplier<Collection<StageInfo>> allStageInfo;
+ private final Supplier<ImmutableList<Status>> pendingErrorsToReport;
+ private final StreamingCounters streamingCounters;
+ private final MemoryMonitor memoryMonitor;
+ private final BoundedQueueExecutor workExecutor;
+ private final AtomicLong previousTimeAtMaxThreads;
+ private final Supplier<Integer> maxThreads;
Review Comment:
can we rename `maxThreads` to `maxNumHarnessThreads`? It is unclear what
thread pool the maxThreads is reffering to
##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/harness/StreamingWorkerStatusReporter.java:
##########
@@ -0,0 +1,370 @@
+/*
+ * 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.harness;
+
+import static
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkNotNull;
+
+import com.google.api.services.dataflow.model.CounterUpdate;
+import com.google.api.services.dataflow.model.PerStepNamespaceMetrics;
+import com.google.api.services.dataflow.model.PerWorkerMetrics;
+import com.google.api.services.dataflow.model.Status;
+import com.google.api.services.dataflow.model.StreamingScalingReport;
+import com.google.api.services.dataflow.model.WorkItemStatus;
+import com.google.api.services.dataflow.model.WorkerMessage;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+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.AtomicLong;
+import java.util.function.Function;
+import java.util.function.Supplier;
+import javax.annotation.concurrent.ThreadSafe;
+import org.apache.beam.runners.dataflow.worker.DataflowSystemMetrics;
+import org.apache.beam.runners.dataflow.worker.StreamingStepMetricsContainer;
+import org.apache.beam.runners.dataflow.worker.WorkUnitClient;
+import org.apache.beam.runners.dataflow.worker.counters.CounterSet;
+import
org.apache.beam.runners.dataflow.worker.counters.DataflowCounterUpdateExtractor;
+import
org.apache.beam.runners.dataflow.worker.logging.DataflowWorkerLoggingMDC;
+import org.apache.beam.runners.dataflow.worker.streaming.StageInfo;
+import org.apache.beam.runners.dataflow.worker.util.BoundedQueueExecutor;
+import org.apache.beam.runners.dataflow.worker.util.MemoryMonitor;
+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.ImmutableList;
+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.util.concurrent.ThreadFactoryBuilder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Reports the status of the worker to Dataflow Service. */
+@Internal
+@ThreadSafe
+public final class StreamingWorkerStatusReporter {
+ private static final Logger LOG =
LoggerFactory.getLogger(StreamingWorkerStatusReporter.class);
+
+ // Reserved ID for counter updates.
+ // Matches kWindmillCounterUpdate in workflow_worker_service_multi_hubs.cc.
+ private static final String WINDMILL_COUNTER_UPDATE_WORK_ID = "3";
+ private static final int COUNTER_UPDATES_SIZE = 128;
+ 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 WorkUnitClient dataflowServiceClient;
+ private final Supplier<Long> windmillQuotaThrottleTime;
+ private final Supplier<Collection<StageInfo>> allStageInfo;
+ private final Supplier<ImmutableList<Status>> pendingErrorsToReport;
+ private final StreamingCounters streamingCounters;
+ private final MemoryMonitor memoryMonitor;
+ private final BoundedQueueExecutor workExecutor;
+ private final AtomicLong previousTimeAtMaxThreads;
+ private final Supplier<Integer> maxThreads;
+ private final ScheduledExecutorService globalWorkerUpdateReporter;
+ private final ScheduledExecutorService workerMessageReporter;
+
+ private StreamingWorkerStatusReporter(
+ boolean publishCounters,
+ WorkUnitClient dataflowServiceClient,
+ Supplier<Long> windmillQuotaThrottleTime,
+ Supplier<Collection<StageInfo>> allStageInfo,
+ Supplier<ImmutableList<Status>> pendingErrorsToReport,
+ StreamingCounters streamingCounters,
+ MemoryMonitor memoryMonitor,
+ BoundedQueueExecutor workExecutor,
+ AtomicLong previousTimeAtMaxThreads,
+ Supplier<Integer> maxThreads,
+ Function<String, ScheduledExecutorService> executorFactory) {
+ this.publishCounters = publishCounters;
+ this.dataflowServiceClient = dataflowServiceClient;
+ this.windmillQuotaThrottleTime = windmillQuotaThrottleTime;
+ this.allStageInfo = allStageInfo;
+ this.pendingErrorsToReport = pendingErrorsToReport;
+ this.streamingCounters = streamingCounters;
+ this.memoryMonitor = memoryMonitor;
+ this.workExecutor = workExecutor;
+ this.previousTimeAtMaxThreads = previousTimeAtMaxThreads;
+ this.maxThreads = maxThreads;
+ this.globalWorkerUpdateReporter =
executorFactory.apply(GLOBAL_WORKER_UPDATE_REPORTER_THREAD);
+ this.workerMessageReporter =
executorFactory.apply(WORKER_MESSAGE_REPORTER_THREAD);
+ }
+
+ public static StreamingWorkerStatusReporter create(
+ boolean publishCounters,
+ WorkUnitClient workUnitClient,
+ Supplier<Long> windmillQuotaThrottleTime,
+ Supplier<Collection<StageInfo>> allStageInfo,
+ Supplier<ImmutableList<Status>> pendingErrorsToReport,
+ StreamingCounters streamingCounters,
+ MemoryMonitor memoryMonitor,
+ BoundedQueueExecutor workExecutor,
+ Supplier<Integer> maxThreads) {
+ return new StreamingWorkerStatusReporter(
+ publishCounters,
+ workUnitClient,
+ windmillQuotaThrottleTime,
+ allStageInfo,
+ pendingErrorsToReport,
+ streamingCounters,
+ memoryMonitor,
+ workExecutor,
+ new AtomicLong(),
+ maxThreads,
+ threadName ->
+ Executors.newSingleThreadScheduledExecutor(
+ new ThreadFactoryBuilder().setNameFormat(threadName).build()));
+ }
+
+ @VisibleForTesting
+ public static StreamingWorkerStatusReporter forTesting(
+ boolean publishCounters,
+ WorkUnitClient workUnitClient,
+ Supplier<Long> windmillQuotaThrottleTime,
+ Supplier<Collection<StageInfo>> allStageInfo,
+ Supplier<ImmutableList<Status>> pendingErrorsToReport,
+ StreamingCounters streamingCounters,
+ MemoryMonitor memoryMonitor,
+ BoundedQueueExecutor workExecutor,
+ Supplier<Integer> maxThreads,
+ Function<String, ScheduledExecutorService> executorFactory) {
+ return new StreamingWorkerStatusReporter(
+ publishCounters,
+ workUnitClient,
+ windmillQuotaThrottleTime,
+ allStageInfo,
+ pendingErrorsToReport,
+ streamingCounters,
+ memoryMonitor,
+ workExecutor,
+ new AtomicLong(),
+ maxThreads,
+ executorFactory);
+ }
+
+ /**
+ * Returns key for a counter update. It is a String in case of legacy
counter and
+ * CounterStructuredName in the case of a structured counter.
+ */
+ private static Object getCounterUpdateKey(CounterUpdate counterUpdate) {
+ Object key = null;
+ if (counterUpdate.getNameAndKind() != null) {
+ key = counterUpdate.getNameAndKind().getName();
+ } else if (counterUpdate.getStructuredNameAndMetadata() != null) {
+ key = counterUpdate.getStructuredNameAndMetadata().getName();
+ }
+ return checkNotNull(key, "Could not find name for CounterUpdate: %s",
counterUpdate);
+ }
+
+ /**
+ * Clears counterUpdates and enqueues unique counters from counterMultimap.
If a counter appears
+ * more than once, one of them is extracted leaving the remaining in the map.
+ */
+ private static void extractUniqueCounters(
+ List<CounterUpdate> counterUpdates, ListMultimap<Object, CounterUpdate>
counterMultimap) {
+ counterUpdates.clear();
+ for (Iterator<Object> iter = counterMultimap.keySet().iterator();
iter.hasNext(); ) {
+ List<CounterUpdate> counters = counterMultimap.get(iter.next());
+ counterUpdates.add(counters.get(0));
+ if (counters.size() == 1) {
+ // There is single value. Remove the entry through the iterator.
+ iter.remove();
+ } else {
+ // Otherwise remove the first value.
+ counters.remove(0);
+ }
+ }
+ }
+
+ private static void shutdownExecutor(ScheduledExecutorService executor) {
+ executor.shutdown();
+ try {
+ executor.awaitTermination(10, TimeUnit.SECONDS);
+ } catch (InterruptedException e) {
+ LOG.warn("Error occurred trying to gracefully shutdown executor={}",
executor, e);
+ executor.shutdownNow();
+ }
+ }
+
+ @SuppressWarnings("FutureReturnValueIgnored")
+ public void start(long windmillHarnessUpdateReportingPeriod) {
+ reportHarnessStartup();
+ // Periodically report workers counters and other updates.
+ globalWorkerUpdateReporter.scheduleWithFixedDelay(
Review Comment:
do we also need to guard this by `if (windmillHarnessUpdateReportingPeriod >
0)`?
Or we could add a check not <= 0 and remove the if check.
##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/harness/StreamingCounters.java:
##########
@@ -0,0 +1,134 @@
+/*
+ * 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.harness;
+
+import com.google.auto.value.AutoValue;
+import java.util.concurrent.atomic.AtomicLong;
+import
org.apache.beam.runners.dataflow.worker.DataflowSystemMetrics.StreamingSystemCounterNames;
+import org.apache.beam.runners.dataflow.worker.counters.Counter;
+import org.apache.beam.runners.dataflow.worker.counters.CounterSet;
+import org.apache.beam.runners.dataflow.worker.util.BoundedQueueExecutor;
+import org.apache.beam.sdk.annotations.Internal;
+
+/** Streaming pipeline counters to report pipeline processing metrics to
Dataflow backend. */
+@Internal
+@AutoValue
+public abstract class StreamingCounters {
+
+ public static StreamingCounters create() {
+ CounterSet pendingDeltaCounters = new CounterSet();
+ CounterSet pendingCumulativeCounters = new CounterSet();
+ return new AutoValue_StreamingCounters(
+ pendingDeltaCounters,
+ pendingCumulativeCounters,
+ pendingDeltaCounters.longSum(
+
StreamingSystemCounterNames.WINDMILL_SHUFFLE_BYTES_READ.counterName()),
+ pendingDeltaCounters.longSum(
+
StreamingSystemCounterNames.WINDMILL_STATE_BYTES_READ.counterName()),
+ pendingDeltaCounters.longSum(
+
StreamingSystemCounterNames.WINDMILL_STATE_BYTES_WRITTEN.counterName()),
+ pendingDeltaCounters.longSum(
+
StreamingSystemCounterNames.WINDMILL_QUOTA_THROTTLING.counterName()),
+ pendingDeltaCounters.longSum(
+
StreamingSystemCounterNames.TIME_AT_MAX_ACTIVE_THREADS.counterName()),
+ pendingCumulativeCounters.longSum(
+
StreamingSystemCounterNames.JAVA_HARNESS_USED_MEMORY.counterName()),
+ pendingCumulativeCounters.longSum(
+ StreamingSystemCounterNames.JAVA_HARNESS_MAX_MEMORY.counterName()),
+
pendingCumulativeCounters.intSum(StreamingSystemCounterNames.ACTIVE_THREADS.counterName()),
+ pendingCumulativeCounters.intSum(
+ StreamingSystemCounterNames.TOTAL_ALLOCATED_THREADS.counterName()),
+ pendingCumulativeCounters.longSum(
+ StreamingSystemCounterNames.OUTSTANDING_BYTES.counterName()),
+ pendingCumulativeCounters.longSum(
+ StreamingSystemCounterNames.MAX_OUTSTANDING_BYTES.counterName()),
+ pendingCumulativeCounters.longSum(
+ StreamingSystemCounterNames.OUTSTANDING_BUNDLES.counterName()),
+ pendingCumulativeCounters.longSum(
+ StreamingSystemCounterNames.MAX_OUTSTANDING_BUNDLES.counterName()),
+ pendingCumulativeCounters.intMax(
+
StreamingSystemCounterNames.WINDMILL_MAX_WORK_ITEM_COMMIT_BYTES.counterName()),
+ pendingCumulativeCounters.intSum(
+ StreamingSystemCounterNames.MEMORY_THRASHING.counterName()));
+ }
+
+ public abstract CounterSet pendingDeltaCounters();
+
+ public abstract CounterSet pendingCumulativeCounters();
+ // Built-in delta counters.
+ public abstract Counter<Long, Long> windmillShuffleBytesRead();
+
+ public abstract Counter<Long, Long> windmillStateBytesRead();
+
+ public abstract Counter<Long, Long> windmillStateBytesWritten();
+
+ public abstract Counter<Long, Long> windmillQuotaThrottling();
+
+ public abstract Counter<Long, Long> timeAtMaxActiveThreads();
+ // Built-in cumulative counters.
+ public abstract Counter<Long, Long> javaHarnessUsedMemory();
+
+ public abstract Counter<Long, Long> javaHarnessMaxMemory();
+
+ public abstract Counter<Integer, Integer> activeThreads();
+
+ public abstract Counter<Integer, Integer> totalAllocatedThreads();
+
+ public abstract Counter<Long, Long> outstandingBytes();
+
+ public abstract Counter<Long, Long> maxOutstandingBytes();
+
+ public abstract Counter<Long, Long> outstandingBundles();
+
+ public abstract Counter<Long, Long> maxOutstandingBundles();
+
+ public abstract Counter<Integer, Integer>
windmillMaxObservedWorkItemCommitBytes();
+
+ public abstract Counter<Integer, Integer> memoryThrashing();
+
+ final void updateVMMetrics() {
+ Runtime rt = Runtime.getRuntime();
+ long usedMemory = rt.totalMemory() - rt.freeMemory();
+ long maxMemory = rt.maxMemory();
+
+ javaHarnessUsedMemory().getAndReset();
+ javaHarnessUsedMemory().addValue(usedMemory);
+ javaHarnessMaxMemory().getAndReset();
+ javaHarnessMaxMemory().addValue(maxMemory);
+ }
+
+ final void updateThreadMetrics(
+ BoundedQueueExecutor workUnitExecutor, AtomicLong
previousTimeAtMaxThreads, int maxThreads) {
+ timeAtMaxActiveThreads().getAndReset();
+ long allThreadsActiveTime = workUnitExecutor.allThreadsActiveTime();
+ timeAtMaxActiveThreads().addValue(allThreadsActiveTime -
previousTimeAtMaxThreads.get());
+ previousTimeAtMaxThreads.set(allThreadsActiveTime);
+ activeThreads().getAndReset();
+ activeThreads().addValue(workUnitExecutor.activeCount());
+ totalAllocatedThreads().getAndReset();
+ totalAllocatedThreads().addValue(maxThreads);
Review Comment:
you can expose and get `maxThreads` from `workUnitExecutor` and remove
`maxThreads`
##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/harness/StreamingWorkerStatusReporter.java:
##########
@@ -0,0 +1,370 @@
+/*
+ * 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.harness;
+
+import static
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkNotNull;
+
+import com.google.api.services.dataflow.model.CounterUpdate;
+import com.google.api.services.dataflow.model.PerStepNamespaceMetrics;
+import com.google.api.services.dataflow.model.PerWorkerMetrics;
+import com.google.api.services.dataflow.model.Status;
+import com.google.api.services.dataflow.model.StreamingScalingReport;
+import com.google.api.services.dataflow.model.WorkItemStatus;
+import com.google.api.services.dataflow.model.WorkerMessage;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+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.AtomicLong;
+import java.util.function.Function;
+import java.util.function.Supplier;
+import javax.annotation.concurrent.ThreadSafe;
+import org.apache.beam.runners.dataflow.worker.DataflowSystemMetrics;
+import org.apache.beam.runners.dataflow.worker.StreamingStepMetricsContainer;
+import org.apache.beam.runners.dataflow.worker.WorkUnitClient;
+import org.apache.beam.runners.dataflow.worker.counters.CounterSet;
+import
org.apache.beam.runners.dataflow.worker.counters.DataflowCounterUpdateExtractor;
+import
org.apache.beam.runners.dataflow.worker.logging.DataflowWorkerLoggingMDC;
+import org.apache.beam.runners.dataflow.worker.streaming.StageInfo;
+import org.apache.beam.runners.dataflow.worker.util.BoundedQueueExecutor;
+import org.apache.beam.runners.dataflow.worker.util.MemoryMonitor;
+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.ImmutableList;
+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.util.concurrent.ThreadFactoryBuilder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Reports the status of the worker to Dataflow Service. */
+@Internal
+@ThreadSafe
+public final class StreamingWorkerStatusReporter {
+ private static final Logger LOG =
LoggerFactory.getLogger(StreamingWorkerStatusReporter.class);
+
+ // Reserved ID for counter updates.
+ // Matches kWindmillCounterUpdate in workflow_worker_service_multi_hubs.cc.
+ private static final String WINDMILL_COUNTER_UPDATE_WORK_ID = "3";
+ private static final int COUNTER_UPDATES_SIZE = 128;
+ 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 WorkUnitClient dataflowServiceClient;
+ private final Supplier<Long> windmillQuotaThrottleTime;
+ private final Supplier<Collection<StageInfo>> allStageInfo;
+ private final Supplier<ImmutableList<Status>> pendingErrorsToReport;
+ private final StreamingCounters streamingCounters;
+ private final MemoryMonitor memoryMonitor;
+ private final BoundedQueueExecutor workExecutor;
+ private final AtomicLong previousTimeAtMaxThreads;
+ private final Supplier<Integer> maxThreads;
+ private final ScheduledExecutorService globalWorkerUpdateReporter;
+ private final ScheduledExecutorService workerMessageReporter;
+
+ private StreamingWorkerStatusReporter(
+ boolean publishCounters,
+ WorkUnitClient dataflowServiceClient,
+ Supplier<Long> windmillQuotaThrottleTime,
+ Supplier<Collection<StageInfo>> allStageInfo,
+ Supplier<ImmutableList<Status>> pendingErrorsToReport,
+ StreamingCounters streamingCounters,
+ MemoryMonitor memoryMonitor,
+ BoundedQueueExecutor workExecutor,
+ AtomicLong previousTimeAtMaxThreads,
Review Comment:
looks like this is always `new AtomicLong()`, can we remove this from
constructor and initialize to `new AtomicLong()`?
##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/processing/failures/FailureReporter.java:
##########
@@ -0,0 +1,103 @@
+/*
+ * 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.processing.failures;
+
+import static
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList.toImmutableList;
+
+import com.google.api.services.dataflow.model.Status;
+import com.google.rpc.Code;
+import java.util.function.Supplier;
+import javax.annotation.concurrent.GuardedBy;
+import javax.annotation.concurrent.ThreadSafe;
+import org.apache.beam.runners.dataflow.worker.windmill.Windmill;
+import org.apache.beam.runners.dataflow.worker.windmill.Windmill.WorkItem;
+import org.apache.beam.sdk.annotations.Internal;
+import
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.EvictingQueue;
+import
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList;
+
+/** Reports failures that occur during user processing. */
+@ThreadSafe
+@Internal
+public abstract class FailureReporter implements
Supplier<ImmutableList<Status>> {
+
+ private final int maxStackTraceDepthToReport;
+
+ @GuardedBy("pendingFailuresToReport")
+ private final EvictingQueue<String> pendingFailuresToReport;
+
+ protected FailureReporter(
+ int maxStackTraceDepthToReport, EvictingQueue<String>
pendingFailuresToReport) {
+ this.maxStackTraceDepthToReport = maxStackTraceDepthToReport;
+ this.pendingFailuresToReport = pendingFailuresToReport;
+ }
+
+ /**
+ * Reports the failure to streaming backend. Returns whether the processing
should be retried
Review Comment:
IIUC, the returned boolean says if the reportFailure succeeded or not. The
decision to retry locally is made upstream. We should rename
`shouldRetryLocally` to something like `reportFailureInternal`
##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/processing/failures/WorkFailureProcessor.java:
##########
@@ -0,0 +1,160 @@
+/*
+ * 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.processing.failures;
+
+import java.io.File;
+import java.util.Optional;
+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.KeyTokenInvalidException;
+import org.apache.beam.runners.dataflow.worker.WorkItemCancelledException;
+import
org.apache.beam.runners.dataflow.worker.status.LastExceptionDataProvider;
+import org.apache.beam.runners.dataflow.worker.streaming.Work;
+import org.apache.beam.runners.dataflow.worker.util.BoundedQueueExecutor;
+import org.apache.beam.sdk.annotations.Internal;
+import org.apache.beam.sdk.util.UserCodeException;
+import
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.Uninterruptibles;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Processes a failure that occurs during user processing of {@link Work}. */
+@ThreadSafe
+@Internal
+public final class WorkFailureProcessor {
+ private static final Logger LOG =
LoggerFactory.getLogger(WorkFailureProcessor.class);
+ private static final Duration MAX_LOCAL_PROCESSING_RETRY_DURATION =
Duration.standardMinutes(5);
+
+ private final BoundedQueueExecutor workUnitExecutor;
+ private final FailureReporter failureReporter;
+ private final Supplier<Optional<File>> heapDumpFetcher;
Review Comment:
I would take `MemoryMonitor` or a new `HeapDumper` instead of
`Supplier<Optional<File>>`
##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/harness/StreamingWorkerStatusReporter.java:
##########
@@ -0,0 +1,370 @@
+/*
+ * 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.harness;
+
+import static
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkNotNull;
+
+import com.google.api.services.dataflow.model.CounterUpdate;
+import com.google.api.services.dataflow.model.PerStepNamespaceMetrics;
+import com.google.api.services.dataflow.model.PerWorkerMetrics;
+import com.google.api.services.dataflow.model.Status;
+import com.google.api.services.dataflow.model.StreamingScalingReport;
+import com.google.api.services.dataflow.model.WorkItemStatus;
+import com.google.api.services.dataflow.model.WorkerMessage;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+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.AtomicLong;
+import java.util.function.Function;
+import java.util.function.Supplier;
+import javax.annotation.concurrent.ThreadSafe;
+import org.apache.beam.runners.dataflow.worker.DataflowSystemMetrics;
+import org.apache.beam.runners.dataflow.worker.StreamingStepMetricsContainer;
+import org.apache.beam.runners.dataflow.worker.WorkUnitClient;
+import org.apache.beam.runners.dataflow.worker.counters.CounterSet;
+import
org.apache.beam.runners.dataflow.worker.counters.DataflowCounterUpdateExtractor;
+import
org.apache.beam.runners.dataflow.worker.logging.DataflowWorkerLoggingMDC;
+import org.apache.beam.runners.dataflow.worker.streaming.StageInfo;
+import org.apache.beam.runners.dataflow.worker.util.BoundedQueueExecutor;
+import org.apache.beam.runners.dataflow.worker.util.MemoryMonitor;
+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.ImmutableList;
+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.util.concurrent.ThreadFactoryBuilder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Reports the status of the worker to Dataflow Service. */
+@Internal
+@ThreadSafe
+public final class StreamingWorkerStatusReporter {
+ private static final Logger LOG =
LoggerFactory.getLogger(StreamingWorkerStatusReporter.class);
+
+ // Reserved ID for counter updates.
+ // Matches kWindmillCounterUpdate in workflow_worker_service_multi_hubs.cc.
+ private static final String WINDMILL_COUNTER_UPDATE_WORK_ID = "3";
+ private static final int COUNTER_UPDATES_SIZE = 128;
+ 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 WorkUnitClient dataflowServiceClient;
+ private final Supplier<Long> windmillQuotaThrottleTime;
+ private final Supplier<Collection<StageInfo>> allStageInfo;
+ private final Supplier<ImmutableList<Status>> pendingErrorsToReport;
+ private final StreamingCounters streamingCounters;
+ private final MemoryMonitor memoryMonitor;
+ private final BoundedQueueExecutor workExecutor;
+ private final AtomicLong previousTimeAtMaxThreads;
+ private final Supplier<Integer> maxThreads;
+ private final ScheduledExecutorService globalWorkerUpdateReporter;
+ private final ScheduledExecutorService workerMessageReporter;
+
+ private StreamingWorkerStatusReporter(
+ boolean publishCounters,
+ WorkUnitClient dataflowServiceClient,
+ Supplier<Long> windmillQuotaThrottleTime,
+ Supplier<Collection<StageInfo>> allStageInfo,
+ Supplier<ImmutableList<Status>> pendingErrorsToReport,
+ StreamingCounters streamingCounters,
+ MemoryMonitor memoryMonitor,
+ BoundedQueueExecutor workExecutor,
+ AtomicLong previousTimeAtMaxThreads,
+ Supplier<Integer> maxThreads,
+ Function<String, ScheduledExecutorService> executorFactory) {
+ this.publishCounters = publishCounters;
+ this.dataflowServiceClient = dataflowServiceClient;
+ this.windmillQuotaThrottleTime = windmillQuotaThrottleTime;
+ this.allStageInfo = allStageInfo;
+ this.pendingErrorsToReport = pendingErrorsToReport;
+ this.streamingCounters = streamingCounters;
+ this.memoryMonitor = memoryMonitor;
+ this.workExecutor = workExecutor;
+ this.previousTimeAtMaxThreads = previousTimeAtMaxThreads;
+ this.maxThreads = maxThreads;
+ this.globalWorkerUpdateReporter =
executorFactory.apply(GLOBAL_WORKER_UPDATE_REPORTER_THREAD);
+ this.workerMessageReporter =
executorFactory.apply(WORKER_MESSAGE_REPORTER_THREAD);
+ }
+
+ public static StreamingWorkerStatusReporter create(
+ boolean publishCounters,
+ WorkUnitClient workUnitClient,
+ Supplier<Long> windmillQuotaThrottleTime,
+ Supplier<Collection<StageInfo>> allStageInfo,
+ Supplier<ImmutableList<Status>> pendingErrorsToReport,
Review Comment:
Prefer taking specific types over Supplier when possible.
Take `FailureReporter` here, It took a minute to figure out who is calling
FailureReporter::get
##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/processing/failures/StreamingApplianceFailureReporter.java:
##########
@@ -0,0 +1,64 @@
+/*
+ * 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.processing.failures;
+
+import java.util.function.Function;
+import javax.annotation.concurrent.ThreadSafe;
+import org.apache.beam.runners.dataflow.worker.windmill.Windmill;
+import
org.apache.beam.runners.dataflow.worker.windmill.Windmill.ReportStatsRequest;
+import
org.apache.beam.runners.dataflow.worker.windmill.Windmill.ReportStatsResponse;
+import org.apache.beam.sdk.annotations.Internal;
+import
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.EvictingQueue;
+
+/** Implementation of {@link FailureReporter} that reports failures to
Streaming Appliance. */
+@ThreadSafe
+@Internal
+public final class StreamingApplianceFailureReporter extends FailureReporter {
+ private final Function<ReportStatsRequest, ReportStatsResponse>
reportStatsFn;
+
+ private StreamingApplianceFailureReporter(
+ int maxStackTraceDepthToReport,
+ EvictingQueue<String> pendingFailuresToReport,
+ Function<ReportStatsRequest, ReportStatsResponse> reportStatsFn) {
Review Comment:
Instead of `Function<>` Can we take WindmillServerStub or a specialized
WindmillStatsReporter interface?
It is hard to follow what reportStatsFn does, It looked like a transform
function to me.
##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/processing/failures/FailureReporter.java:
##########
@@ -0,0 +1,103 @@
+/*
+ * 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.processing.failures;
+
+import static
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList.toImmutableList;
+
+import com.google.api.services.dataflow.model.Status;
+import com.google.rpc.Code;
+import java.util.function.Supplier;
+import javax.annotation.concurrent.GuardedBy;
+import javax.annotation.concurrent.ThreadSafe;
+import org.apache.beam.runners.dataflow.worker.windmill.Windmill;
+import org.apache.beam.runners.dataflow.worker.windmill.Windmill.WorkItem;
+import org.apache.beam.sdk.annotations.Internal;
+import
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.EvictingQueue;
+import
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList;
+
+/** Reports failures that occur during user processing. */
+@ThreadSafe
+@Internal
+public abstract class FailureReporter implements
Supplier<ImmutableList<Status>> {
Review Comment:
is `FailureTracker` is better than `FailureReporter`?
Since in addition to reporting failures to windmill, this also tracks and
returns the errors to be sent to DFE?
##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/processing/failures/FailureReporter.java:
##########
@@ -0,0 +1,103 @@
+/*
+ * 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.processing.failures;
+
+import static
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList.toImmutableList;
+
+import com.google.api.services.dataflow.model.Status;
+import com.google.rpc.Code;
+import java.util.function.Supplier;
+import javax.annotation.concurrent.GuardedBy;
+import javax.annotation.concurrent.ThreadSafe;
+import org.apache.beam.runners.dataflow.worker.windmill.Windmill;
+import org.apache.beam.runners.dataflow.worker.windmill.Windmill.WorkItem;
+import org.apache.beam.sdk.annotations.Internal;
+import
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.EvictingQueue;
+import
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList;
+
+/** Reports failures that occur during user processing. */
+@ThreadSafe
+@Internal
+public abstract class FailureReporter implements
Supplier<ImmutableList<Status>> {
+
+ private final int maxStackTraceDepthToReport;
+
+ @GuardedBy("pendingFailuresToReport")
+ private final EvictingQueue<String> pendingFailuresToReport;
+
+ protected FailureReporter(
+ int maxStackTraceDepthToReport, EvictingQueue<String>
pendingFailuresToReport) {
+ this.maxStackTraceDepthToReport = maxStackTraceDepthToReport;
+ this.pendingFailuresToReport = pendingFailuresToReport;
+ }
+
+ /**
+ * Reports the failure to streaming backend. Returns whether the processing
should be retried
+ * locally.
+ */
+ public boolean reportFailure(String computationId, WorkItem work, Throwable
failure) {
+ // Adds the given failure message to the queue of messages to be reported
to DFE in periodic
+ // updates.
+ synchronized (pendingFailuresToReport) {
+ pendingFailuresToReport.add(buildExceptionStackTrace(failure));
+ }
+ return shouldRetryLocally(computationId, work);
+ }
+
+ @Override
+ public ImmutableList<Status> get() {
Review Comment:
Can we remove the `implements Supplier<ImmutableList<Status>>` and name this
`getErrorsToReport()`?
##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/processing/failures/WorkFailureProcessor.java:
##########
@@ -0,0 +1,160 @@
+/*
+ * 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.processing.failures;
+
+import java.io.File;
+import java.util.Optional;
+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.KeyTokenInvalidException;
+import org.apache.beam.runners.dataflow.worker.WorkItemCancelledException;
+import
org.apache.beam.runners.dataflow.worker.status.LastExceptionDataProvider;
+import org.apache.beam.runners.dataflow.worker.streaming.Work;
+import org.apache.beam.runners.dataflow.worker.util.BoundedQueueExecutor;
+import org.apache.beam.sdk.annotations.Internal;
+import org.apache.beam.sdk.util.UserCodeException;
+import
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.Uninterruptibles;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Processes a failure that occurs during user processing of {@link Work}. */
+@ThreadSafe
+@Internal
+public final class WorkFailureProcessor {
+ private static final Logger LOG =
LoggerFactory.getLogger(WorkFailureProcessor.class);
+ private static final Duration MAX_LOCAL_PROCESSING_RETRY_DURATION =
Duration.standardMinutes(5);
+
+ private final BoundedQueueExecutor workUnitExecutor;
+ private final FailureReporter failureReporter;
+ private final Supplier<Optional<File>> heapDumpFetcher;
+ private final Supplier<Instant> clock;
+ private final int retryLocallyDelayMs;
Review Comment:
Move DEFAULT_RETRY_LOCALLY_MS to this file and remove retryLocallyDelayMs?
Then we won't need to make DEFAULT_RETRY_LOCALLY_MS visible to testing.
##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/processing/failures/WorkFailureProcessor.java:
##########
@@ -0,0 +1,160 @@
+/*
+ * 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.processing.failures;
+
+import java.io.File;
+import java.util.Optional;
+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.KeyTokenInvalidException;
+import org.apache.beam.runners.dataflow.worker.WorkItemCancelledException;
+import
org.apache.beam.runners.dataflow.worker.status.LastExceptionDataProvider;
+import org.apache.beam.runners.dataflow.worker.streaming.Work;
+import org.apache.beam.runners.dataflow.worker.util.BoundedQueueExecutor;
+import org.apache.beam.sdk.annotations.Internal;
+import org.apache.beam.sdk.util.UserCodeException;
+import
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.Uninterruptibles;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Processes a failure that occurs during user processing of {@link Work}. */
+@ThreadSafe
+@Internal
+public final class WorkFailureProcessor {
+ private static final Logger LOG =
LoggerFactory.getLogger(WorkFailureProcessor.class);
+ private static final Duration MAX_LOCAL_PROCESSING_RETRY_DURATION =
Duration.standardMinutes(5);
+
+ private final BoundedQueueExecutor workUnitExecutor;
+ private final FailureReporter failureReporter;
+ private final Supplier<Optional<File>> heapDumpFetcher;
+ private final Supplier<Instant> clock;
+ private final int retryLocallyDelayMs;
+
+ public WorkFailureProcessor(
+ BoundedQueueExecutor workUnitExecutor,
+ FailureReporter failureReporter,
+ Supplier<Optional<File>> heapDumpFetcher,
+ Supplier<Instant> clock,
+ int retryLocallyDelayMs) {
+ this.workUnitExecutor = workUnitExecutor;
+ this.failureReporter = failureReporter;
+ this.heapDumpFetcher = heapDumpFetcher;
+ this.clock = clock;
+ this.retryLocallyDelayMs = retryLocallyDelayMs;
+ }
+
+ /** Returns whether an exception was caused by a {@link OutOfMemoryError}. */
+ private static boolean isOutOfMemoryError(Throwable t) {
+ while (t != null) {
+ if (t instanceof OutOfMemoryError) {
+ return true;
+ }
+ t = t.getCause();
+ }
+ return false;
+ }
+
+ /**
+ * Processes failures caused by thrown exceptions that occur during
execution of {@link Work}. May
+ * attempt to retry execution of the {@link Work} or drop it if it is
invalid.
+ */
+ public void logAndProcessFailure(
+ String computationId, Work work, Throwable t, Consumer<Work>
onInvalidWork) {
+ if (shouldRetryLocally(
+ computationId, work, t instanceof UserCodeException ? t.getCause() :
t)) {
Review Comment:
move `t instanceof UserCodeException ? t.getCause() : t` inside
`shouldRetryLocally`?
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]