Skip to content

Commit 092647c

Browse files
committed
[FLINK-26063][runtime] Set current key before polling next PQ element
Some InternalPriorityQueue implementations need a correct key/group set before performing poll() or remove(). In particular, ChangelogKeyGroupedPriorityQueue logs key group so that state changes can be re-distributed or shuffled. This change re-orders queue.poll and keyContext.setCurrentKey.
1 parent 7fe549e commit 092647c

File tree

3 files changed

+362
-2
lines changed

3 files changed

+362
-2
lines changed

flink-runtime/src/main/java/org/apache/flink/runtime/state/InternalPriorityQueue.java

Lines changed: 6 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -39,6 +39,9 @@ public interface InternalPriorityQueue<T> {
3939
* Retrieves and removes the first element (w.r.t. the order) of this set, or returns {@code
4040
* null} if this set is empty.
4141
*
42+
* <p>NOTE: Correct key (i.e. the key of the polled element) must be set on KeyContext before
43+
* calling this method.
44+
*
4245
* @return the first element of this ordered set, or {@code null} if this set is empty.
4346
*/
4447
@Nullable
@@ -67,6 +70,9 @@ public interface InternalPriorityQueue<T> {
6770
/**
6871
* Removes the given element from the set, if is contained in the set.
6972
*
73+
* <p>NOTE: Correct key (i.e. the key of the polled element) must be set on KeyContext before
74+
* calling this method.
75+
*
7076
* @param toRemove the element to remove.
7177
* @return <code>true</code> if the operation changed the head element or if it is unclear if
7278
* the head element changed. Only returns <code>false</code> if the head element was not

flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimerServiceImpl.java

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -279,8 +279,8 @@ private void onProcessingTime(long time) throws Exception {
279279
InternalTimer<K, N> timer;
280280

281281
while ((timer = processingTimeTimersQueue.peek()) != null && timer.getTimestamp() <= time) {
282-
processingTimeTimersQueue.poll();
283282
keyContext.setCurrentKey(timer.getKey());
283+
processingTimeTimersQueue.poll();
284284
triggerTarget.onProcessingTime(timer);
285285
}
286286

@@ -297,8 +297,8 @@ public void advanceWatermark(long time) throws Exception {
297297
InternalTimer<K, N> timer;
298298

299299
while ((timer = eventTimeTimersQueue.peek()) != null && timer.getTimestamp() <= time) {
300-
eventTimeTimersQueue.poll();
301300
keyContext.setCurrentKey(timer.getKey());
301+
eventTimeTimersQueue.poll();
302302
triggerTarget.onEventTime(timer);
303303
}
304304
}
Lines changed: 354 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,354 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one
3+
* or more contributor license agreements. See the NOTICE file
4+
* distributed with this work for additional information
5+
* regarding copyright ownership. The ASF licenses this file
6+
* to you under the Apache License, Version 2.0 (the
7+
* "License"); you may not use this file except in compliance
8+
* with the License. You may obtain a copy of the License at
9+
*
10+
* http://www.apache.org/licenses/LICENSE-2.0
11+
*
12+
* Unless required by applicable law or agreed to in writing, software
13+
* distributed under the License is distributed on an "AS IS" BASIS,
14+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15+
* See the License for the specific language governing permissions and
16+
* limitations under the License.
17+
*/
18+
19+
package org.apache.flink.test.state;
20+
21+
import org.apache.flink.api.common.JobID;
22+
import org.apache.flink.api.common.eventtime.WatermarkStrategy;
23+
import org.apache.flink.api.connector.source.ReaderOutput;
24+
import org.apache.flink.api.connector.source.SourceReader;
25+
import org.apache.flink.api.connector.source.SourceReaderContext;
26+
import org.apache.flink.api.connector.source.lib.NumberSequenceSource;
27+
import org.apache.flink.api.connector.source.lib.util.IteratorSourceReader;
28+
import org.apache.flink.api.connector.source.lib.util.IteratorSourceSplit;
29+
import org.apache.flink.changelog.fs.FsStateChangelogStorageFactory;
30+
import org.apache.flink.configuration.Configuration;
31+
import org.apache.flink.configuration.MemorySize;
32+
import org.apache.flink.core.io.InputStatus;
33+
import org.apache.flink.runtime.jobgraph.JobGraph;
34+
import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration;
35+
import org.apache.flink.streaming.api.CheckpointingMode;
36+
import org.apache.flink.streaming.api.datastream.DataStreamUtils;
37+
import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
38+
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
39+
import org.apache.flink.streaming.api.functions.sink.DiscardingSink;
40+
import org.apache.flink.streaming.api.functions.windowing.ProcessWindowFunction;
41+
import org.apache.flink.streaming.api.windowing.assigners.SlidingProcessingTimeWindows;
42+
import org.apache.flink.streaming.api.windowing.time.Time;
43+
import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
44+
import org.apache.flink.test.util.MiniClusterWithClientResource;
45+
import org.apache.flink.util.Collector;
46+
import org.apache.flink.util.TestLogger;
47+
48+
import org.junit.After;
49+
import org.junit.Before;
50+
import org.junit.Rule;
51+
import org.junit.Test;
52+
import org.junit.rules.TemporaryFolder;
53+
import org.junit.runner.RunWith;
54+
import org.junit.runners.Parameterized;
55+
import org.junit.runners.Parameterized.Parameters;
56+
57+
import java.io.File;
58+
import java.io.IOException;
59+
import java.io.Serializable;
60+
import java.time.Duration;
61+
import java.util.Iterator;
62+
import java.util.Timer;
63+
import java.util.TimerTask;
64+
import java.util.concurrent.ExecutionException;
65+
import java.util.concurrent.ThreadLocalRandom;
66+
import java.util.concurrent.atomic.AtomicBoolean;
67+
import java.util.function.Consumer;
68+
69+
import static org.apache.flink.changelog.fs.FsStateChangelogOptions.PREEMPTIVE_PERSIST_THRESHOLD;
70+
import static org.apache.flink.configuration.CheckpointingOptions.CHECKPOINTS_DIRECTORY;
71+
import static org.apache.flink.configuration.CheckpointingOptions.CHECKPOINT_STORAGE;
72+
import static org.apache.flink.configuration.CheckpointingOptions.FS_SMALL_FILE_THRESHOLD;
73+
import static org.apache.flink.configuration.CheckpointingOptions.LOCAL_RECOVERY;
74+
import static org.apache.flink.configuration.CoreOptions.DEFAULT_PARALLELISM;
75+
import static org.apache.flink.configuration.PipelineOptions.OBJECT_REUSE;
76+
import static org.apache.flink.configuration.RestartStrategyOptions.RESTART_STRATEGY;
77+
import static org.apache.flink.configuration.StateBackendOptions.STATE_BACKEND;
78+
import static org.apache.flink.configuration.StateChangelogOptions.ENABLE_STATE_CHANGE_LOG;
79+
import static org.apache.flink.configuration.StateChangelogOptions.PERIODIC_MATERIALIZATION_INTERVAL;
80+
import static org.apache.flink.configuration.TaskManagerOptions.BUFFER_DEBLOAT_ENABLED;
81+
import static org.apache.flink.runtime.jobgraph.SavepointRestoreSettings.forPath;
82+
import static org.apache.flink.runtime.testutils.CommonTestUtils.waitForAllTaskRunning;
83+
import static org.apache.flink.streaming.api.environment.CheckpointConfig.ExternalizedCheckpointCleanup.RETAIN_ON_CANCELLATION;
84+
import static org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions.ALIGNED_CHECKPOINT_TIMEOUT;
85+
import static org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions.CHECKPOINTING_INTERVAL;
86+
import static org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions.CHECKPOINTING_MODE;
87+
import static org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions.ENABLE_UNALIGNED;
88+
import static org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions.EXTERNALIZED_CHECKPOINT;
89+
import static org.apache.flink.test.util.TestUtils.getMostRecentCompletedCheckpoint;
90+
import static org.apache.flink.test.util.TestUtils.getMostRecentCompletedCheckpointMaybe;
91+
import static org.apache.flink.util.Preconditions.checkArgument;
92+
93+
/**
94+
* Tests rescaling with Changelog enabled and with timers in state. It uses an adaptation of a
95+
* ChangelogTestProgram that aims to generate the highest load possible while still allowing
96+
* checkpointing. For that, it uses rate-limited FLIP-27 source and Unaligned checkpoints.
97+
*/
98+
@RunWith(Parameterized.class)
99+
public class ChangelogRescalingITCase extends TestLogger {
100+
/** The rate at which events will be generated by the source. */
101+
private static final int EVENTS_PER_SECOND_PER_READER = 100;
102+
/** Payload size of each event generated randomly. */
103+
private static final int PAYLOAD_SIZE = 1000;
104+
/** Size of (ProcessingTime) windows. */
105+
private static final Time WINDOW_SIZE = Time.milliseconds(100);
106+
/** Slide size. */
107+
private static final Time WINDOW_SLIDE = Time.milliseconds(10);
108+
/** Time to Accumulate some timer delete operations. */
109+
private static final int ACCUMULATE_TIME_MILLIS = 5_000;
110+
111+
@Rule public TemporaryFolder temporaryFolder = new TemporaryFolder();
112+
113+
@Parameters(name = "Rescale {0} -> {1}")
114+
public static Object[] parameters() {
115+
return new Object[][] {new Object[] {6, 4}, new Object[] {4, 6}};
116+
}
117+
118+
private final int parallelism1;
119+
private final int parallelism2;
120+
121+
private MiniClusterWithClientResource cluster;
122+
123+
public ChangelogRescalingITCase(int parallelism1, int parallelism2) {
124+
this.parallelism1 = parallelism1;
125+
this.parallelism2 = parallelism2;
126+
}
127+
128+
@Before
129+
public void before() throws Exception {
130+
Configuration configuration = new Configuration();
131+
FsStateChangelogStorageFactory.configure(configuration, temporaryFolder.newFolder());
132+
cluster =
133+
new MiniClusterWithClientResource(
134+
new MiniClusterResourceConfiguration.Builder()
135+
.setConfiguration(configuration)
136+
.setNumberSlotsPerTaskManager(Math.max(parallelism1, parallelism2))
137+
.build());
138+
cluster.before();
139+
}
140+
141+
@After
142+
public void after() {
143+
if (cluster != null) {
144+
cluster.after();
145+
cluster = null;
146+
}
147+
}
148+
149+
@Test
150+
public void test() throws Exception {
151+
// before rescale
152+
File cpDir1 = temporaryFolder.newFolder();
153+
JobID jobID1 = submit(configureJob(parallelism1, cpDir1), graph -> {});
154+
155+
Thread.sleep(ACCUMULATE_TIME_MILLIS);
156+
File cpLocation = checkpointAndCancel(jobID1, cpDir1);
157+
158+
// rescale and checkpoint to verify
159+
JobID jobID2 =
160+
submit(
161+
configureJob(parallelism2, temporaryFolder.newFolder()),
162+
graph ->
163+
graph.setSavepointRestoreSettings(
164+
forPath(cpLocation.toURI().toString())));
165+
waitForAllTaskRunning(cluster.getMiniCluster(), jobID2, true);
166+
cluster.getClusterClient().cancel(jobID2).get();
167+
}
168+
169+
private JobID submit(Configuration conf, Consumer<JobGraph> updateGraph)
170+
throws InterruptedException, ExecutionException {
171+
JobGraph jobGraph = createJobGraph(conf);
172+
updateGraph.accept(jobGraph);
173+
return cluster.getClusterClient().submitJob(jobGraph).get();
174+
}
175+
176+
private JobGraph createJobGraph(Configuration conf) {
177+
StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(conf);
178+
SingleOutputStreamOperator<TestEvent> map =
179+
env.fromSource(
180+
new ThrottlingNumberSequenceSource(
181+
0, Long.MAX_VALUE, EVENTS_PER_SECOND_PER_READER),
182+
WatermarkStrategy.noWatermarks(),
183+
"Sequence Source")
184+
.keyBy(ChangelogRescalingITCase::key)
185+
.map(
186+
el -> {
187+
// Thread.sleep(100); // don't block barriers
188+
byte[] bytes = new byte[PAYLOAD_SIZE];
189+
ThreadLocalRandom.current().nextBytes(bytes);
190+
return new TestEvent(el, bytes);
191+
});
192+
DataStreamUtils.reinterpretAsKeyedStream(map, e -> key(e.id))
193+
.window(SlidingProcessingTimeWindows.of(WINDOW_SIZE, WINDOW_SLIDE))
194+
.process(
195+
new ProcessWindowFunction<TestEvent, String, Long, TimeWindow>() {
196+
@Override
197+
public void process(
198+
Long key,
199+
ProcessWindowFunction<TestEvent, String, Long, TimeWindow>
200+
.Context
201+
context,
202+
Iterable<TestEvent> elements,
203+
Collector<String> out) {}
204+
})
205+
.addSink(new DiscardingSink<>());
206+
207+
return env.getStreamGraph().getJobGraph();
208+
}
209+
210+
private static long key(Long num) {
211+
return num % 1000;
212+
}
213+
214+
private Configuration configureJob(int parallelism, File cpDir) {
215+
Configuration conf = new Configuration();
216+
217+
conf.set(EXTERNALIZED_CHECKPOINT, RETAIN_ON_CANCELLATION);
218+
conf.set(DEFAULT_PARALLELISM, parallelism);
219+
conf.set(ENABLE_STATE_CHANGE_LOG, true);
220+
conf.set(CHECKPOINTING_MODE, CheckpointingMode.EXACTLY_ONCE);
221+
conf.set(CHECKPOINTING_INTERVAL, Duration.ofMillis(10));
222+
conf.set(CHECKPOINT_STORAGE, "filesystem");
223+
conf.set(CHECKPOINTS_DIRECTORY, cpDir.toURI().toString());
224+
conf.set(STATE_BACKEND, "hashmap");
225+
conf.set(LOCAL_RECOVERY, false); // not supported by changelog
226+
// tune changelog
227+
conf.set(PREEMPTIVE_PERSIST_THRESHOLD, MemorySize.ofMebiBytes(10));
228+
conf.set(PERIODIC_MATERIALIZATION_INTERVAL, Duration.ofMinutes(3));
229+
// tune flink
230+
conf.set(FS_SMALL_FILE_THRESHOLD, MemorySize.ofMebiBytes(1));
231+
conf.set(OBJECT_REUSE, true);
232+
233+
conf.set(ENABLE_UNALIGNED, true); // speedup
234+
conf.set(ALIGNED_CHECKPOINT_TIMEOUT, Duration.ZERO); // prevent randomization
235+
conf.set(BUFFER_DEBLOAT_ENABLED, false); // prevent randomization
236+
conf.set(RESTART_STRATEGY, "none"); // not expecting any failures
237+
238+
return conf;
239+
}
240+
241+
private static final class TestEvent implements Serializable {
242+
private final long id;
243+
244+
@SuppressWarnings({"FieldCanBeLocal", "unused"})
245+
private final byte[] payload;
246+
247+
private TestEvent(long id, byte[] payload) {
248+
this.id = id;
249+
this.payload = payload;
250+
}
251+
}
252+
253+
private static class ThrottlingNumberSequenceSource extends NumberSequenceSource {
254+
private final int numbersPerSecond;
255+
256+
public ThrottlingNumberSequenceSource(long from, long to, int numbersPerSecondPerReader) {
257+
super(from, to);
258+
this.numbersPerSecond = numbersPerSecondPerReader;
259+
}
260+
261+
@Override
262+
public SourceReader<Long, NumberSequenceSplit> createReader(
263+
SourceReaderContext readerContext) {
264+
return new ThrottlingIteratorSourceReader<>(
265+
readerContext, new SourceRateLimiter(numbersPerSecond));
266+
}
267+
}
268+
269+
private static class ThrottlingIteratorSourceReader<
270+
E, IterT extends Iterator<E>, SplitT extends IteratorSourceSplit<E, IterT>>
271+
extends IteratorSourceReader<E, IterT, SplitT> {
272+
private final SourceRateLimiter rateLimiter;
273+
274+
public ThrottlingIteratorSourceReader(
275+
SourceReaderContext context, SourceRateLimiter rateLimiter) {
276+
super(context);
277+
this.rateLimiter = rateLimiter;
278+
}
279+
280+
@Override
281+
public InputStatus pollNext(ReaderOutput<E> output) {
282+
if (rateLimiter.request()) {
283+
return super.pollNext(output);
284+
} else {
285+
return InputStatus.NOTHING_AVAILABLE;
286+
}
287+
}
288+
}
289+
290+
private static final class SourceRateLimiter {
291+
private final AtomicBoolean newTokensAdded = new AtomicBoolean(false);
292+
private final int tokensToAdd;
293+
private int tokensAvailable;
294+
295+
public SourceRateLimiter(int tokensPerSecond) {
296+
this(
297+
tokensPerSecond < 10 ? 1000 : 100,
298+
tokensPerSecond < 10 ? tokensPerSecond : tokensPerSecond / 10);
299+
}
300+
301+
public SourceRateLimiter(int intervalMs, int tokensToAdd) {
302+
checkArgument(intervalMs > 0);
303+
checkArgument(tokensToAdd > 0);
304+
this.tokensToAdd = tokensToAdd;
305+
this.tokensAvailable = tokensToAdd;
306+
new Timer("source-limiter", true)
307+
.scheduleAtFixedRate(
308+
new TimerTask() {
309+
@Override
310+
public void run() {
311+
newTokensAdded.set(true); // "catch up" is ok
312+
}
313+
},
314+
intervalMs,
315+
intervalMs);
316+
}
317+
318+
public boolean request() {
319+
if (tokensAvailable == 0 && newTokensAdded.compareAndSet(true, false)) {
320+
tokensAvailable = tokensToAdd;
321+
}
322+
if (tokensAvailable > 0) {
323+
tokensAvailable--;
324+
return true;
325+
} else {
326+
return false;
327+
}
328+
}
329+
}
330+
331+
private File checkpointAndCancel(JobID jobID, File cpDir)
332+
throws IOException, InterruptedException, ExecutionException {
333+
while (!getMostRecentCompletedCheckpointMaybe(cpDir).isPresent()) {
334+
checkStatus(jobID);
335+
Thread.sleep(50);
336+
}
337+
cluster.getClusterClient().cancel(jobID).get();
338+
checkStatus(jobID);
339+
return getMostRecentCompletedCheckpoint(cpDir);
340+
}
341+
342+
private void checkStatus(JobID jobID) throws InterruptedException, ExecutionException {
343+
if (cluster.getClusterClient().getJobStatus(jobID).get().isGloballyTerminalState()) {
344+
cluster.getClusterClient()
345+
.requestJobResult(jobID)
346+
.get()
347+
.getSerializedThrowable()
348+
.ifPresent(
349+
serializedThrowable -> {
350+
throw new RuntimeException(serializedThrowable);
351+
});
352+
}
353+
}
354+
}

0 commit comments

Comments
 (0)