Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -353,7 +353,7 @@ private BasicRequestInfo createRequestInfo() {
private void flush() throws InterruptedException {
RequestInfo requestInfo = createRequestInfo();
while (rateLimitingStrategy.shouldBlock(requestInfo)) {
mailboxExecutor.yield();
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I am curious, is the yield behaviour an explicit contract documented on the yield method or due to the implementations of yield?

If it is not explicit, we should update the javadoc for the yield method in the interface.

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It is explicitly mentioned in the Javadoc that until or unless another command is not triggered :
/**
* This method starts running the command at the head of the mailbox and is intended to be used
* by the mailbox thread to yield from a currently ongoing action to another command. The method
* blocks until another command to run is available in the mailbox and must only be called from
* the mailbox thread. Must only be called from the mailbox thread to not violate the
* single-threaded execution model.
*
* @throws InterruptedException on interruption.
* @throws IllegalStateException if the mailbox is closed and can no longer supply runnables for
* yielding.
* @throws FlinkRuntimeException if executed {@link RunnableWithException} thrown an exception.
*/

yieldIfThereExistsInFlightRequests();
requestInfo = createRequestInfo();
}

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,207 @@
/*
* 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.connector.base.sink.writer;

import org.apache.flink.api.common.operators.MailboxExecutor;
import org.apache.flink.connector.base.sink.writer.config.AsyncSinkWriterConfiguration;
import org.apache.flink.connector.base.sink.writer.strategy.AIMDScalingStrategy;
import org.apache.flink.connector.base.sink.writer.strategy.CongestionControlRateLimitingStrategy;
import org.apache.flink.util.function.ThrowingRunnable;

import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;

import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.concurrent.atomic.AtomicInteger;

import static org.assertj.core.api.Assertions.assertThat;

/**
* Tests for conditional yielding behavior in AsyncSinkWriter flush method. Verifies that
* yieldIfThereExistsInFlightRequests() only yields when there are actual in-flight requests.
*/
class AsyncSinkWriterYieldTest {

private TestSinkInitContext sinkInitContext;
private final List<Integer> results = new ArrayList<>();
private AtomicInteger yieldCallCount;

@BeforeEach
void setup() {
yieldCallCount = new AtomicInteger(0);
sinkInitContext = new TestSinkInitContext();
results.clear();
}

@Test
void testFlushWithEmptyBufferDoesNotYield() throws Exception {
TrackingMailboxExecutor trackingMailbox = new TrackingMailboxExecutor(yieldCallCount);
TestSinkInitContext contextWithTracking =
new TestSinkInitContextWithCustomMailbox(trackingMailbox);

TestAsyncSinkWriter sink =
new TestAsyncSinkWriter(contextWithTracking, 10, 100, 1000, results);

int yieldCountBefore = yieldCallCount.get();
sink.flush(false);
int yieldCountAfter = yieldCallCount.get();

assertThat(yieldCountAfter).isEqualTo(yieldCountBefore);
assertThat(yieldCountAfter).isEqualTo(0);
}

@Test
void testFlushWithBufferedElementsButNoInFlightRequestsDoesNotYield() throws Exception {
TrackingMailboxExecutor trackingMailbox = new TrackingMailboxExecutor(yieldCallCount);
TestSinkInitContext contextWithTracking =
new TestSinkInitContextWithCustomMailbox(trackingMailbox);

TestAsyncSinkWriter sink =
new TestAsyncSinkWriter(contextWithTracking, 10, 100, 1000, results);

sink.write("1");
sink.write("2");

int yieldCountBefore = yieldCallCount.get();
sink.flush(false);
int yieldCountAfter = yieldCallCount.get();

assertThat(yieldCountAfter).isEqualTo(yieldCountBefore);
assertThat(yieldCountAfter).isEqualTo(0);
}

@Test
void testFlushWithTrueFlushesAllElementsWithoutYielding() throws Exception {
TrackingMailboxExecutor trackingMailbox = new TrackingMailboxExecutor(yieldCallCount);
TestSinkInitContext contextWithTracking =
new TestSinkInitContextWithCustomMailbox(trackingMailbox);

TestAsyncSinkWriter sink =
new TestAsyncSinkWriter(contextWithTracking, 10, 100, 1000, results);

sink.write("1");
sink.write("2");
sink.write("3");

sink.flush(true);

assertThat(results).containsExactly(1, 2, 3);
assertThat(yieldCallCount.get()).isEqualTo(0);
}

private static class TestAsyncSinkWriter extends AsyncSinkWriter<String, Integer> {
private final List<Integer> results;

TestAsyncSinkWriter(
TestSinkInitContext context,
int maxBatchSize,
int maxBufferedRequests,
long maxBatchSizeInBytes,
List<Integer> results) {
super(
(elem, ctx) -> Integer.parseInt(elem),
context,
AsyncSinkWriterConfiguration.builder()
.setMaxBatchSize(maxBatchSize)
.setMaxBatchSizeInBytes(maxBatchSizeInBytes)
.setMaxInFlightRequests(1)
.setMaxBufferedRequests(maxBufferedRequests)
.setMaxTimeInBufferMS(1000)
.setMaxRecordSizeInBytes(100)
.setRateLimitingStrategy(
CongestionControlRateLimitingStrategy.builder()
.setInitialMaxInFlightMessages(maxBatchSize)
.setMaxInFlightRequests(1)
.setScalingStrategy(
AIMDScalingStrategy.builder(maxBatchSize)
.build())
.build())
.build(),
Collections.emptyList());
this.results = results;
}

@Override
protected void submitRequestEntries(
List<Integer> requestEntries, ResultHandler<Integer> resultHandler) {
results.addAll(requestEntries);
resultHandler.complete();
}

@Override
protected long getSizeInBytes(Integer requestEntry) {
return 4;
}

public void write(String val) throws IOException, InterruptedException {
write(val, null);
}
}

private static class TrackingMailboxExecutor implements MailboxExecutor {
private final AtomicInteger yieldCount;

TrackingMailboxExecutor(AtomicInteger yieldCount) {
this.yieldCount = yieldCount;
}

@Override
public void execute(
MailOptions mailOptions,
ThrowingRunnable<? extends Exception> command,
String descriptionFormat,
Object... descriptionArgs) {
try {
command.run();
} catch (Exception e) {
throw new RuntimeException(e);
}
}

@Override
public void yield() throws InterruptedException {
yieldCount.incrementAndGet();
}

@Override
public boolean tryYield() {
return false;
}

@Override
public boolean shouldInterrupt() {
return false;
}
}

private static class TestSinkInitContextWithCustomMailbox extends TestSinkInitContext {
private final MailboxExecutor customMailbox;

TestSinkInitContextWithCustomMailbox(MailboxExecutor mailbox) {
this.customMailbox = mailbox;
}

@Override
public MailboxExecutor getMailboxExecutor() {
return customMailbox;
}
}
}
100 changes: 100 additions & 0 deletions flink-examples/base-connector-examples/pom.xml
Original file line number Diff line number Diff line change
@@ -0,0 +1,100 @@
<?xml version="1.0" encoding="UTF-8"?>
<!--
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.
-->
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">

<modelVersion>4.0.0</modelVersion>

<parent>
<groupId>org.apache.flink</groupId>
<artifactId>flink-examples</artifactId>
<version>2.3-SNAPSHOT</version>
</parent>


<artifactId>flink-base-connector</artifactId>
<version>1.0-SNAPSHOT</version>

<properties>
<maven.compiler.source>11</maven.compiler.source>
<maven.compiler.target>11</maven.compiler.target>
<flink.version>2.2.0</flink.version>
</properties>

<dependencies>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-streaming-java</artifactId>
<version>${flink.version}</version>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-clients</artifactId>
<version>${flink.version}</version>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-connector-base</artifactId>
<version>${flink.version}</version>
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-connector-files</artifactId>
<version>${flink.version}</version>
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-connector-datagen</artifactId>
<version>${flink.version}</version>
</dependency>
<dependency>
<groupId>com.github.vladimir-bukhtoyarov</groupId>
<artifactId>bucket4j-core</artifactId>
<version>7.6.0</version>
</dependency>
</dependencies>

<build>
<plugins>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-shade-plugin</artifactId>
<version>3.2.4</version>
<executions>
<execution>
<phase>package</phase>
<goals>
<goal>shade</goal>
</goals>
<configuration>
<artifactSet>
<includes>
<include>com.github.vladimir-bukhtoyarov:bucket4j-core</include>
</includes>
</artifactSet>
</configuration>
</execution>
</executions>
</plugin>
</plugins>
</build>
</project>
Original file line number Diff line number Diff line change
@@ -0,0 +1,57 @@
package org.apache.flink;

import org.apache.flink.api.common.eventtime.WatermarkStrategy;
import org.apache.flink.api.common.typeinfo.Types;
import org.apache.flink.connector.datagen.source.DataGeneratorSource;
import org.apache.flink.streaming.api.datastream.DataStream;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;

public class AsyncSinkHangDemo {

public static void main(String[] args) throws Exception {
StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
env.enableCheckpointing(5000); // Checkpoint every 5 seconds - will timeout due to hang

// Generate continuous stream of data using DataGeneratorSource
DataGeneratorSource<String> source = new DataGeneratorSource<>(
(Long index) -> "Record-" + index,
1000, // records per second
Types.STRING
);

DataStream<String> stream = env.fromSource(
source,
WatermarkStrategy.noWatermarks(),
"DataGenerator"
);

// AsyncSink with rate limiting - demonstrates the hang when rate limiting blocks with no in-flight requests
// Configuration: maxInFlightRequests=1, tokensPerSecond=5, tokensPerMinute=100
// This will cause tokens to be exhausted quickly, triggering the hang scenario:
// 1. Tokens get consumed by initial requests
// 2. shouldBlock() returns true (no tokens available)
// 3. currentInFlightRequests drops to 0 (requests complete fast)
// 4. flush() loops on mailboxExecutor.yield() indefinitely
// 5. Checkpoint cannot complete -> timeout -> job failure
TokenBucketRateLimitingStrategy rateLimiter = new TokenBucketRateLimitingStrategy(1, 5, 100);
DummyAsyncSink sink = new DummyAsyncSink(rateLimiter);
stream.sinkTo(sink);

// File sink - original implementation (commented out)
/*
FileSink<String> sink = FileSink
.forRowFormat(new Path("/tmp/flink-output"), new SimpleStringEncoder<String>("UTF-8"))
.withRollingPolicy(
DefaultRollingPolicy.builder()
.withRolloverInterval(Duration.ofMinutes(1))
.withInactivityInterval(Duration.ofSeconds(30))
.withMaxPartSize(MemorySize.ofMebiBytes(1))
.build())
.build();

stream.sinkTo(sink);
*/

env.execute("AsyncSink Hang Demo");
}
}
Loading