-
Notifications
You must be signed in to change notification settings - Fork 4.3k
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[Dataflow Streaming] Use separate heartbeat streams based on job sett…
…ings (#32511)
- Loading branch information
1 parent
aabae27
commit c7fb9a0
Showing
5 changed files
with
253 additions
and
13 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
52 changes: 52 additions & 0 deletions
52
...java/org/apache/beam/runners/dataflow/worker/streaming/config/FakeGlobalConfigHandle.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,52 @@ | ||
/* | ||
* 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 java.util.function.Consumer; | ||
import javax.annotation.Nonnull; | ||
import javax.annotation.concurrent.ThreadSafe; | ||
import org.apache.beam.sdk.annotations.Internal; | ||
|
||
@Internal | ||
@ThreadSafe | ||
/* | ||
* Fake StreamingGlobalConfigHandle used for Tests. Allows setting fake configs. | ||
*/ | ||
public class FakeGlobalConfigHandle implements StreamingGlobalConfigHandle { | ||
|
||
private final StreamingGlobalConfigHandleImpl globalConfigHandle; | ||
|
||
public FakeGlobalConfigHandle(StreamingGlobalConfig config) { | ||
this.globalConfigHandle = new StreamingGlobalConfigHandleImpl(); | ||
this.globalConfigHandle.setConfig(config); | ||
} | ||
|
||
@Override | ||
public StreamingGlobalConfig getConfig() { | ||
return globalConfigHandle.getConfig(); | ||
} | ||
|
||
public void setConfig(StreamingGlobalConfig config) { | ||
globalConfigHandle.setConfig(config); | ||
} | ||
|
||
@Override | ||
public void registerConfigObserver(@Nonnull Consumer<StreamingGlobalConfig> callback) { | ||
globalConfigHandle.registerConfigObserver(callback); | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
132 changes: 132 additions & 0 deletions
132
...che/beam/runners/dataflow/worker/windmill/work/refresh/StreamPoolHeartbeatSenderTest.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,132 @@ | ||
/* | ||
* 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.refresh; | ||
|
||
import static org.junit.Assert.assertEquals; | ||
|
||
import java.util.Optional; | ||
import org.apache.beam.runners.dataflow.worker.FakeWindmillServer; | ||
import org.apache.beam.runners.dataflow.worker.streaming.config.FakeGlobalConfigHandle; | ||
import org.apache.beam.runners.dataflow.worker.streaming.config.StreamingGlobalConfig; | ||
import org.apache.beam.runners.dataflow.worker.windmill.Windmill.HeartbeatRequest; | ||
import org.apache.beam.runners.dataflow.worker.windmill.Windmill.UserWorkerRunnerV1Settings; | ||
import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStreamPool; | ||
import org.joda.time.Duration; | ||
import org.junit.Test; | ||
import org.junit.rules.ErrorCollector; | ||
import org.junit.runner.RunWith; | ||
import org.junit.runners.JUnit4; | ||
|
||
@RunWith(JUnit4.class) | ||
public class StreamPoolHeartbeatSenderTest { | ||
|
||
@Test | ||
public void sendsHeartbeatsOnStream() { | ||
FakeWindmillServer server = new FakeWindmillServer(new ErrorCollector(), c -> Optional.empty()); | ||
StreamPoolHeartbeatSender heartbeatSender = | ||
StreamPoolHeartbeatSender.Create( | ||
WindmillStreamPool.create(1, Duration.standardSeconds(10), server::getDataStream)); | ||
Heartbeats.Builder heartbeatsBuilder = Heartbeats.builder(); | ||
heartbeatsBuilder | ||
.heartbeatRequestsBuilder() | ||
.put("key", HeartbeatRequest.newBuilder().setWorkToken(123).build()); | ||
heartbeatSender.sendHeartbeats(heartbeatsBuilder.build()); | ||
assertEquals(1, server.getGetDataRequests().size()); | ||
} | ||
|
||
@Test | ||
public void sendsHeartbeatsOnDedicatedStream() { | ||
FakeWindmillServer dedicatedServer = | ||
new FakeWindmillServer(new ErrorCollector(), c -> Optional.empty()); | ||
FakeWindmillServer getDataServer = | ||
new FakeWindmillServer(new ErrorCollector(), c -> Optional.empty()); | ||
|
||
FakeGlobalConfigHandle configHandle = | ||
new FakeGlobalConfigHandle(getGlobalConfig(/*useSeparateHeartbeatStreams=*/ true)); | ||
StreamPoolHeartbeatSender heartbeatSender = | ||
StreamPoolHeartbeatSender.Create( | ||
WindmillStreamPool.create( | ||
1, Duration.standardSeconds(10), dedicatedServer::getDataStream), | ||
WindmillStreamPool.create( | ||
1, Duration.standardSeconds(10), getDataServer::getDataStream), | ||
configHandle); | ||
Heartbeats.Builder heartbeatsBuilder = Heartbeats.builder(); | ||
heartbeatsBuilder | ||
.heartbeatRequestsBuilder() | ||
.put("key", HeartbeatRequest.newBuilder().setWorkToken(123).build()); | ||
heartbeatSender.sendHeartbeats(heartbeatsBuilder.build()); | ||
assertEquals(1, dedicatedServer.getGetDataRequests().size()); | ||
assertEquals(0, getDataServer.getGetDataRequests().size()); | ||
|
||
heartbeatSender.sendHeartbeats(heartbeatsBuilder.build()); | ||
assertEquals(2, dedicatedServer.getGetDataRequests().size()); | ||
assertEquals(0, getDataServer.getGetDataRequests().size()); | ||
|
||
// Turn off separate heartbeats | ||
configHandle.setConfig(getGlobalConfig(/*useSeparateHeartbeatStreams=*/ false)); | ||
heartbeatSender.sendHeartbeats(heartbeatsBuilder.build()); | ||
// request to getDataServer increases and dedicatedServer remains same | ||
assertEquals(2, dedicatedServer.getGetDataRequests().size()); | ||
assertEquals(1, getDataServer.getGetDataRequests().size()); | ||
} | ||
|
||
private static StreamingGlobalConfig getGlobalConfig(boolean useSeparateHeartbeatStreams) { | ||
return StreamingGlobalConfig.builder() | ||
.setUserWorkerJobSettings( | ||
UserWorkerRunnerV1Settings.newBuilder() | ||
.setUseSeparateWindmillHeartbeatStreams(useSeparateHeartbeatStreams) | ||
.build()) | ||
.build(); | ||
} | ||
|
||
@Test | ||
public void sendsHeartbeatsOnGetDataStream() { | ||
FakeWindmillServer dedicatedServer = | ||
new FakeWindmillServer(new ErrorCollector(), c -> Optional.empty()); | ||
FakeWindmillServer getDataServer = | ||
new FakeWindmillServer(new ErrorCollector(), c -> Optional.empty()); | ||
|
||
FakeGlobalConfigHandle configHandle = | ||
new FakeGlobalConfigHandle(getGlobalConfig(/*useSeparateHeartbeatStreams=*/ false)); | ||
StreamPoolHeartbeatSender heartbeatSender = | ||
StreamPoolHeartbeatSender.Create( | ||
WindmillStreamPool.create( | ||
1, Duration.standardSeconds(10), dedicatedServer::getDataStream), | ||
WindmillStreamPool.create( | ||
1, Duration.standardSeconds(10), getDataServer::getDataStream), | ||
configHandle); | ||
Heartbeats.Builder heartbeatsBuilder = Heartbeats.builder(); | ||
heartbeatsBuilder | ||
.heartbeatRequestsBuilder() | ||
.put("key", HeartbeatRequest.newBuilder().setWorkToken(123).build()); | ||
heartbeatSender.sendHeartbeats(heartbeatsBuilder.build()); | ||
assertEquals(0, dedicatedServer.getGetDataRequests().size()); | ||
assertEquals(1, getDataServer.getGetDataRequests().size()); | ||
|
||
heartbeatSender.sendHeartbeats(heartbeatsBuilder.build()); | ||
assertEquals(0, dedicatedServer.getGetDataRequests().size()); | ||
assertEquals(2, getDataServer.getGetDataRequests().size()); | ||
|
||
// Turn on separate heartbeats | ||
configHandle.setConfig(getGlobalConfig(/*useSeparateHeartbeatStreams=*/ true)); | ||
heartbeatSender.sendHeartbeats(heartbeatsBuilder.build()); | ||
// request to dedicatedServer increases and getDataServer remains same | ||
assertEquals(1, dedicatedServer.getGetDataRequests().size()); | ||
assertEquals(2, getDataServer.getGetDataRequests().size()); | ||
} | ||
} |