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] Client print event which received from jobmaster #8696

Open
wants to merge 17 commits into
base: dev
Choose a base branch
from
2 changes: 2 additions & 0 deletions config/seatunnel.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,8 @@ seatunnel:
queue-type: blockingqueue
print-execution-info-interval: 60
print-job-metrics-info-interval: 60
print-job-event-info-interval: 30
job-event-queue-size: 100
slot-service:
dynamic-slot: true
checkpoint:
Expand Down
42 changes: 42 additions & 0 deletions docs/en/seatunnel-engine/rest-api-v1.md
Original file line number Diff line number Diff line change
Expand Up @@ -843,4 +843,46 @@ Returns a list of logs from the requested node.
To get a list of logs from the current node: `http://localhost:5801/hazelcast/rest/maps/log`
To get the content of a log file: `http://localhost:5801/hazelcast/rest/maps/log/job-898380162133917698.log`

</details>

------------------------------------------------------------------------------------------

### Return Job Event Information

<details>
<summary><code>GET</code> <code><b>/hazelcast/rest/maps/event/:jobId</b></code> <code>(Returns the list of job events)</code></summary>

#### Parameters

> | Parameter Name | Required | Type | Description |
> |----------------|----------|---------|-------------------|
> | jobId | Yes | string | The Job ID |

#### Response

```json
[
{
"createdTime": 1739501227166,
"eventType": "LIFECYCLE_READER_OPEN"
},
{
"createdTime": 1739501227232,
"eventType": "LIFECYCLE_ENUMERATOR_OPEN"
},
{
"createdTime": 1739501227457,
"eventType": "LIFECYCLE_ENUMERATOR_CLOSE"
},
{
"createdTime": 1739501227516,
"eventType": "LIFECYCLE_WRITER_CLOSE"
},
{
"createdTime": 1739501228305,
"eventType": "LIFECYCLE_READER_CLOSE"
}
]
```

</details>
46 changes: 45 additions & 1 deletion docs/en/seatunnel-engine/rest-api-v2.md
Original file line number Diff line number Diff line change
Expand Up @@ -907,4 +907,48 @@ To get the metrics, you need to open `Telemetry` first, or you will get an empty

More information about `Telemetry` can be found in the [Telemetry](telemetry.md) documentation.

</details>
</details>



------------------------------------------------------------------------------------------

### Return Job Event Information

<details>
<summary><code>GET</code> <code><b>/event/:jobId</b></code> <code>(Returns the list of job events)</code></summary>

#### Parameters

> | Parameter Name | Required | Type | Description |
> |----------------|----------|---------|-------------------|
> | jobId | Yes | string | The Job ID |

#### Response

```json
[
{
"createdTime": 1739501227166,
"eventType": "LIFECYCLE_READER_OPEN"
},
{
"createdTime": 1739501227232,
"eventType": "LIFECYCLE_ENUMERATOR_OPEN"
},
{
"createdTime": 1739501227457,
"eventType": "LIFECYCLE_ENUMERATOR_CLOSE"
},
{
"createdTime": 1739501227516,
"eventType": "LIFECYCLE_WRITER_CLOSE"
},
{
"createdTime": 1739501228305,
"eventType": "LIFECYCLE_READER_CLOSE"
}
]
```

</details>
43 changes: 43 additions & 0 deletions docs/zh/seatunnel-engine/rest-api-v1.md
Original file line number Diff line number Diff line change
Expand Up @@ -850,3 +850,46 @@ network:
获取日志文件内容:`http://localhost:5801/hazelcast/rest/maps/log/job-898380162133917698.log`

</details>


------------------------------------------------------------------------------------------

### 返回作业的事件信息

<details>
<summary><code>GET</code> <code><b>/hazelcast/rest/maps/event/:jobId</b></code> <code>(返回作业事件列表)</code></summary>

#### 参数

> | 参数名称 | 是否必传 | 参数类型 | 参数描述 |
> |-------|------|--------|------|
> | jobId | 是 | string | Job ID |

#### 响应

