Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[Feature][Zeta]The expiration time of a historical Job can be config #5180

Merged
merged 14 commits into from
Aug 10, 2023
Merged
Show file tree
Hide file tree
Changes from 11 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
1 change: 1 addition & 0 deletions config/seatunnel.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@

seatunnel:
engine:
history-job-expire-minutes: 1440
backup-count: 1
queue-type: blockingqueue
print-execution-info-interval: 60
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@
import org.apache.seatunnel.engine.client.job.JobExecutionEnvironment;
import org.apache.seatunnel.engine.common.config.ConfigProvider;
import org.apache.seatunnel.engine.common.config.JobConfig;
import org.apache.seatunnel.engine.core.job.JobResult;
import org.apache.seatunnel.engine.core.job.JobStatus;
import org.apache.seatunnel.engine.server.SeaTunnelServerStarter;

Expand Down Expand Up @@ -129,6 +130,32 @@ public void cancelJobTest() throws Exception {
objectCompletableFuture.get())));
}

@Test
public void testExpiredJobWasDeleted() throws Exception {
Common.setDeployMode(DeployMode.CLIENT);
String filePath = TestUtils.getResource("batch_fakesource_to_file.conf");
JobConfig jobConfig = new JobConfig();
jobConfig.setName("job_expire");

ClientConfig clientConfig = ConfigProvider.locateAndGetClientConfig();
clientConfig.setClusterName(TestUtils.getClusterName("JobExecutionIT"));
SeaTunnelClient engineClient = new SeaTunnelClient(clientConfig);
JobExecutionEnvironment jobExecutionEnv =
engineClient.createExecutionContext(filePath, jobConfig);

final ClientJobProxy clientJobProxy = jobExecutionEnv.execute();

JobResult result = clientJobProxy.doWaitForJobComplete().get();
Assertions.assertEquals(result.getStatus(), JobStatus.FINISHED);
Awaitility.await()
.atMost(65, TimeUnit.SECONDS)
.untilAsserted(
() ->
Assertions.assertThrowsExactly(
NullPointerException.class,
wu-a-ge marked this conversation as resolved.
Show resolved Hide resolved
() -> clientJobProxy.getJobStatus()));
}

