Skip to content

use WindmillChannelFactory to control what types of channels to generate #34653

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Open
wants to merge 1 commit into
base: master
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view

Large diffs are not rendered by default.

Original file line number Diff line number Diff line change
@@ -0,0 +1,54 @@
/*
* 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.ExecutorService;
import java.util.concurrent.Executors;
import org.apache.beam.runners.dataflow.worker.util.MemoryMonitor;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.ThreadFactoryBuilder;

/**
* Monitors memory pressure on a background executor. May be used to throttle calls, blocking if
* there is memory pressure.
*/
@AutoValue
public abstract class BackgroundMemoryMonitor {
public static BackgroundMemoryMonitor create(MemoryMonitor memoryMonitor) {
return new AutoValue_BackgroundMemoryMonitor(
memoryMonitor,
Executors.newSingleThreadScheduledExecutor(
new ThreadFactoryBuilder()
.setNameFormat("MemoryMonitor")
.setPriority(Thread.MIN_PRIORITY)
.build()));
}

abstract MemoryMonitor memoryMonitor();

abstract ExecutorService executor();

public void start() {
executor().execute(memoryMonitor());
}

public void shutdown() {
memoryMonitor().stop();
executor().shutdown();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -26,19 +26,15 @@
import java.util.Set;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicReference;
import javax.annotation.concurrent.GuardedBy;
import javax.annotation.concurrent.ThreadSafe;
import org.apache.beam.runners.dataflow.options.DataflowWorkerHarnessOptions;
import org.apache.beam.runners.dataflow.worker.streaming.config.StreamingGlobalConfig;
import org.apache.beam.runners.dataflow.worker.windmill.CloudWindmillMetadataServiceV1Alpha1Grpc;
import org.apache.beam.runners.dataflow.worker.windmill.CloudWindmillMetadataServiceV1Alpha1Grpc.CloudWindmillMetadataServiceV1Alpha1Stub;
import org.apache.beam.runners.dataflow.worker.windmill.CloudWindmillServiceV1Alpha1Grpc;
import org.apache.beam.runners.dataflow.worker.windmill.CloudWindmillServiceV1Alpha1Grpc.CloudWindmillServiceV1Alpha1Stub;
import org.apache.beam.runners.dataflow.worker.windmill.WindmillServiceAddress;
import org.apache.beam.runners.dataflow.worker.windmill.client.grpc.stubs.WindmillStubFactory;
import org.apache.beam.runners.dataflow.worker.windmill.client.grpc.stubs.WindmillStubFactoryFactory;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList;
Expand All @@ -64,52 +60,33 @@ public class GrpcDispatcherClient {
@GuardedBy("this")
private final Random rand;

private final WindmillStubFactoryFactory windmillStubFactoryFactory;

private final AtomicReference<WindmillStubFactory> windmillStubFactory = new AtomicReference<>();

private final AtomicBoolean useIsolatedChannels = new AtomicBoolean();
private final boolean reactToIsolatedChannelsJobSetting;
private final WindmillStubFactory windmillStubFactory;

private GrpcDispatcherClient(
DataflowWorkerHarnessOptions options,
WindmillStubFactoryFactory windmillStubFactoryFactory,
WindmillStubFactory windmillStubFactory,
DispatcherStubs initialDispatcherStubs,
Random rand) {
this.windmillStubFactoryFactory = windmillStubFactoryFactory;
if (options.getUseWindmillIsolatedChannels() != null) {
this.useIsolatedChannels.set(options.getUseWindmillIsolatedChannels());
this.reactToIsolatedChannelsJobSetting = false;
} else {
this.useIsolatedChannels.set(false);
this.reactToIsolatedChannelsJobSetting = true;
}
this.windmillStubFactory.set(
windmillStubFactoryFactory.makeWindmillStubFactory(useIsolatedChannels.get()));
this.windmillStubFactory = windmillStubFactory;
this.rand = rand;
this.dispatcherStubs = new AtomicReference<>(initialDispatcherStubs);
this.onInitializedEndpoints = new CountDownLatch(1);
}

public static GrpcDispatcherClient create(
DataflowWorkerHarnessOptions options, WindmillStubFactoryFactory windmillStubFactoryFactory) {
return new GrpcDispatcherClient(
options, windmillStubFactoryFactory, DispatcherStubs.empty(), new Random());
public static GrpcDispatcherClient create(WindmillStubFactory windmillStubFactory) {
return new GrpcDispatcherClient(windmillStubFactory, DispatcherStubs.empty(), new Random());
}

@VisibleForTesting
public static GrpcDispatcherClient forTesting(
DataflowWorkerHarnessOptions options,
WindmillStubFactoryFactory windmillStubFactoryFactory,
WindmillStubFactory windmillStubFactory,
List<CloudWindmillServiceV1Alpha1Stub> windmillServiceStubs,
List<CloudWindmillMetadataServiceV1Alpha1Stub> windmillMetadataServiceStubs,
Set<HostAndPort> dispatcherEndpoints) {
Preconditions.checkArgument(
dispatcherEndpoints.size() == windmillServiceStubs.size()
&& windmillServiceStubs.size() == windmillMetadataServiceStubs.size());
return new GrpcDispatcherClient(
options,
windmillStubFactoryFactory,
windmillStubFactory,
DispatcherStubs.create(
dispatcherEndpoints, windmillServiceStubs, windmillMetadataServiceStubs),
new Random());
Expand Down Expand Up @@ -164,28 +141,16 @@ private synchronized <T> T randomlySelectNextStub(List<T> stubs) {
return stubs.get(rand.nextInt(stubs.size()));
}

public void onJobConfig(StreamingGlobalConfig config) {
if (config.windmillServiceEndpoints().isEmpty()) {
LOG.warn("Dispatcher client received empty windmill service endpoints from global config");
return;
}
boolean forceRecreateStubs = false;
if (reactToIsolatedChannelsJobSetting) {
boolean useIsolatedChannels = config.userWorkerJobSettings().getUseWindmillIsolatedChannels();
if (this.useIsolatedChannels.getAndSet(useIsolatedChannels) != useIsolatedChannels) {
windmillStubFactory.set(
windmillStubFactoryFactory.makeWindmillStubFactory(useIsolatedChannels));
forceRecreateStubs = true;
}
}
consumeWindmillDispatcherEndpoints(config.windmillServiceEndpoints(), forceRecreateStubs);
}

public synchronized void consumeWindmillDispatcherEndpoints(
ImmutableSet<HostAndPort> dispatcherEndpoints) {
consumeWindmillDispatcherEndpoints(dispatcherEndpoints, /* forceRecreateStubs= */ false);
}

public synchronized void reloadDispatcherEndpoints(
ImmutableSet<HostAndPort> dispatcherEndpoints) {
consumeWindmillDispatcherEndpoints(dispatcherEndpoints, /* forceRecreateStubs= */ true);
}

private synchronized void consumeWindmillDispatcherEndpoints(
ImmutableSet<HostAndPort> dispatcherEndpoints, boolean forceRecreateStubs) {
ImmutableSet<HostAndPort> currentDispatcherEndpoints =
Expand All @@ -204,7 +169,7 @@ private synchronized void consumeWindmillDispatcherEndpoints(
}

LOG.info("Initializing Streaming Engine GRPC client for endpoints: {}", dispatcherEndpoints);
dispatcherStubs.set(DispatcherStubs.create(dispatcherEndpoints, windmillStubFactory.get()));
dispatcherStubs.set(DispatcherStubs.create(dispatcherEndpoints, windmillStubFactory));
onInitializedEndpoints.countDown();
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -50,10 +50,11 @@
import org.apache.beam.runners.dataflow.worker.windmill.Windmill.ReportStatsResponse;
import org.apache.beam.runners.dataflow.worker.windmill.WindmillApplianceGrpc;
import org.apache.beam.runners.dataflow.worker.windmill.WindmillServerStub;
import org.apache.beam.runners.dataflow.worker.windmill.WindmillServiceAddress;
import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStream.CommitWorkStream;
import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStream.GetDataStream;
import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStream.GetWorkStream;
import org.apache.beam.runners.dataflow.worker.windmill.client.grpc.stubs.WindmillStubFactoryFactory;
import org.apache.beam.runners.dataflow.worker.windmill.client.grpc.stubs.WindmillStubFactory;
import org.apache.beam.runners.dataflow.worker.windmill.client.throttling.StreamingEngineThrottleTimers;
import org.apache.beam.runners.dataflow.worker.windmill.work.WorkItemReceiver;
import org.apache.beam.sdk.extensions.gcp.options.GcpOptions;
Expand Down Expand Up @@ -154,7 +155,7 @@ static GrpcWindmillServer newTestInstance(
String name,
List<String> experiments,
long clientId,
WindmillStubFactoryFactory windmillStubFactoryFactory) {
WindmillStubFactory windmillStubFactory) {
ManagedChannel inProcessChannel = inProcessChannel(name);
CloudWindmillServiceV1Alpha1Stub stub =
CloudWindmillServiceV1Alpha1Grpc.newStub(inProcessChannel);
Expand All @@ -170,8 +171,7 @@ static GrpcWindmillServer newTestInstance(
Set<HostAndPort> dispatcherEndpoints = Sets.newHashSet(HostAndPort.fromHost(name));
GrpcDispatcherClient dispatcherClient =
GrpcDispatcherClient.forTesting(
testOptions,
windmillStubFactoryFactory,
windmillStubFactory,
windmillServiceStubs,
windmillMetadataServiceStubs,
dispatcherEndpoints);
Expand All @@ -193,16 +193,28 @@ static GrpcWindmillServer newTestInstance(
}

@VisibleForTesting
static GrpcWindmillServer newApplianceTestInstance(
Channel channel, WindmillStubFactoryFactory windmillStubFactoryFactory) {
static GrpcWindmillServer newApplianceTestInstance(Channel channel) {
DataflowWorkerHarnessOptions options =
testOptions(/* enableStreamingEngine= */ false, new ArrayList<>());
GrpcWindmillServer testServer =
new GrpcWindmillServer(
options,
GrpcWindmillStreamFactory.of(createJobHeader(options, 1)).build(),
// No-op, Appliance does not use Dispatcher to call Streaming Engine.
GrpcDispatcherClient.create(options, windmillStubFactoryFactory));
GrpcDispatcherClient.create(
new WindmillStubFactory() {
@Override
public CloudWindmillServiceV1Alpha1Stub createWindmillServiceStub(
WindmillServiceAddress serviceAddress) {
throw new UnsupportedOperationException();
}

@Override
public CloudWindmillMetadataServiceV1Alpha1Stub createWindmillMetadataServiceStub(
WindmillServiceAddress serviceAddress) {
throw new UnsupportedOperationException();
}
}));
testServer.syncApplianceStub = createWindmillApplianceStubWithDeadlineInterceptor(channel);
return testServer;
}
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,71 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.beam.runners.dataflow.worker.windmill.client.grpc.stubs;

import static org.apache.beam.runners.dataflow.worker.windmill.client.grpc.stubs.WindmillChannels.remoteChannel;

import javax.annotation.concurrent.ThreadSafe;
import org.apache.beam.runners.dataflow.worker.streaming.config.StreamingGlobalConfig;
import org.apache.beam.runners.dataflow.worker.windmill.Windmill;
import org.apache.beam.runners.dataflow.worker.windmill.WindmillServiceAddress;
import org.apache.beam.vendor.grpc.v1p69p0.io.grpc.Internal;
import org.apache.beam.vendor.grpc.v1p69p0.io.grpc.ManagedChannel;
import org.checkerframework.checker.nullness.qual.MonotonicNonNull;

/** Creates gRPC channels based on the current {@link StreamingGlobalConfig}. */
@Internal
@ThreadSafe
public final class ConfigAwareChannelFactory implements WindmillChannelFactory {
private final int windmillServiceRpcChannelAliveTimeoutSec;
@MonotonicNonNull private StreamingGlobalConfig currentConfig = null;

public ConfigAwareChannelFactory(int windmillServiceRpcChannelAliveTimeoutSec) {
this.windmillServiceRpcChannelAliveTimeoutSec = windmillServiceRpcChannelAliveTimeoutSec;
}

@Override
public synchronized ManagedChannel create(
Windmill.UserWorkerGrpcFlowControlSettings flowControlSettings,
WindmillServiceAddress serviceAddress) {
return currentConfig != null
&& currentConfig.userWorkerJobSettings().getUseWindmillIsolatedChannels()
// IsolationChannel will create and manage separate RPC channels to the same
// serviceAddress via calling the channelFactory, else just directly return
// the RPC channel.
? IsolationChannel.create(
() ->
remoteChannel(
serviceAddress.getServiceAddress(),
windmillServiceRpcChannelAliveTimeoutSec,
flowControlSettings))
: remoteChannel(
serviceAddress.getServiceAddress(),
windmillServiceRpcChannelAliveTimeoutSec,
flowControlSettings);
}

public synchronized boolean tryConsumeJobConfig(StreamingGlobalConfig config) {
if (currentConfig == null
|| config.userWorkerJobSettings().getUseWindmillIsolatedChannels()
!= currentConfig.userWorkerJobSettings().getUseWindmillIsolatedChannels()) {
currentConfig = config;
return true;
}
return false;
}
}

This file was deleted.

This file was deleted.

Loading
Loading