Skip to content

Commit eae32bf

Browse files
[Core][Flink] Resolve streaming source high CPU usage (#8354)
1 parent a1c9001 commit eae32bf

File tree

2 files changed

+56
-2
lines changed

2 files changed

+56
-2
lines changed

seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/source/FlinkRowCollector.java

+14
Original file line numberDiff line numberDiff line change
@@ -46,6 +46,8 @@ public class FlinkRowCollector implements Collector<SeaTunnelRow> {
4646

4747
private final Meter sourceReadQPS;
4848

49+
private boolean emptyThisPollNext = true;
50+
4951
public FlinkRowCollector(Config envConfig, MetricsContext metricsContext) {
5052
this.flowControlGate = FlowControlGate.create(FlowControlStrategy.fromConfig(envConfig));
5153
this.sourceReadCount = metricsContext.counter(MetricNames.SOURCE_RECEIVED_COUNT);
@@ -61,6 +63,7 @@ public void collect(SeaTunnelRow record) {
6163
sourceReadCount.inc();
6264
sourceReadBytes.inc(record.getBytesSize());
6365
sourceReadQPS.markEvent();
66+
emptyThisPollNext = false;
6467
} catch (Exception e) {
6568
throw new RuntimeException(e);
6669
}
@@ -71,8 +74,19 @@ public Object getCheckpointLock() {
7174
return this;
7275
}
7376

77+
@Override
78+
public boolean isEmptyThisPollNext() {
79+
return emptyThisPollNext;
80+
}
81+
82+
@Override
83+
public void resetEmptyThisPollNext() {
84+
this.emptyThisPollNext = true;
85+
}
86+
7487
public FlinkRowCollector withReaderOutput(ReaderOutput<SeaTunnelRow> readerOutput) {
7588
this.readerOutput = readerOutput;
89+
this.emptyThisPollNext = true;
7690
return this;
7791
}
7892
}

seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/source/FlinkSourceReader.java

+42-2
Original file line numberDiff line numberDiff line change
@@ -17,6 +17,7 @@
1717

1818
package org.apache.seatunnel.translation.flink.source;
1919

20+
import org.apache.seatunnel.shade.com.google.common.util.concurrent.ThreadFactoryBuilder;
2021
import org.apache.seatunnel.shade.com.typesafe.config.Config;
2122

2223
import org.apache.seatunnel.api.source.SourceSplit;
@@ -34,6 +35,9 @@
3435

3536
import java.util.List;
3637
import java.util.concurrent.CompletableFuture;
38+
import java.util.concurrent.Executors;
39+
import java.util.concurrent.ScheduledExecutorService;
40+
import java.util.concurrent.TimeUnit;
3741
import java.util.stream.Collectors;
3842

3943
/**
@@ -55,10 +59,25 @@ public class FlinkSourceReader<SplitT extends SourceSplit>
5559

5660
private InputStatus inputStatus = InputStatus.MORE_AVAILABLE;
5761

62+
private volatile CompletableFuture<Void> availabilityFuture;
63+
64+
private static final long DEFAULT_WAIT_TIME_MILLIS = 1000L;
65+
66+
private final ScheduledExecutorService scheduledExecutor;
67+
5868
public FlinkSourceReader(
5969
org.apache.seatunnel.api.source.SourceReader<SeaTunnelRow, SplitT> sourceReader,
6070
org.apache.seatunnel.api.source.SourceReader.Context context,
6171
Config envConfig) {
72+
this.scheduledExecutor =
73+
Executors.newSingleThreadScheduledExecutor(
74+
new ThreadFactoryBuilder()
75+
.setDaemon(true)
76+
.setNameFormat(
77+
String.format(
78+
"source-reader-scheduler-%d",
79+
context.getIndexOfSubtask()))
80+
.build());
6281
this.sourceReader = sourceReader;
6382
this.context = context;
6483
this.flinkRowCollector = new FlinkRowCollector(envConfig, context.getMetricsContext());
@@ -78,9 +97,19 @@ public void start() {
7897
public InputStatus pollNext(ReaderOutput<SeaTunnelRow> output) throws Exception {
7998
if (!((FlinkSourceReaderContext) context).isSendNoMoreElementEvent()) {
8099
sourceReader.pollNext(flinkRowCollector.withReaderOutput(output));
100+
if (flinkRowCollector.isEmptyThisPollNext()) {
101+
synchronized (this) {
102+
if (availabilityFuture == null || availabilityFuture.isDone()) {
103+
availabilityFuture = new CompletableFuture<>();
104+
scheduleComplete(availabilityFuture);
105+
LOGGER.debug("No data available, wait for next poll.");
106+
}
107+
}
108+
return InputStatus.NOTHING_AVAILABLE;
109+
}
81110
} else {
82111
// reduce CPU idle
83-
Thread.sleep(1000L);
112+
Thread.sleep(DEFAULT_WAIT_TIME_MILLIS);
84113
}
85114
return inputStatus;
86115
}
@@ -97,7 +126,8 @@ public List<SplitWrapper<SplitT>> snapshotState(long checkpointId) {
97126

98127
@Override
99128
public CompletableFuture<Void> isAvailable() {
100-
return CompletableFuture.completedFuture(null);
129+
CompletableFuture<Void> future = availabilityFuture;
130+
return future != null ? future : CompletableFuture.completedFuture(null);
101131
}
102132

103133
@Override
@@ -123,8 +153,13 @@ public void handleSourceEvents(SourceEvent sourceEvent) {
123153

124154
@Override
125155
public void close() throws Exception {
156+
CompletableFuture<Void> future = availabilityFuture;
157+
if (future != null && !future.isDone()) {
158+
future.complete(null);
159+
}
126160
sourceReader.close();
127161
context.getEventListener().onEvent(new ReaderCloseEvent());
162+
scheduledExecutor.shutdown();
128163
}
129164

130165
@Override
@@ -136,4 +171,9 @@ public void notifyCheckpointComplete(long checkpointId) throws Exception {
136171
public void notifyCheckpointAborted(long checkpointId) throws Exception {
137172
sourceReader.notifyCheckpointAborted(checkpointId);
138173
}
174+
175+
private void scheduleComplete(CompletableFuture<Void> future) {
176+
scheduledExecutor.schedule(
177+
() -> future.complete(null), DEFAULT_WAIT_TIME_MILLIS, TimeUnit.MILLISECONDS);
178+
}
139179
}

0 commit comments

Comments
 (0)