Skip to content

Commit

Permalink
address cl comments
Browse files Browse the repository at this point in the history
  • Loading branch information
m-trieu committed Apr 29, 2024
1 parent f8732c8 commit e60cb1e
Show file tree
Hide file tree
Showing 30 changed files with 375 additions and 83 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -37,7 +37,7 @@ private MetricsLogger(@Nullable String stepName) {
super(stepName);
}

public static MetricsLogger createUnboundedMetricsLogger() {
public static MetricsLogger forWorkerMetrics() {
return new MetricsLogger(null);
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -41,7 +41,7 @@ public void testGeneratedLogMessageShowsDeltas() {
MonitoringInfoConstants.Urns.ELEMENT_COUNT,
Collections.singletonMap("name", "histogram"));

MetricsLogger logger = MetricsLogger.createUnboundedMetricsLogger();
MetricsLogger logger = MetricsLogger.forWorkerMetrics();
logger.getCounter(cName).inc(2L);
// Set buckets counts to: [0,1,1,,0,0,...]
logger.getHistogram(hName, bucketType).update(1);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@
import org.apache.beam.sdk.options.Default;
import org.apache.beam.sdk.options.DefaultValueFactory;
import org.apache.beam.sdk.options.Description;
import org.apache.beam.sdk.options.ExperimentalOptions;
import org.apache.beam.sdk.options.Hidden;
import org.apache.beam.sdk.options.PipelineOptions;
import org.joda.time.Duration;
Expand Down Expand Up @@ -224,10 +225,10 @@ public interface DataflowStreamingPipelineOptions extends PipelineOptions {
@Description(
"If true, Dataflow streaming pipeline will be running in direct path mode."
+ " VMs must have IPv6 enabled for this to work.")
@Default.Boolean(false)
boolean getIsWindmillServiceDirectPathEnabled();
@Default.InstanceFactory(EnableWindmillServiceDirectPathFactory.class)
boolean isEnableWindmillServiceDirectPath();

void setIsWindmillServiceDirectPathEnabled(boolean isWindmillServiceDirectPathEnabled);
void setEnableWindmillServiceDirectPath(boolean value);

/**
* Factory for creating local Windmill address. Reads from system propery 'windmill.hostport' for
Expand Down Expand Up @@ -302,4 +303,13 @@ public Integer create(PipelineOptions options) {
return streamingOptions.isEnableStreamingEngine() ? Integer.MAX_VALUE : 1;
}
}

/** EnableStreamingEngine defaults to false unless one of the two experiments is set. */
class EnableWindmillServiceDirectPathFactory implements DefaultValueFactory<Boolean> {
private static final String DIRECT_PATH_EXPERIMENT = "enable_windmill_service_direct_path";
@Override
public Boolean create(PipelineOptions options) {
return ExperimentalOptions.hasExperiment(options, DIRECT_PATH_EXPERIMENT);
}
}
}
1 change: 0 additions & 1 deletion runners/google-cloud-dataflow-java/worker/build.gradle
Original file line number Diff line number Diff line change
Expand Up @@ -257,4 +257,3 @@ checkstyleMain.enabled = false
checkstyleTest.enabled = false
//TODO(https://github.com/apache/beam/issues/19119): javadoc task should be enabled in the future.
javadoc.enabled = false
test.outputs.upToDateWhen {false}
Original file line number Diff line number Diff line change
Expand Up @@ -67,7 +67,7 @@ public DataflowWorkProgressUpdater(
super(worker, Integer.MAX_VALUE);
this.workItemStatusClient = workItemStatusClient;
this.workItem = workItem;
this.hotKeyLogger = HotKeyLogger.ofSystemClock();
this.hotKeyLogger = HotKeyLogger.create();
this.options = options;
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -46,7 +46,7 @@ private HotKeyLogger(Clock clock) {
this.clock = clock;
}

public static HotKeyLogger ofSystemClock() {
public static HotKeyLogger create() {
return new HotKeyLogger(Clock.SYSTEM);
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -51,19 +51,25 @@ public static void main(String[] args) throws Exception {

// Use the MetricsLogger container which is used by BigQueryIO to periodically log process-wide
// metrics.
MetricsEnvironment.setProcessWideContainer(MetricsLogger.createUnboundedMetricsLogger());
MetricsEnvironment.setProcessWideContainer(MetricsLogger.forWorkerMetrics());
JvmInitializers.runBeforeProcessing(options);

LOG.info("Starting worker harness in mode={}.", worker.mode());
worker.start();
}

private static boolean isDirectPathPipeline(DataflowWorkerHarnessOptions options) {
boolean isEnableIpV6 = Optional.ofNullable(options.getDataflowServiceOptions())
.map(dataflowServiceOptions -> dataflowServiceOptions.contains(ENABLE_IPV6_EXPERIMENT))
.orElse(false);

if (options.isEnableWindmillServiceDirectPath() && !isEnableIpV6) {
LOG.warn("IPv6 is required for direct path, launching in dispatched mode.");
}

return options.isEnableStreamingEngine()
&& options.getIsWindmillServiceDirectPathEnabled()
&& Optional.ofNullable(options.getDataflowServiceOptions())
.map(dataflowServiceOptions -> dataflowServiceOptions.contains(ENABLE_IPV6_EXPERIMENT))
.orElse(false);
&& options.isEnableWindmillServiceDirectPath()
&& isEnableIpV6;
}

private static StreamingWorkerHarness createStreamingWorkerHarness(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,9 +20,6 @@
import org.checkerframework.checker.nullness.qual.Nullable;

/** Mapped diagnostic context for the Dataflow worker. */
@SuppressWarnings({
"nullness" // TODO(https://github.com/apache/beam/issues/20497)
})
public class DataflowWorkerLoggingMDC {
private static final InheritableThreadLocal<String> jobId = new InheritableThreadLocal<>();
private static final InheritableThreadLocal<@Nullable String> stageName =
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,12 +21,12 @@
import java.util.ArrayDeque;
import java.util.Collection;
import java.util.Deque;
import java.util.HashMap;
import java.util.Iterator;
import java.util.Map;
import java.util.Map.Entry;
import java.util.Optional;
import java.util.Queue;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.atomic.AtomicReference;
import java.util.function.BiConsumer;
import javax.annotation.Nullable;
Expand All @@ -38,6 +38,7 @@
import org.apache.beam.sdk.annotations.Internal;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableListMultimap;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Multimap;
import org.joda.time.Duration;
Expand Down Expand Up @@ -83,7 +84,7 @@ private ActiveWorkState(
}

static ActiveWorkState create(WindmillStateCache.ForComputation computationStateCache) {
return new ActiveWorkState(new ConcurrentHashMap<>(), computationStateCache);
return new ActiveWorkState(new HashMap<>(), computationStateCache);
}

@VisibleForTesting
Expand Down Expand Up @@ -291,8 +292,15 @@ private synchronized ImmutableMap<ShardedKey, WorkId> getStuckCommitsAt(
return stuckCommits.build();
}

synchronized ImmutableMap<ShardedKey, Deque<Work>> getReadOnlyActiveWork() {
return ImmutableMap.copyOf(activeWork);
synchronized ImmutableListMultimap<ShardedKey, Work> getActiveWork() {
ImmutableListMultimap.Builder<ShardedKey, Work> currentActiveWork =
ImmutableListMultimap.builder();

for (Entry<ShardedKey, Deque<Work>> keyedWorkQueue : activeWork.entrySet()) {
currentActiveWork.putAll(keyedWorkQueue.getKey(), keyedWorkQueue.getValue());
}

return currentActiveWork.build();
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,6 @@

import com.google.api.services.dataflow.model.MapTask;
import java.io.PrintWriter;
import java.util.Deque;
import java.util.Map;
import java.util.Optional;
import java.util.concurrent.ConcurrentLinkedQueue;
Expand All @@ -29,6 +28,7 @@
import org.apache.beam.runners.dataflow.worker.windmill.work.budget.GetWorkBudget;
import org.apache.beam.sdk.annotations.Internal;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableListMultimap;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Multimap;
import org.joda.time.Instant;
Expand Down Expand Up @@ -80,18 +80,16 @@ public ImmutableMap<String, String> getTransformUserNameToStateFamily() {
}

/**
* Adds the {@link ExecutionState} to the internal {@link #executionStateQueue} so that it can be
* re-used in future processing.
* Cache the {@link ExecutionState} so that it can be re-used in future {@link
* #acquireExecutionState()} calls.
*/
public void releaseExecutionState(ExecutionState executionState) {
executionStateQueue.offer(executionState);
}

/**
* Removes an {@link ExecutionState} instance from {@link #executionStateQueue} if one exists, and
* returns it. Calls to this method must be followed by a call to {@link
* #releaseExecutionState(ExecutionState)} to return the {@link ExecutionState} if it is to be
* reused.
* Returns {@link ExecutionState} that was previously offered in {@link
* #releaseExecutionState(ExecutionState)} or {@link Optional#empty()} if one does not exist.
*/
public Optional<ExecutionState> acquireExecutionState() {
return Optional.ofNullable(executionStateQueue.poll());
Expand Down Expand Up @@ -152,8 +150,8 @@ private void forceExecute(Work work) {
executor.forceExecute(work, work.getWorkItem().getSerializedSize());
}

public ImmutableMap<ShardedKey, Deque<Work>> currentActiveWorkReadOnly() {
return activeWorkState.getReadOnlyActiveWork();
public ImmutableListMultimap<ShardedKey, Work> getActiveWork() {
return activeWorkState.getActiveWork();
}

public void printActiveWork(PrintWriter writer) {
Expand All @@ -167,11 +165,7 @@ public GetWorkBudget getActiveWorkBudget() {
public final void close() {
@Nullable ExecutionState executionState;
while ((executionState = executionStateQueue.poll()) != null) {
try {
executionState.workExecutor().close();
} catch (Exception e) {
LOG.warn("Failed to close workExecutor {}.", executionState.workExecutor(), e);
}
executionState.close();
}
executionStateQueue.clear();
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -45,14 +45,14 @@ public final class ComputationStateCache implements StatusDataProvider {

private static final Logger LOG = LoggerFactory.getLogger(ComputationStateCache.class);

private final LoadingCache<String, Optional<ComputationState>> computationCache;
private final LoadingCache<String, ComputationState> computationCache;

private ComputationStateCache(LoadingCache<String, Optional<ComputationState>> computationCache) {
private ComputationStateCache(LoadingCache<String, ComputationState> computationCache) {
this.computationCache = computationCache;
}

public static ComputationStateCache create(
CacheLoader<String, Optional<ComputationState>> cacheLoader) {
CacheLoader<String, ComputationState> cacheLoader) {
return new ComputationStateCache(CacheBuilder.newBuilder().build(cacheLoader));
}

Expand Down
Original file line number Diff line number Diff line change
@@ -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.streaming;

import java.util.Optional;
import java.util.function.Function;
import org.apache.beam.runners.dataflow.worker.streaming.config.ComputationConfig;
import org.apache.beam.runners.dataflow.worker.util.BoundedQueueExecutor;
import org.apache.beam.runners.dataflow.worker.windmill.state.WindmillStateCache;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.cache.CacheLoader;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap;

final class ComputationStateCacheLoader extends CacheLoader<String, ComputationState> {
private final ComputationConfig.Factory computationConfigFactory;
private final BoundedQueueExecutor workUnitExecutor;
private final Function<String, WindmillStateCache.ForComputation>
perComputationStateCacheViewFactory;

ComputationStateCacheLoader(
ComputationConfig.Factory computationConfigFactory,
BoundedQueueExecutor workUnitExecutor,
Function<String, WindmillStateCache.ForComputation> perComputationStateCacheViewFactory) {
this.computationConfigFactory = computationConfigFactory;
this.workUnitExecutor = workUnitExecutor;
this.perComputationStateCacheViewFactory = perComputationStateCacheViewFactory;
}

@Override
public ComputationState load(String computationId) {
ComputationConfig computationConfig =
computationConfigFactory
.getComputationConfig(computationId)
.orElseThrow(() -> new ComputationStateNotFoundException(computationId));
return new ComputationState(
computationId,
computationConfig.mapTask(),
workUnitExecutor,
Optional.ofNullable(computationConfig.userTransformToStateFamilyName())
.orElseGet(ImmutableMap::of),
perComputationStateCacheViewFactory.apply(computationId));
}

static class ComputationStateNotFoundException extends IllegalStateException {
public ComputationStateNotFoundException(String computationId) {
super("No computation found for computationId=[ " + computationId + " ]");
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -56,8 +56,8 @@ public abstract static class Builder {
}

public final void close() {
context().invalidateCache();
try {
context().invalidateCache();
workExecutor().close();
} catch (Exception e) {
LOG.warn("Failed to close map task executor: ", e);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,8 @@
import java.util.Optional;
import java.util.concurrent.ConcurrentHashMap;
import java.util.function.Function;

import org.apache.beam.runners.dataflow.worker.streaming.config.ComputationConfig;
import org.apache.beam.runners.dataflow.worker.streaming.config.StreamingConfigLoader;
import org.apache.beam.runners.dataflow.worker.streaming.harness.StreamingWorkerEnvironment;
import org.apache.beam.runners.dataflow.worker.util.BoundedQueueExecutor;
Expand All @@ -47,7 +49,7 @@ public final class StreamingApplianceComputationStateCacheLoader
private static final Logger LOG =
LoggerFactory.getLogger(StreamingApplianceComputationStateCacheLoader.class);

private final StreamingConfigLoader<Windmill.GetConfigResponse> streamingApplianceConfigLoader;
private final ComputationConfig.Factory streamingApplianceConfigLoader;
private final BoundedQueueExecutor workUnitExecutor;
private final ConcurrentHashMap<String, String> systemNameToComputationIdMap;
private final Function<String, WindmillStateCache.ForComputation>
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,48 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.beam.runners.dataflow.worker.streaming.config;

import com.google.api.services.dataflow.model.MapTask;
import com.google.auto.value.AutoValue;
import java.util.Map;
import java.util.Optional;
import javax.annotation.Nullable;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap;

@AutoValue
public abstract class ComputationConfig {

public static ComputationConfig create(
MapTask mapTask, @Nullable Map<String, String> userTransformToStateFamilyName) {
return new AutoValue_ComputationConfig(
mapTask,
Optional.ofNullable(userTransformToStateFamilyName)
.map(ImmutableMap::copyOf)
.orElseGet(ImmutableMap::of));
}

public abstract MapTask mapTask();

public abstract ImmutableMap<String, String> userTransformToStateFamilyName();

@FunctionalInterface
public interface Factory {
Optional<ComputationConfig> getComputationConfig(String computationId);
}
}

0 comments on commit e60cb1e

Please sign in to comment.