Skip to content
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

Update windmill proto definition #30046

Merged
merged 15 commits into from
Feb 15, 2024
Merged

Conversation

m-trieu
Copy link
Contributor

@m-trieu m-trieu commented Jan 19, 2024

Internal and external protos were not matching leading to gRPC errors.

  • make both protos match

  • add client id to the job headers for communication with Windmill Service

  • let the client id be owned by GrpcWindmillServer (WindmillServerStub) for now since its injected.

  • Will need to find a way to inject the client id (maybe from StreamingDataflowWorkerOptions) as well as other components. I wonder if that will be cleaner

r: @scwhittle

Thank you for your contribution! Follow this checklist to help us incorporate your contribution quickly and easily:

  • Mention the appropriate issue in your description (for example: addresses #123), if applicable. This will automatically add a link to the pull request in the issue. If you would like the issue to automatically close on merging the pull request, comment fixes #<ISSUE NUMBER> instead.
  • Update CHANGES.md with noteworthy changes.
  • If this contribution is large, please file an Apache Individual Contributor License Agreement.

See the Contributor Guide for more tips on how to make review process smoother.

To check the build health, please visit https://github.com/apache/beam/blob/master/.test-infra/BUILD_STATUS.md

GitHub Actions Tests Status (on master branch)

Build python source distribution and wheels
Python tests
Java tests
Go tests

See CI.md for more information about GitHub Actions CI or the workflows README to see a list of phrases to trigger workflows.

Copy link
Contributor

Checks are failing. Will not request review until checks are succeeding. If you'd like to override that behavior, comment assign set of reviewers

@m-trieu
Copy link
Contributor Author

m-trieu commented Jan 19, 2024

assign set of reviewers

@m-trieu
Copy link
Contributor Author

m-trieu commented Jan 19, 2024

run java precommit

Copy link
Contributor

Assigning reviewers. If you would like to opt out of this review, comment assign to next reviewer:

R: @lostluck added as fallback since no labels match configuration

Available commands:

  • stop reviewer notifications - opt out of the automated review tooling
  • remind me after tests pass - tag the comment author after tests pass
  • waiting on author - shift the attention set back to the author (any comment or push by the author will return the attention set to the reviewers)

The PR bot will only process comments in the main thread (not review comments).


return directEndpointIpV6Address.isPresent()
? directEndpointIpV6Address
: tryParseEndpointIntoHostAndPort(directEndpoint).map(WindmillServiceAddress::create);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit, the WindmillServiceAddress::create is common,
so maybe cleaner to do
return
tryParseDirectEndpointIntoIpV6Address(directEndpoint).orElseGet(tryParseEndpointIntoHostAndPort(directEndpoint)).map(WindmillServiceAddress::create);

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

i am wondering if we should jsut drop the ipv6
since HostAndPort supports ipv6

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

its possible that the tryParseEndpointIntoHostAndPort fails so it returns Optional and orElseGet requires a signature that returns HostAndPort

public static Endpoint.Builder builder() {
return new AutoValue_WindmillEndpoints_Endpoint.Builder();
}

public static Endpoint from(Windmill.WorkerMetadataResponse.Endpoint endpointProto) {
public static Endpoint from(
Windmill.WorkerMetadataResponse.Endpoint endpointProto, String authenticatingService) {
Endpoint.Builder endpointBuilder = Endpoint.builder();
if (endpointProto.hasDirectEndpoint() && !endpointProto.getDirectEndpoint().isEmpty()) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

remove the has check, can rely on default value being empty

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

done

@@ -76,6 +76,10 @@ public abstract GetWorkStream getWorkStream(
/** Returns the amount of time the server has been throttled and resets the time to 0. */
public abstract long getAndResetThrottleTime();

public long clientId() {
return 0L;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

client id is supposed to change on process restart. It is otherwise not a value that has to be coordinated in advance with the service so it can just be a random value as it was before. I think that it can stay in StreamingDataflowWorker because it also doesn't need to be different for each endpoint.

This helps the backend speed up retries, ensure caching is correct etc.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

ah ok out of curiosity wouldn't the other components restart also?
looks like we create a new worker in StreamingDataflowWorker.main

And inject the WindmillServerStub (GrpcWindmillServer) via the options.

@@ -45,92 +48,156 @@ class GrpcDispatcherClient {
private final WindmillStubFactory windmillStubFactory;

@GuardedBy("this")
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

seems like this wouldn't need guarding if it is atomic.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

updated

@lostluck
Copy link
Contributor

R: @scwhittle

(Just to get the bot off the hook. The "R" needs to be R and not r.)

Copy link
Contributor

Stopping reviewer notifications for this pull request: review requested by someone other than the bot, ceding control

@m-trieu
Copy link
Contributor Author

m-trieu commented Jan 19, 2024

R: @scwhittle

(Just to get the bot off the hook. The "R" needs to be R and not r.)

ack thanks!

@m-trieu m-trieu force-pushed the mt-metadata-api branch 2 times, most recently from 68409ad to 4de69da Compare January 24, 2024 00:20
}

private synchronized <T> T randomlySelectNextStub(List<T> stubs) {
return stubs.get(rand.nextInt(dispatcherStubs.get().size()));
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

mixing stubs and dispatcherStubs

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

changed to use stubs.size()

!options.isEnableStreamingEngine() && options.getLocalWindmillHostport() != null
? GrpcWindmillServer.LOCALHOST_BACKOFF
: GrpcWindmillServer.MAX_BACKOFF;
GrpcWindmillStreamFactory windmillStreamFactory =
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

seems odd to create this here if the server isn't grpc server.

Can we just pass in somethign generic like JobHeader to windmillServer.start?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

done

this.windmillServer = options.getWindmillServerStub();
this.windmillServer.setProcessHeartbeatResponses(this::handleHeartbeatResponses);
windmillServer.start(
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

could we avoid the start method (which is more complex than a factory method taking necessary requirements and initalizing) by moving the grpc server initialization into this file?

It's kind of odd that it is within getWindmillServerStub default factory. It appears that is done so that the test can inject a fake server. One idea would be to remove the default factory from options and create a server here if the option is null but use injected server otherwise.

Then you could just create the GrpcWindmillServer directly with either a factory or constructor taking the JobHeader without modifying the WindmillServerStub interface.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

changed
opted to allow StreamingDataflowWorker to inject a stream factory

}

synchronized boolean isReady() {
return !dispatcherStubs.isEmpty();
boolean isReady() {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

this seems like it could lead to racy usage
if (client.isReady()) {
client.xxx
}
but since the dispatcherStubs can change in-between, there may not be a stub by the time the next call is made.

Can this be removed?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We use this to make sure that the endpoints are set in StreamingDataflowWorker before we fetch the pipeline config as well as schedule periodic config fetches

if we remove this we would need another mechanism for that.

we could use some kind of publish/subscribe mechanism to signal when the endpoint is ready (kind of like a count down latch) and then send another signal during an update (i.e was ready with endpoints, but is no longer ready).

How often does this value change? I would think that the service endpoint itself would not change during pipeline execution. @scwhittle

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

ping

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

i guess we can do it async

how often does this value change? @scwhittle

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

this is the current behavior which won't allow empty updates

synchronized void consumeWindmillDispatcherEndpoints(
      ImmutableSet<HostAndPort> dispatcherEndpoints) {
    ImmutableSet<HostAndPort> currentDispatcherEndpoints =
        dispatcherStubs.get().dispatcherEndpoints();
    Preconditions.checkArgument(
        dispatcherEndpoints != null && !dispatcherEndpoints.isEmpty(),
        "Cannot set dispatcher endpoints to nothing.");
    if (this.dispatcherEndpoints.equals(dispatcherEndpoints)) {
    if (currentDispatcherEndpoints.equals(dispatcherEndpoints)) {
      // The endpoints are equal don't recreate the stubs.
      return;
    }....

We currently will throw an exception if we try to set the endpoints to nothing, should we just log a warning and ignore the update here?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

that seems fine, and we can just comment here that once ready it remains ready

could rename isReady to endpointsInitialized() or something that clarifies what it means more and is clearer it won't go back to false.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

ack done

@@ -101,36 +103,21 @@ public final class GrpcWindmillServer extends WindmillServerStub {
// newer ComputationHeartbeatRequests.
private final boolean sendKeyedGetDataRequests;
private Consumer<List<ComputationHeartbeatResponse>> processHeartbeatResponses;
private @Nullable GrpcWindmillStreamFactory windmillStreamFactory;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

if we can change constructor to require the streamfactory this nullable and preconditons below can be removed, see other comment

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

done

private final String testName;
private final Function<String, ManagedChannel> channelFactory;

InProcessWindmillStubFactory(String testName, Function<String, ManagedChannel> channelFactory) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: coudl just take a Supplier
If tests have a Function<String, ManagedChannel> they can always bind the test name themselves.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

done

import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.ManagedChannel;

/**
* Creates in process stubs to talk to Streaming Engine. Only recommended to be used for testing.
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

should this be moved to test folder and renamed FakeWindmillStubFactory? it ignores the serviceAddress

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

done

import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting;

@Internal
public final class WindmillStubFactories {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

should we get rid of this one? If the fake is moved to test folder this just has single method. I think callers could use RemoteWindmillStubFactory constructor directly

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

done

@m-trieu m-trieu force-pushed the mt-metadata-api branch 3 times, most recently from d04cdf9 to 23d6047 Compare February 8, 2024 21:04
@m-trieu
Copy link
Contributor Author

m-trieu commented Feb 9, 2024

theres some tests failing for windows that are unrelated (passing in other configurations)


public static StreamingDataflowWorker fromOptions(StreamingDataflowWorkerOptions options) {
StreamingDataflowWorkerOptions streamingOptions =
options.as(StreamingDataflowWorkerOptions.class);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

no longer needed, already right option type

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

done

ShardedKey shardedKey = entry.getKey();
Optional.ofNullable(failedWork.get(shardedKey.shardingKey()))
.ifPresent(
failedWorkTokens ->
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: I find this harder to read than just having a variable storing result of failedWork.get and using an if statement, as it's harder to eyeball and takes a little more effort to realize that this is processing on the result of failedWork.get

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

done!

private final Function<String, Optional<ComputationState>> computationStateFetcher;

public WorkHeartbeatProcessor(
Function<String, Optional<ComputationState>> computationStateFetcher) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

comment on what string is

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

done

}

synchronized boolean isReady() {
return !dispatcherStubs.isEmpty();
boolean isReady() {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

ping

private static final Logger LOG = LoggerFactory.getLogger(GrpcWindmillServerTest.class);
private static final int STREAM_CHUNK_SIZE = 2 << 20;
private final long clientId = new Random().nextLong();
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: just hard-code something?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

done.

/*
* 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
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

merge with #29082 and remove this class

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

done

}

synchronized boolean isReady() {
return !dispatcherStubs.isEmpty();
boolean isReady() {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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.


private ManagedChannel createChannel(WindmillServiceAddress serviceAddress) {
ManagedChannel channel = remoteChannel(serviceAddress, rpcChannelTimeoutSec);
return useIsolatedChannels ? IsolationChannel.create(() -> channel) : channel;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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 channelFactory = () -> remoteChannel(service_address, rpcChannelTimeoutSec);
return useIsolatedChannels ? IsolationChannel.create(channelFactory) : channelFactory();

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

done

computationHeartbeatResponse.getHeartbeatResponsesList()) {
if (heartbeatResponse.getFailed()) {
failedWork
.computeIfAbsent(heartbeatResponse.getShardingKey(), key -> new ArrayList<>())
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

update based upon changes from other PR

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

done

@m-trieu
Copy link
Contributor Author

m-trieu commented Feb 15, 2024

failures are not related to changes
attempt to fix flaky streaming tests in #30322

ready for another look! @scwhittle looks like we already do not allow empty updates to the dispatcher endpoints

Copy link
Contributor

@scwhittle scwhittle left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

just a couple minor comments, otherwise looks good

import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Multimap;

@Internal
public final class WorkHeartbeatProcessor implements Consumer<List<ComputationHeartbeatResponse>> {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

WorkHeartbeatResponseProcessor?
might help prevent confusion that it is processing for outgoing heartbeats

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

done and added comment

}

synchronized boolean isReady() {
return !dispatcherStubs.isEmpty();
boolean isReady() {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

that seems fine, and we can just comment here that once ready it remains ready

could rename isReady to endpointsInitialized() or something that clarifies what it means more and is clearer it won't go back to false.

@scwhittle scwhittle merged commit 690a5a4 into apache:master Feb 15, 2024
17 checks passed
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
Development

Successfully merging this pull request may close these issues.

3 participants