scwhittle commented on code in PR #30046: URL: https://github.com/apache/beam/pull/30046#discussion_r1487783620
########## runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/FailedWorkToken.java: ########## @@ -0,0 +1,40 @@ +/* + * 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 Review Comment: merge with https://github.com/apache/beam/pull/29082 and remove this class ########## runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/WorkHeartbeatProcessor.java: ########## @@ -0,0 +1,65 @@ +/* + * 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.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.function.Consumer; +import java.util.function.Function; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill.ComputationHeartbeatResponse; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill.HeartbeatResponse; +import org.apache.beam.sdk.annotations.Internal; + +@Internal +public final class WorkHeartbeatProcessor implements Consumer<List<ComputationHeartbeatResponse>> { + /** Fetches a {@link ComputationState} for a computationId. */ + private final Function<String, Optional<ComputationState>> computationStateFetcher; + + public WorkHeartbeatProcessor( + /* Fetches a {@link ComputationState} for a String computationId. */ + Function<String, Optional<ComputationState>> computationStateFetcher) { + this.computationStateFetcher = computationStateFetcher; + } + + @Override + public void accept(List<ComputationHeartbeatResponse> responses) { + for (ComputationHeartbeatResponse computationHeartbeatResponse : responses) { + // Maps sharding key to (work token, cache token) for work that should be marked failed. + Map<Long, List<FailedWorkToken>> failedWork = new HashMap<>(); + for (HeartbeatResponse heartbeatResponse : + computationHeartbeatResponse.getHeartbeatResponsesList()) { + if (heartbeatResponse.getFailed()) { + failedWork + .computeIfAbsent(heartbeatResponse.getShardingKey(), key -> new ArrayList<>()) Review Comment: update based upon changes from other PR ########## runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcDispatcherClient.java: ########## @@ -44,93 +47,173 @@ class GrpcDispatcherClient { private static final Logger LOG = LoggerFactory.getLogger(GrpcDispatcherClient.class); private final WindmillStubFactory windmillStubFactory; - @GuardedBy("this") - private final List<CloudWindmillServiceV1Alpha1Stub> dispatcherStubs; - - @GuardedBy("this") - private final Set<HostAndPort> dispatcherEndpoints; + /** + * Current dispatcher endpoints and stubs used to communicate with Windmill Dispatcher. + * + * @implNote Reads are lock free, writes are synchronized. + */ + private final AtomicReference<DispatcherStubs> dispatcherStubs; @GuardedBy("this") private final Random rand; private GrpcDispatcherClient( WindmillStubFactory windmillStubFactory, - List<CloudWindmillServiceV1Alpha1Stub> dispatcherStubs, - Set<HostAndPort> dispatcherEndpoints, + DispatcherStubs initialDispatcherStubs, Random rand) { this.windmillStubFactory = windmillStubFactory; - this.dispatcherStubs = dispatcherStubs; - this.dispatcherEndpoints = dispatcherEndpoints; this.rand = rand; + this.dispatcherStubs = new AtomicReference<>(initialDispatcherStubs); } static GrpcDispatcherClient create(WindmillStubFactory windmillStubFactory) { - return new GrpcDispatcherClient( - windmillStubFactory, new ArrayList<>(), new HashSet<>(), new Random()); + return new GrpcDispatcherClient(windmillStubFactory, DispatcherStubs.empty(), new Random()); } @VisibleForTesting static GrpcDispatcherClient forTesting( WindmillStubFactory windmillGrpcStubFactory, - List<CloudWindmillServiceV1Alpha1Stub> dispatcherStubs, + List<CloudWindmillServiceV1Alpha1Stub> windmillServiceStubs, + List<CloudWindmillMetadataServiceV1Alpha1Stub> windmillMetadataServiceStubs, Set<HostAndPort> dispatcherEndpoints) { - Preconditions.checkArgument(dispatcherEndpoints.size() == dispatcherStubs.size()); + Preconditions.checkArgument( + dispatcherEndpoints.size() == windmillServiceStubs.size() + && windmillServiceStubs.size() == windmillMetadataServiceStubs.size()); return new GrpcDispatcherClient( - windmillGrpcStubFactory, dispatcherStubs, dispatcherEndpoints, new Random()); + windmillGrpcStubFactory, + DispatcherStubs.create( + dispatcherEndpoints, windmillServiceStubs, windmillMetadataServiceStubs), + new Random()); + } + + CloudWindmillServiceV1Alpha1Stub getWindmillServiceStub() { + ImmutableList<CloudWindmillServiceV1Alpha1Stub> windmillServiceStubs = + dispatcherStubs.get().windmillServiceStubs(); + Preconditions.checkState( + !windmillServiceStubs.isEmpty(), "windmillServiceEndpoint has not been set"); + + return (windmillServiceStubs.size() == 1 + ? windmillServiceStubs.get(0) + : randomlySelectNextStub(windmillServiceStubs)); } - synchronized CloudWindmillServiceV1Alpha1Stub getDispatcherStub() { + CloudWindmillMetadataServiceV1Alpha1Stub getWindmillMetadataServiceStub() { + ImmutableList<CloudWindmillMetadataServiceV1Alpha1Stub> windmillMetadataServiceStubs = + dispatcherStubs.get().windmillMetadataServiceStubs(); Preconditions.checkState( - !dispatcherStubs.isEmpty(), "windmillServiceEndpoint has not been set"); + !windmillMetadataServiceStubs.isEmpty(), "windmillServiceEndpoint has not been set"); + + return (windmillMetadataServiceStubs.size() == 1 + ? windmillMetadataServiceStubs.get(0) + : randomlySelectNextStub(windmillMetadataServiceStubs)); + } - return (dispatcherStubs.size() == 1 - ? dispatcherStubs.get(0) - : dispatcherStubs.get(rand.nextInt(dispatcherStubs.size()))); + private synchronized <T> T randomlySelectNextStub(List<T> stubs) { + return stubs.get(rand.nextInt(stubs.size())); } - synchronized boolean isReady() { - return !dispatcherStubs.isEmpty(); + boolean isReady() { Review Comment: We never really expect it to change. It was added in case we needed to migrate dispatcher endpoint since there are possibly stale long-running client processes. How about ignoring updates that are empty? Then we only go from not read -> ready and not backwards? That prevents the possible race, shouldn't cause any problems and is simpler than another type of notification. ########## runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/stubs/RemoteWindmillStubFactory.java: ########## @@ -0,0 +1,70 @@ +/* + * 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.WindmillChannelFactory.remoteChannel; + +import com.google.auth.Credentials; +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.auth.VendoredCredentialsAdapter; +import org.apache.beam.sdk.annotations.Internal; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.ManagedChannel; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.auth.MoreCallCredentials; + +/** Creates remote stubs to talk to Streaming Engine. */ +@Internal +public final class RemoteWindmillStubFactory implements WindmillStubFactory { + private final int rpcChannelTimeoutSec; + private final Credentials gcpCredentials; + private final boolean useIsolatedChannels; + + public RemoteWindmillStubFactory( + int rpcChannelTimeoutSec, Credentials gcpCredentials, boolean useIsolatedChannels) { + this.rpcChannelTimeoutSec = rpcChannelTimeoutSec; + this.gcpCredentials = gcpCredentials; + this.useIsolatedChannels = useIsolatedChannels; + } + + @Override + public CloudWindmillServiceV1Alpha1Stub createWindmillServiceStub( + WindmillServiceAddress serviceAddress) { + CloudWindmillServiceV1Alpha1Stub windmillServiceStub = + CloudWindmillServiceV1Alpha1Grpc.newStub(createChannel(serviceAddress)); + return serviceAddress.getKind() != WindmillServiceAddress.Kind.AUTHENTICATED_GCP_SERVICE_ADDRESS + ? windmillServiceStub.withCallCredentials( + MoreCallCredentials.from(new VendoredCredentialsAdapter(gcpCredentials))) + : windmillServiceStub; + } + + @Override + public CloudWindmillMetadataServiceV1Alpha1Stub createWindmillMetadataServiceStub( + WindmillServiceAddress serviceAddress) { + return CloudWindmillMetadataServiceV1Alpha1Grpc.newStub(createChannel(serviceAddress)) + .withCallCredentials( + MoreCallCredentials.from(new VendoredCredentialsAdapter(gcpCredentials))); + } + + private ManagedChannel createChannel(WindmillServiceAddress serviceAddress) { + ManagedChannel channel = remoteChannel(serviceAddress, rpcChannelTimeoutSec); + return useIsolatedChannels ? IsolationChannel.create(() -> channel) : channel; Review Comment: this is using the same ManagedChannel for all the separate channels the IsolationChannel is trying to create. Instead you can share the code this way. Maybe add a comment since it wasn't clear before Producer<ManagedChanne> channelFactory = () -> remoteChannel(service_address, rpcChannelTimeoutSec); return useIsolatedChannels ? IsolationChannel.create(channelFactory) : channelFactory(); -- 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]
