Skip to content

Commit

Permalink
[Core][Flink] Resolve streaming source high CPU usage (#8354)
Browse files Browse the repository at this point in the history
  • Loading branch information
zhangshenghang authored Dec 24, 2024
1 parent a1c9001 commit eae32bf
Show file tree
Hide file tree
Showing 2 changed files with 56 additions and 2 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -46,6 +46,8 @@ public class FlinkRowCollector implements Collector<SeaTunnelRow> {

private final Meter sourceReadQPS;

private boolean emptyThisPollNext = true;

public FlinkRowCollector(Config envConfig, MetricsContext metricsContext) {
this.flowControlGate = FlowControlGate.create(FlowControlStrategy.fromConfig(envConfig));
this.sourceReadCount = metricsContext.counter(MetricNames.SOURCE_RECEIVED_COUNT);
Expand All @@ -61,6 +63,7 @@ public void collect(SeaTunnelRow record) {
sourceReadCount.inc();
sourceReadBytes.inc(record.getBytesSize());
sourceReadQPS.markEvent();
emptyThisPollNext = false;
} catch (Exception e) {
throw new RuntimeException(e);
}
Expand All @@ -71,8 +74,19 @@ public Object getCheckpointLock() {
return this;
}

@Override
public boolean isEmptyThisPollNext() {
return emptyThisPollNext;
}

@Override
public void resetEmptyThisPollNext() {
this.emptyThisPollNext = true;
}

public FlinkRowCollector withReaderOutput(ReaderOutput<SeaTunnelRow> readerOutput) {
this.readerOutput = readerOutput;
this.emptyThisPollNext = true;
return this;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@

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

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

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

import java.util.List;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.Executors;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.TimeUnit;
import java.util.stream.Collectors;

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

private InputStatus inputStatus = InputStatus.MORE_AVAILABLE;

private volatile CompletableFuture<Void> availabilityFuture;

private static final long DEFAULT_WAIT_TIME_MILLIS = 1000L;

private final ScheduledExecutorService scheduledExecutor;

public FlinkSourceReader(
org.apache.seatunnel.api.source.SourceReader<SeaTunnelRow, SplitT> sourceReader,
org.apache.seatunnel.api.source.SourceReader.Context context,
Config envConfig) {
this.scheduledExecutor =
Executors.newSingleThreadScheduledExecutor(
new ThreadFactoryBuilder()
.setDaemon(true)
.setNameFormat(
String.format(
"source-reader-scheduler-%d",
context.getIndexOfSubtask()))
.build());
this.sourceReader = sourceReader;
this.context = context;
this.flinkRowCollector = new FlinkRowCollector(envConfig, context.getMetricsContext());
Expand All @@ -78,9 +97,19 @@ public void start() {
public InputStatus pollNext(ReaderOutput<SeaTunnelRow> output) throws Exception {
if (!((FlinkSourceReaderContext) context).isSendNoMoreElementEvent()) {
sourceReader.pollNext(flinkRowCollector.withReaderOutput(output));
if (flinkRowCollector.isEmptyThisPollNext()) {
synchronized (this) {
if (availabilityFuture == null || availabilityFuture.isDone()) {
availabilityFuture = new CompletableFuture<>();
scheduleComplete(availabilityFuture);
LOGGER.debug("No data available, wait for next poll.");
}
}
return InputStatus.NOTHING_AVAILABLE;
}
} else {
// reduce CPU idle
Thread.sleep(1000L);
Thread.sleep(DEFAULT_WAIT_TIME_MILLIS);
}
return inputStatus;
}
Expand All @@ -97,7 +126,8 @@ public List<SplitWrapper<SplitT>> snapshotState(long checkpointId) {

@Override
public CompletableFuture<Void> isAvailable() {
return CompletableFuture.completedFuture(null);
CompletableFuture<Void> future = availabilityFuture;
return future != null ? future : CompletableFuture.completedFuture(null);
}

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

@Override
public void close() throws Exception {
CompletableFuture<Void> future = availabilityFuture;
if (future != null && !future.isDone()) {
future.complete(null);
}
sourceReader.close();
context.getEventListener().onEvent(new ReaderCloseEvent());
scheduledExecutor.shutdown();
}

@Override
Expand All @@ -136,4 +171,9 @@ public void notifyCheckpointComplete(long checkpointId) throws Exception {
public void notifyCheckpointAborted(long checkpointId) throws Exception {
sourceReader.notifyCheckpointAborted(checkpointId);
}

private void scheduleComplete(CompletableFuture<Void> future) {
scheduledExecutor.schedule(
() -> future.complete(null), DEFAULT_WAIT_TIME_MILLIS, TimeUnit.MILLISECONDS);
}
}

0 comments on commit eae32bf

Please sign in to comment.