@AfterAll
static void afterClass() {
if (hazelcastInstance != null) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@

seatunnel:
engine:
history-job-expire-minutes: 1
backup-count: 2
queue-type: blockingqueue
print-execution-info-interval: 10
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,7 @@
@Data
@SuppressWarnings("checkstyle:MagicNumber")
public class EngineConfig {

private int backupCount = ServerConfigOptions.BACKUP_COUNT.defaultValue();
private int printExecutionInfoInterval =
ServerConfigOptions.PRINT_EXECUTION_INFO_INTERVAL.defaultValue();
Expand All @@ -50,6 +51,8 @@ public class EngineConfig {
private CheckpointConfig checkpointConfig = ServerConfigOptions.CHECKPOINT.defaultValue();

private QueueType queueType = ServerConfigOptions.QUEUE_TYPE.defaultValue();
private int historyJobExpireMinutes =
ServerConfigOptions.HISTORY_JOB_EXPIRE_MINUTES.defaultValue();

public void setBackupCount(int newBackupCount) {
checkBackupCount(newBackupCount, 0);
Expand Down Expand Up @@ -82,6 +85,13 @@ public void setTaskExecutionThreadShareMode(ThreadShareMode taskExecutionThreadS
this.taskExecutionThreadShareMode = taskExecutionThreadShareMode;
}

public void setHistoryJobExpireMinutes(int historyJobExpireMinutes) {
checkPositive(
historyJobExpireMinutes,
ServerConfigOptions.HISTORY_JOB_EXPIRE_MINUTES + " must be > 0");
this.historyJobExpireMinutes = historyJobExpireMinutes;
}

public EngineConfig setQueueType(QueueType queueType) {
checkNotNull(queueType);
this.queueType = queueType;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -131,6 +131,11 @@ private void parseEngineConfig(Node engineNode, SeaTunnelConfig config) {
engineConfig.setSlotServiceConfig(parseSlotServiceConfig(node));
} else if (ServerConfigOptions.CHECKPOINT.key().equals(name)) {
engineConfig.setCheckpointConfig(parseCheckpointConfig(node));
} else if (ServerConfigOptions.HISTORY_JOB_EXPIRE_MINUTES.key().equals(name)) {
engineConfig.setHistoryJobExpireMinutes(
getIntegerValue(
ServerConfigOptions.HISTORY_JOB_EXPIRE_MINUTES.key(),
getTextContent(node)));
} else {
LOGGER.warning("Unrecognized element: " + name);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -145,4 +145,9 @@ public class ServerConfigOptions {
.type(new TypeReference<Map<String, String>>() {})
.noDefaultValue()
.withDescription("The checkpoint storage instance configuration.");
public static final Option<Integer> HISTORY_JOB_EXPIRE_MINUTES =
Options.key("history-job-expire-minutes")
.intType()
.defaultValue(1440)
.withDescription("The expire time of history jobs.time unit minute");
}
Original file line number Diff line number Diff line change
Expand Up @@ -220,7 +220,8 @@ private void initCoordinatorService() {
.getMap(Constant.IMAP_FINISHED_JOB_METRICS),
nodeEngine
.getHazelcastInstance()
.getMap(Constant.IMAP_FINISHED_JOB_VERTEX_INFO));
.getMap(Constant.IMAP_FINISHED_JOB_VERTEX_INFO),
engineConfig.getHistoryJobExpireMinutes());

List<CompletableFuture<Void>> collect =
runningJobInfoIMap.entrySet().stream()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -76,20 +76,22 @@ public class JobHistoryService {
* finishedJobStateImap key is jobId and value is jobState(json) JobStateData Indicates the
* status of the job, pipeline, and task
*/
// TODO need to limit the amount of storage
private final IMap<Long, JobState> finishedJobStateImap;

private final IMap<Long, JobMetrics> finishedJobMetricsImap;

private final ObjectMapper objectMapper;

private final int finishedJobExpireTime;

public JobHistoryService(
IMap<Object, Object> runningJobStateIMap,
ILogger logger,
Map<Long, JobMaster> runningJobMasterMap,
IMap<Long, JobState> finishedJobStateImap,
IMap<Long, JobMetrics> finishedJobMetricsImap,
IMap<Long, JobDAGInfo> finishedJobVertexInfoImap) {
IMap<Long, JobDAGInfo> finishedJobVertexInfoImap,
int finishedJobExpireTime) {
this.runningJobStateIMap = runningJobStateIMap;
this.logger = logger;
this.runningJobMasterMap = runningJobMasterMap;
Expand All @@ -98,6 +100,7 @@ public JobHistoryService(
this.finishedJobDAGInfoImap = finishedJobVertexInfoImap;
this.objectMapper = new ObjectMapper();
this.objectMapper.configure(SerializationFeature.FAIL_ON_EMPTY_BEANS, false);
this.finishedJobExpireTime = finishedJobExpireTime;
}

// Gets the status of a running and completed job
Expand Down Expand Up @@ -168,14 +171,14 @@ public String getJobDetailStateAsString(Long jobId) {
public void storeFinishedJobState(JobMaster jobMaster) {
JobState jobState = toJobStateMapper(jobMaster, false);
jobState.setFinishTime(System.currentTimeMillis());
finishedJobStateImap.put(jobState.jobId, jobState, 14, TimeUnit.DAYS);
finishedJobStateImap.put(jobState.jobId, jobState, finishedJobExpireTime, TimeUnit.MINUTES);
}

@SuppressWarnings("checkstyle:MagicNumber")
public void storeFinishedPipelineMetrics(long jobId, JobMetrics metrics) {
finishedJobMetricsImap.computeIfAbsent(jobId, key -> JobMetrics.of(new HashMap<>()));
JobMetrics newMetrics = finishedJobMetricsImap.get(jobId).merge(metrics);
finishedJobMetricsImap.put(jobId, newMetrics, 14, TimeUnit.DAYS);
finishedJobMetricsImap.put(jobId, newMetrics, finishedJobExpireTime, TimeUnit.MINUTES);
}

private JobState toJobStateMapper(JobMaster jobMaster, boolean simple) {
Expand Down Expand Up @@ -234,7 +237,7 @@ private JobState toJobStateMapper(JobMaster jobMaster, boolean simple) {
}

public void storeJobInfo(long jobId, JobDAGInfo jobInfo) {
finishedJobDAGInfoImap.put(jobId, jobInfo);
finishedJobDAGInfoImap.put(jobId, jobInfo, finishedJobExpireTime, TimeUnit.MINUTES);
}

@AllArgsConstructor
Expand Down