-
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] ActiveWorkRefresh (#30390)
* factor out async work management * add tests * remove sleep from test * add sleep in tests
- Loading branch information
Showing
5 changed files
with
501 additions
and
51 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
107 changes: 107 additions & 0 deletions
107
...va/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/ActiveWorkRefresher.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,107 @@ | ||
/* | ||
* 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 java.util.Collection; | ||
import java.util.concurrent.ScheduledExecutorService; | ||
import java.util.concurrent.TimeUnit; | ||
import java.util.function.Supplier; | ||
import javax.annotation.concurrent.ThreadSafe; | ||
import org.apache.beam.runners.dataflow.worker.DataflowExecutionStateSampler; | ||
import org.apache.beam.runners.dataflow.worker.streaming.ComputationState; | ||
import org.joda.time.Duration; | ||
import org.joda.time.Instant; | ||
import org.slf4j.Logger; | ||
import org.slf4j.LoggerFactory; | ||
|
||
/** | ||
* Asynchronously GetData requests to Streaming Engine for all sufficiently old active work and | ||
* invalidates stuck commits. | ||
* | ||
* <p>This informs Windmill that processing is ongoing and the work should not be retried. The age | ||
* threshold is determined by {@link #activeWorkRefreshPeriodMillis} | ||
*/ | ||
@ThreadSafe | ||
public abstract class ActiveWorkRefresher { | ||
private static final Logger LOG = LoggerFactory.getLogger(ActiveWorkRefresher.class); | ||
|
||
protected final Supplier<Instant> clock; | ||
protected final int activeWorkRefreshPeriodMillis; | ||
protected final Supplier<Collection<ComputationState>> computations; | ||
protected final DataflowExecutionStateSampler sampler; | ||
private final int stuckCommitDurationMillis; | ||
private final ScheduledExecutorService activeWorkRefreshExecutor; | ||
|
||
protected ActiveWorkRefresher( | ||
Supplier<Instant> clock, | ||
int activeWorkRefreshPeriodMillis, | ||
int stuckCommitDurationMillis, | ||
Supplier<Collection<ComputationState>> computations, | ||
DataflowExecutionStateSampler sampler, | ||
ScheduledExecutorService activeWorkRefreshExecutor) { | ||
this.clock = clock; | ||
this.activeWorkRefreshPeriodMillis = activeWorkRefreshPeriodMillis; | ||
this.stuckCommitDurationMillis = stuckCommitDurationMillis; | ||
this.computations = computations; | ||
this.sampler = sampler; | ||
this.activeWorkRefreshExecutor = activeWorkRefreshExecutor; | ||
} | ||
|
||
@SuppressWarnings("FutureReturnValueIgnored") | ||
public void start() { | ||
if (activeWorkRefreshPeriodMillis > 0) { | ||
activeWorkRefreshExecutor.scheduleWithFixedDelay( | ||
() -> { | ||
try { | ||
refreshActiveWork(); | ||
} catch (RuntimeException e) { | ||
LOG.warn("Failed to refresh active work: ", e); | ||
} | ||
}, | ||
activeWorkRefreshPeriodMillis, | ||
activeWorkRefreshPeriodMillis, | ||
TimeUnit.MILLISECONDS); | ||
} | ||
|
||
if (stuckCommitDurationMillis > 0) { | ||
int periodMillis = Math.max(stuckCommitDurationMillis / 10, 100); | ||
activeWorkRefreshExecutor.scheduleWithFixedDelay( | ||
this::invalidateStuckCommits, periodMillis, periodMillis, TimeUnit.MILLISECONDS); | ||
} | ||
} | ||
|
||
public void stop() { | ||
if (activeWorkRefreshPeriodMillis > 0 || stuckCommitDurationMillis > 0) { | ||
activeWorkRefreshExecutor.shutdown(); | ||
try { | ||
activeWorkRefreshExecutor.awaitTermination(300, TimeUnit.SECONDS); | ||
} catch (InterruptedException e) { | ||
activeWorkRefreshExecutor.shutdownNow(); | ||
} | ||
} | ||
} | ||
|
||
private void invalidateStuckCommits() { | ||
Instant stuckCommitDeadline = clock.get().minus(Duration.millis(stuckCommitDurationMillis)); | ||
for (ComputationState computationState : computations.get()) { | ||
computationState.invalidateStuckCommits(stuckCommitDeadline); | ||
} | ||
} | ||
|
||
protected abstract void refreshActiveWork(); | ||
} |
50 changes: 50 additions & 0 deletions
50
...a/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/ActiveWorkRefreshers.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,50 @@ | ||
/* | ||
* 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 java.util.Collection; | ||
import java.util.List; | ||
import java.util.Map; | ||
import java.util.concurrent.ScheduledExecutorService; | ||
import java.util.function.Consumer; | ||
import java.util.function.Supplier; | ||
import org.apache.beam.runners.dataflow.worker.DataflowExecutionStateSampler; | ||
import org.apache.beam.runners.dataflow.worker.streaming.ComputationState; | ||
import org.apache.beam.runners.dataflow.worker.windmill.Windmill.HeartbeatRequest; | ||
import org.joda.time.Instant; | ||
|
||
/** Utility class for {@link ActiveWorkRefresher}. */ | ||
public final class ActiveWorkRefreshers { | ||
public static ActiveWorkRefresher createDispatchedActiveWorkRefresher( | ||
Supplier<Instant> clock, | ||
int activeWorkRefreshPeriodMillis, | ||
int stuckCommitDurationMillis, | ||
Supplier<Collection<ComputationState>> computations, | ||
DataflowExecutionStateSampler sampler, | ||
Consumer<Map<String, List<HeartbeatRequest>>> activeWorkRefresherFn, | ||
ScheduledExecutorService scheduledExecutorService) { | ||
return new DispatchedActiveWorkRefresher( | ||
clock, | ||
activeWorkRefreshPeriodMillis, | ||
stuckCommitDurationMillis, | ||
computations, | ||
sampler, | ||
activeWorkRefresherFn, | ||
scheduledExecutorService); | ||
} | ||
} |
68 changes: 68 additions & 0 deletions
68
...che/beam/runners/dataflow/worker/windmill/work/refresh/DispatchedActiveWorkRefresher.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,68 @@ | ||
/* | ||
* 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 java.util.Collection; | ||
import java.util.HashMap; | ||
import java.util.List; | ||
import java.util.Map; | ||
import java.util.concurrent.ScheduledExecutorService; | ||
import java.util.function.Consumer; | ||
import java.util.function.Supplier; | ||
import org.apache.beam.runners.dataflow.worker.DataflowExecutionStateSampler; | ||
import org.apache.beam.runners.dataflow.worker.streaming.ComputationState; | ||
import org.apache.beam.runners.dataflow.worker.windmill.Windmill; | ||
import org.joda.time.Duration; | ||
import org.joda.time.Instant; | ||
|
||
final class DispatchedActiveWorkRefresher extends ActiveWorkRefresher { | ||
|
||
private final Consumer<Map<String, List<Windmill.HeartbeatRequest>>> activeWorkRefresherFn; | ||
|
||
DispatchedActiveWorkRefresher( | ||
Supplier<Instant> clock, | ||
int activeWorkRefreshPeriodMillis, | ||
int stuckCommitDurationMillis, | ||
Supplier<Collection<ComputationState>> computations, | ||
DataflowExecutionStateSampler sampler, | ||
Consumer<Map<String, List<Windmill.HeartbeatRequest>>> activeWorkRefresherFn, | ||
ScheduledExecutorService scheduledExecutorService) { | ||
super( | ||
clock, | ||
activeWorkRefreshPeriodMillis, | ||
stuckCommitDurationMillis, | ||
computations, | ||
sampler, | ||
scheduledExecutorService); | ||
this.activeWorkRefresherFn = activeWorkRefresherFn; | ||
} | ||
|
||
@Override | ||
protected void refreshActiveWork() { | ||
Map<String, List<Windmill.HeartbeatRequest>> heartbeats = new HashMap<>(); | ||
Instant refreshDeadline = clock.get().minus(Duration.millis(activeWorkRefreshPeriodMillis)); | ||
|
||
for (ComputationState computationState : computations.get()) { | ||
heartbeats.put( | ||
computationState.getComputationId(), | ||
computationState.getKeyHeartbeats(refreshDeadline, sampler)); | ||
} | ||
|
||
activeWorkRefresherFn.accept(heartbeats); | ||
} | ||
} |
Oops, something went wrong.