```json
[
{
"createdTime": 1739501227166,
"eventType": "LIFECYCLE_READER_OPEN"
},
{
"createdTime": 1739501227232,
"eventType": "LIFECYCLE_ENUMERATOR_OPEN"
},
{
"createdTime": 1739501227457,
"eventType": "LIFECYCLE_ENUMERATOR_CLOSE"
},
{
"createdTime": 1739501227516,
"eventType": "LIFECYCLE_WRITER_CLOSE"
},
{
"createdTime": 1739501228305,
"eventType": "LIFECYCLE_READER_CLOSE"
}
]
```

</details>
44 changes: 43 additions & 1 deletion docs/zh/seatunnel-engine/rest-api-v2.md
Original file line number Diff line number Diff line change
Expand Up @@ -895,4 +895,46 @@ curl --location 'http://127.0.0.1:8080/submit-job/upload' --form 'config_file=@"

更多关于`Telemetry`的信息可以在[Telemetry](telemetry.md)文档中找到。

</details>
</details>

------------------------------------------------------------------------------------------

### 返回作业的事件信息

<details>
<summary><code>GET</code> <code><b>/event/:jobId</b></code> <code>(返回作业事件列表)</code></summary>

#### 参数

> | 参数名称 | 是否必传 | 参数类型 | 参数描述 |
> |-------|------|--------|------|
> | jobId | 是 | string | Job ID |

#### 响应

```json
[
{
"createdTime": 1739501227166,
"eventType": "LIFECYCLE_READER_OPEN"
},
{
"createdTime": 1739501227232,
"eventType": "LIFECYCLE_ENUMERATOR_OPEN"
},
{
"createdTime": 1739501227457,
"eventType": "LIFECYCLE_ENUMERATOR_CLOSE"
},
{
"createdTime": 1739501227516,
"eventType": "LIFECYCLE_WRITER_CLOSE"
},
{
"createdTime": 1739501228305,
"eventType": "LIFECYCLE_READER_CLOSE"
}
]
```

</details>
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@

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

