|
| 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