-
Notifications
You must be signed in to change notification settings - Fork 13.5k
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[FLINK-37383]Correct throttling logic on ThrottledIterator
The throttle function was updating its last batch check time before the sleep operation, causing it to underestimate the elapsed time and allow approximately double the intended throughput rate. Moving the timestamp update to after the sleep ensures the elapsed time calculation properly accounts for the full duration between batches, maintaining the configured rate limit. The commit refactors ThrottledIterator by: Adding injectable time supplier and sleep function for better testing Improving code maintainability with functional interfaces This change makes the code more testable and reliable while maintaining existing functionality. Add test coverage for ThrottledIterator edge cases Adds test coverage for invalid elements per second, consistent window size, and non-serializable source scenarios in ThrottledIterator tests.
- Loading branch information
1 parent
0d1b5aa
commit d772d70
Showing
2 changed files
with
188 additions
and
5 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
162 changes: 162 additions & 0 deletions
162
...src/test/java/org/apache/flink/streaming/test/examples/utils/ThrottledIteratorITCase.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,162 @@ | ||
/* | ||
* 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.flink.streaming.test.examples.utils; | ||
|
||
import org.apache.flink.streaming.examples.utils.ThrottledIterator; | ||
import org.apache.flink.util.TestLogger; | ||
|
||
import org.junit.Test; | ||
|
||
import java.io.Serializable; | ||
import java.util.Collections; | ||
import java.util.Iterator; | ||
import java.util.concurrent.atomic.AtomicLong; | ||
|
||
import static org.junit.Assert.assertEquals; | ||
import static org.junit.jupiter.api.Assertions.assertThrows; | ||
|
||
/** Tests for {@link ThrottledIterator}. */ | ||
public class ThrottledIteratorITCase extends TestLogger { | ||
|
||
@Test | ||
public void testThrottledIteratorRespectsElementsPerSecondRate() throws Exception { | ||
AtomicLong currentTime = new AtomicLong(1000L); | ||
AtomicLong sleepTime = new AtomicLong(); | ||
|
||
ThrottledIterator.TimeSupplier timeSupplier = getTimeSupplier(currentTime); | ||
ThrottledIterator.SleepFunction sleepFunction = sleepTime::set; | ||
|
||
// Create test data | ||
Iterator<Integer> source = new CustomIntegerIterator(); | ||
ThrottledIterator<Integer> throttledIterator = | ||
new ThrottledIterator<>(source, 2, timeSupplier, sleepFunction); | ||
|
||
// Simulate time progression | ||
currentTime.set(1300L); | ||
assertEquals(Integer.valueOf(1), throttledIterator.next()); | ||
|
||
currentTime.set(1700L); | ||
assertEquals(Integer.valueOf(2), throttledIterator.next()); | ||
|
||
currentTime.set(2000L); | ||
assertEquals(Integer.valueOf(3), throttledIterator.next()); | ||
|
||
// Verify sleep was called | ||
assertEquals(50L, sleepTime.get()); | ||
} | ||
|
||
@Test | ||
public void testThrottledIteratorHasConsistentWindowSizes() throws Exception { | ||
AtomicLong currentTime = new AtomicLong(1000L); | ||
AtomicLong sleepTime = new AtomicLong(); | ||
|
||
ThrottledIterator.TimeSupplier timeSupplier = getTimeSupplier(currentTime); | ||
ThrottledIterator.SleepFunction sleepFunction = sleepTime::set; | ||
|
||
// Create iterator that returns incrementing sequence | ||
Iterator<Integer> source = new CustomIntegerIterator(); | ||
ThrottledIterator<Integer> throttledIterator = | ||
new ThrottledIterator<>(source, 1, timeSupplier, sleepFunction); | ||
|
||
// Simulate time progression and consume elements | ||
currentTime.set(2020L); | ||
throttledIterator.next(); | ||
|
||
currentTime.set(3040L); | ||
throttledIterator.next(); | ||
|
||
currentTime.set(4030L); | ||
throttledIterator.next(); | ||
assertEquals(50L, sleepTime.get()); // First window sleep | ||
sleepTime.set(0L); | ||
|
||
currentTime.set(5100L); | ||
throttledIterator.next(); | ||
|
||
currentTime.set(6120L); | ||
throttledIterator.next(); | ||
|
||
currentTime.set(7140L); | ||
throttledIterator.next(); | ||
|
||
currentTime.set(8150L); | ||
throttledIterator.next(); | ||
assertEquals(50L, sleepTime.get()); // Second window sleep - same size | ||
} | ||
|
||
@Test | ||
public void testThrottledIteratorWithInvalidElementsPerSecond() { | ||
Iterator<Integer> source = Collections.emptyIterator(); | ||
|
||
assertThrows( | ||
IllegalArgumentException.class, | ||
() -> new ThrottledIterator<>(source, 0), | ||
"'elements per second' must be positive and not zero"); | ||
|
||
assertThrows( | ||
IllegalArgumentException.class, | ||
() -> new ThrottledIterator<>(source, -1), | ||
"'elements per second' must be positive and not zero"); | ||
} | ||
|
||
@Test | ||
public void testThrottledIteratorWithNonSerializableSource() { | ||
Iterator<Integer> nonSerializableSource = | ||
new Iterator<Integer>() { | ||
@Override | ||
public boolean hasNext() { | ||
return true; | ||
} | ||
|
||
@Override | ||
public Integer next() { | ||
return 1; | ||
} | ||
}; | ||
|
||
assertThrows( | ||
IllegalArgumentException.class, | ||
() -> new ThrottledIterator<>(nonSerializableSource, 1), | ||
"source must be java.io.Serializable"); | ||
} | ||
|
||
private static ThrottledIterator.TimeSupplier getTimeSupplier(AtomicLong currentTime) { | ||
return new ThrottledIterator.TimeSupplier() { | ||
|
||
@Override | ||
public long getCurrentTimeMillis() { | ||
return currentTime.get(); | ||
} | ||
}; | ||
} | ||
|
||
static class CustomIntegerIterator implements Iterator<Integer>, Serializable { | ||
private static final long serialVersionUID = 1L; | ||
private int count = 0; | ||
|
||
@Override | ||
public boolean hasNext() { | ||
return true; | ||
} | ||
|
||
@Override | ||
public Integer next() { | ||
return ++count; | ||
} | ||
} | ||
} |