import org.apache.seatunnel.api.event.Event;
import org.apache.seatunnel.common.utils.DateTimeUtils;
import org.apache.seatunnel.common.utils.StringFormatUtils;
import org.apache.seatunnel.core.starter.command.Command;
Expand All @@ -29,6 +30,7 @@
import org.apache.seatunnel.engine.client.SeaTunnelClient;
import org.apache.seatunnel.engine.client.job.ClientJobExecutionEnvironment;
import org.apache.seatunnel.engine.client.job.ClientJobProxy;
import org.apache.seatunnel.engine.client.job.JobEventRunner;
import org.apache.seatunnel.engine.client.job.JobMetricsRunner;
import org.apache.seatunnel.engine.client.job.JobStatusRunner;
import org.apache.seatunnel.engine.common.Constant;
Expand Down Expand Up @@ -56,6 +58,7 @@
import java.time.LocalDateTime;
import java.util.Collections;
import java.util.List;
import java.util.Optional;
import java.util.Random;
import java.util.concurrent.Executors;
import java.util.concurrent.ScheduledExecutorService;
Expand Down Expand Up @@ -84,6 +87,7 @@ public void execute() throws CommandExecuteException {
LocalDateTime startTime = LocalDateTime.now();
LocalDateTime endTime = LocalDateTime.now();
SeaTunnelConfig seaTunnelConfig = ConfigProvider.locateAndGetSeaTunnelConfig();
Long jobId = null;
try {
String clusterName = clientCommandArgs.getClusterName();
ClientConfig clientConfig = ConfigProvider.locateAndGetClientConfig();
Expand Down Expand Up @@ -158,7 +162,6 @@ public void execute() throws CommandExecuteException {
? Long.parseLong(clientCommandArgs.getCustomJobId())
: null);
}

// get job start time
startTime = LocalDateTime.now();
// create job proxy
Expand Down Expand Up @@ -189,11 +192,11 @@ public void execute() throws CommandExecuteException {
}
}));
// get job id
long jobId = clientJobProxy.getJobId();
jobId = clientJobProxy.getJobId();
JobMetricsRunner jobMetricsRunner = new JobMetricsRunner(engineClient, jobId);
executorService =
Executors.newScheduledThreadPool(
2,
3,
new ThreadFactoryBuilder()
.setNameFormat("job-metrics-runner-%d")
.setDaemon(true)
Expand All @@ -204,6 +207,15 @@ public void execute() throws CommandExecuteException {
seaTunnelConfig.getEngineConfig().getPrintJobMetricsInfoInterval(),
TimeUnit.SECONDS);

JobEventRunner jobEventRunner =
new JobEventRunner(engineClient.getJobClient(), jobId);
executorService.scheduleAtFixedRate(
jobEventRunner,
0,
seaTunnelConfig.getEngineConfig().getPrintJobEventInfoInterval(),
TimeUnit.SECONDS);
log.info("Init job event runner , job id : {}", jobId);

if (!isLocalMode) {
// LOCAL mode does not require running the job status runner
executorService.schedule(
Expand All @@ -227,6 +239,9 @@ public void execute() throws CommandExecuteException {
} catch (Exception e) {
throw new CommandExecuteException("SeaTunnel job executed failed", e);
} finally {

printEvent(jobId);

if (jobMetricsSummary != null) {
// print job statistics information when job finished
log.info(
Expand All @@ -252,6 +267,25 @@ public void execute() throws CommandExecuteException {
}
}

private void printEvent(Long jobId) {
if (jobId == null) {
jobId =
Optional.ofNullable(clientCommandArgs.getJobId())
.map(Long::parseLong)
.orElseGet(
() ->
Optional.ofNullable(clientCommandArgs.getCustomJobId())
.map(Long::parseLong)
.orElse(null));
}
if (jobId != null) {
List<Event> event = engineClient.getJobClient().getEvent(jobId);
event.forEach(e -> log.info("EventType: {}", e.getEventType()));
} else {
log.warn("JobId is null, can not get job event.");
}
}

private void closeClient() {
if (engineClient != null) {
engineClient.close();
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,68 @@
/*
* 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.seatunnel.engine.e2e;

import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.Test;
import org.testcontainers.containers.Container;

import java.io.IOException;
import java.util.regex.Matcher;
import java.util.regex.Pattern;

public class JobEventIT extends SeaTunnelEngineContainer {

private static final String TEST_CONFIG_FILE = "/fakesource_to_console.conf";
private static final String LIFECYCLE_EVENT = "LIFECYCLE_READER_OPEN";
private static final String JOB_ID_PATTERN = "job id:\\s*(\\d+)";
private static final String EVENT_API_ENDPOINT = "localhost:8080/event/";

@Test
public void testJobRestoreApplyResources() throws IOException, InterruptedException {
// Execute job and verify output
Container.ExecResult execResult = executeJob(server, TEST_CONFIG_FILE);
String stdout = execResult.getStdout();
Assertions.assertNotNull(stdout, "Job execution output should not be null");
Assertions.assertTrue(
stdout.contains(LIFECYCLE_EVENT), "Job output should contain lifecycle event");

// Extract job ID from logs
String jobId = extractJobId(server.getLogs());
Assertions.assertNotNull(jobId, "Failed to extract job ID from logs");

// Query job event API
Container.ExecResult eventResult = queryJobEvent(jobId);
String eventOutput = eventResult.getStdout();
Assertions.assertNotNull(eventOutput, "Event API response should not be null");
Assertions.assertTrue(
eventOutput.contains(LIFECYCLE_EVENT),
"Event API response should contain lifecycle event");
}

private String extractJobId(String logs) {
Pattern pattern = Pattern.compile(JOB_ID_PATTERN);
Matcher matcher = pattern.matcher(logs);
return matcher.find() ? matcher.group(1) : null;
}

private Container.ExecResult queryJobEvent(String jobId)
throws InterruptedException, IOException {
return server.execInContainer(
"sh", "-c", String.format("curl %s%s", EVENT_API_ENDPOINT, jobId));
}
}
Loading
Loading