Skip to content

Commit 6a426b2

Browse files
[FLINK-36039][autoscaler] Support clean historical event handler records in JDBC event handler (#865)
--------- Co-authored-by: Rui Fan <[email protected]>
1 parent 0ef627e commit 6a426b2

File tree

9 files changed

+308
-20
lines changed

9 files changed

+308
-20
lines changed

docs/layouts/shortcodes/generated/autoscaler_standalone_configuration.html

Lines changed: 6 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -38,6 +38,12 @@
3838
<td>Integer</td>
3939
<td>The port of flink cluster when the flink-cluster fetcher is used.</td>
4040
</tr>
41+
<tr>
42+
<td><h5>autoscaler.standalone.jdbc.event-handler.ttl</h5></td>
43+
<td style="word-wrap: break-word;">90 d</td>
44+
<td>Duration</td>
45+
<td>The time to live based on create time for the JDBC event handler records. When the config is set as '0', the ttl strategy for the records would be disabled.</td>
46+
</tr>
4147
<tr>
4248
<td><h5>autoscaler.standalone.jdbc.password-env-variable</h5></td>
4349
<td style="word-wrap: break-word;">"JDBC_PWD"</td>

flink-autoscaler-plugin-jdbc/src/main/java/org/apache/flink/autoscaler/jdbc/event/JdbcAutoScalerEventHandler.java

Lines changed: 73 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -18,18 +18,27 @@
1818
package org.apache.flink.autoscaler.jdbc.event;
1919

2020
import org.apache.flink.annotation.Experimental;
21+
import org.apache.flink.annotation.VisibleForTesting;
2122
import org.apache.flink.autoscaler.JobAutoScalerContext;
2223
import org.apache.flink.autoscaler.ScalingSummary;
2324
import org.apache.flink.autoscaler.event.AutoScalerEventHandler;
2425
import org.apache.flink.runtime.jobgraph.JobVertexID;
26+
import org.apache.flink.util.Preconditions;
27+
28+
import org.apache.flink.shaded.guava31.com.google.common.util.concurrent.ThreadFactoryBuilder;
2529

2630
import lombok.SneakyThrows;
31+
import lombok.extern.slf4j.Slf4j;
2732

2833
import javax.annotation.Nullable;
2934

35+
import java.sql.Timestamp;
3036
import java.time.Duration;
3137
import java.util.Map;
3238
import java.util.Objects;
39+
import java.util.concurrent.Executors;
40+
import java.util.concurrent.ScheduledExecutorService;
41+
import java.util.concurrent.TimeUnit;
3342

3443
/**
3544
* The event handler which persists its event in JDBC related database.
@@ -38,13 +47,34 @@
3847
* @param <Context> The job autoscaler context.
3948
*/
4049
@Experimental
50+
@Slf4j
4151
public class JdbcAutoScalerEventHandler<KEY, Context extends JobAutoScalerContext<KEY>>
4252
implements AutoScalerEventHandler<KEY, Context> {
4353

4454
private final JdbcEventInteractor jdbcEventInteractor;
55+
private final Duration eventHandlerTtl;
56+
@Nullable private final ScheduledExecutorService scheduledEventHandlerCleaner;
4557

46-
public JdbcAutoScalerEventHandler(JdbcEventInteractor jdbcEventInteractor) {
58+
public JdbcAutoScalerEventHandler(
59+
JdbcEventInteractor jdbcEventInteractor, Duration eventHandlerTtl) {
4760
this.jdbcEventInteractor = jdbcEventInteractor;
61+
this.eventHandlerTtl = Preconditions.checkNotNull(eventHandlerTtl);
62+
63+
if (eventHandlerTtl.toMillis() <= 0) {
64+
this.scheduledEventHandlerCleaner = null;
65+
return;
66+
}
67+
this.scheduledEventHandlerCleaner =
68+
Executors.newSingleThreadScheduledExecutor(
69+
new ThreadFactoryBuilder()
70+
.setNameFormat("jdbc-autoscaler-events-cleaner")
71+
.setDaemon(true)
72+
.build());
73+
this.scheduledEventHandlerCleaner.scheduleAtFixedRate(
74+
this::cleanExpiredEvents,
75+
Duration.ofDays(1).toMillis(),
76+
Duration.ofDays(1).toMillis(),
77+
TimeUnit.MILLISECONDS);
4878
}
4979

5080
@SneakyThrows
@@ -104,6 +134,48 @@ public void handleScalingEvent(
104134
}
105135
}
106136

137+
@Override
138+
public void close() {
139+
if (Objects.nonNull(scheduledEventHandlerCleaner)
140+
&& !scheduledEventHandlerCleaner.isShutdown()) {
141+
scheduledEventHandlerCleaner.shutdownNow();
142+
}
143+
}
144+
145+
@VisibleForTesting
146+
void cleanExpiredEvents() {
147+
final var batchSize = 2000;
148+
final var sleepMs = 1000;
149+
150+
var date =
151+
Timestamp.from(
152+
jdbcEventInteractor
153+
.getCurrentInstant()
154+
.minusMillis(eventHandlerTtl.toMillis()));
155+
try {
156+
var deletedTotalCount = 0L;
157+
while (true) {
158+
Long minId = jdbcEventInteractor.queryMinEventIdByCreateTime(date);
159+
if (Objects.isNull(minId)) {
160+
log.info(
161+
"Deleted expired {} event handler records successfully",
162+
deletedTotalCount);
163+
break;
164+
}
165+
166+
for (long startId = minId;
167+
jdbcEventInteractor.deleteExpiredEventsByIdRangeAndDate(
168+
startId, startId + batchSize, date)
169+
== batchSize;
170+
startId += batchSize) {
171+
Thread.sleep(sleepMs);
172+
}
173+
}
174+
} catch (Exception e) {
175+
log.error("Error in cleaning expired event handler records.", e);
176+
}
177+
}
178+
107179
/**
108180
* @return True means the existing event is still in the interval duration we can update it.
109181
* Otherwise, it's too early, we should create a new one instead of updating it.

flink-autoscaler-plugin-jdbc/src/main/java/org/apache/flink/autoscaler/jdbc/event/JdbcEventInteractor.java

Lines changed: 26 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -22,6 +22,7 @@
2222
import org.apache.flink.util.Preconditions;
2323

2424
import javax.annotation.Nonnull;
25+
import javax.annotation.Nullable;
2526

2627
import java.sql.Connection;
2728
import java.sql.ResultSet;
@@ -152,4 +153,29 @@ protected List<AutoScalerEvent> queryEvents(String jobKey, String reason) throws
152153
void setClock(@Nonnull Clock clock) {
153154
this.clock = Preconditions.checkNotNull(clock);
154155
}
156+
157+
@Nullable
158+
Long queryMinEventIdByCreateTime(Timestamp timestamp) throws Exception {
159+
var sql =
160+
"SELECT id from t_flink_autoscaler_event_handler "
161+
+ " where id = (SELECT id FROM t_flink_autoscaler_event_handler order by id asc limit 1) "
162+
+ " and create_time < ?";
163+
try (var pstmt = conn.prepareStatement(sql)) {
164+
pstmt.setObject(1, timestamp);
165+
ResultSet resultSet = pstmt.executeQuery();
166+
return resultSet.next() ? resultSet.getLong(1) : null;
167+
}
168+
}
169+
170+
int deleteExpiredEventsByIdRangeAndDate(long startId, long endId, Timestamp timestamp)
171+
throws Exception {
172+
var query =
173+
"delete from t_flink_autoscaler_event_handler where id >= ? and id < ? and create_time < ?";
174+
try (var pstmt = conn.prepareStatement(query)) {
175+
pstmt.setObject(1, startId);
176+
pstmt.setObject(2, endId);
177+
pstmt.setObject(3, timestamp);
178+
return pstmt.executeUpdate();
179+
}
180+
}
155181
}

0 commit comments

Comments
 (0)