diff --git a/iotdb-collector/collector-core/pom.xml b/iotdb-collector/collector-core/pom.xml
index 7d9c42d3..a0e45028 100644
--- a/iotdb-collector/collector-core/pom.xml
+++ b/iotdb-collector/collector-core/pom.xml
@@ -156,6 +156,10 @@
commons-codec
commons-codec
+
+ org.apache.kafka
+ kafka-clients
+
diff --git a/iotdb-collector/collector-core/src/assembly/resources/conf/application.properties b/iotdb-collector/collector-core/src/assembly/resources/conf/application.properties
index 7cfcea42..d72b18c1 100644
--- a/iotdb-collector/collector-core/src/assembly/resources/conf/application.properties
+++ b/iotdb-collector/collector-core/src/assembly/resources/conf/application.properties
@@ -33,7 +33,7 @@ api_service_port=17070
# The number of concurrent threads for the source task.
# Effective mode: on every start
# Data type: int
-task_source_parallelism_num=1
+task_source_parallelism_num=4
# The number of concurrent threads for the process task.
# Effective mode: on every start
@@ -60,6 +60,16 @@ task_sink_ring_buffer_size=1024
# Data type: string
task_database_file_path=system/database/task.db
+# Proactively triggers the interval for batch deliveries
+# Effective mode: on every start
+# Data type: long
+executor_cron_heartbeat_event_interval_seconds=20
+
+# Task progress storage interval
+# Effective mode: on every start
+# Data type: int
+task_progress_report_interval=60
+
####################
### Plugin Configuration
####################
@@ -146,9 +156,4 @@ pipe_leader_cache_memory_usage_percentage=0.1
# Enable/disable reference tracking for pipe events
# Effective mode: on every start
# Data type: boolean
-pipe_event_reference_tracking_enabled=true
-
-# Proactively triggers the interval for batch deliveries
-# Effective mode: on every start
-# Data type: long
-executor_cron_heartbeat_event_interval_seconds=20
\ No newline at end of file
+pipe_event_reference_tracking_enabled=true
\ No newline at end of file
diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/Application.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/Application.java
index 80b5b42e..26174a13 100644
--- a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/Application.java
+++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/Application.java
@@ -22,9 +22,9 @@
import org.apache.iotdb.collector.config.Configuration;
import org.apache.iotdb.collector.service.ApiService;
import org.apache.iotdb.collector.service.IService;
-import org.apache.iotdb.collector.service.PeriodicalJobService;
import org.apache.iotdb.collector.service.PersistenceService;
import org.apache.iotdb.collector.service.RuntimeService;
+import org.apache.iotdb.collector.service.ScheduleService;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -41,8 +41,8 @@ public class Application {
private Application() {
services.add(new RuntimeService());
services.add(new ApiService());
+ services.add(new ScheduleService());
services.add(new PersistenceService());
- services.add(new PeriodicalJobService());
}
public static void main(String[] args) {
diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/config/PipeRuntimeOptions.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/config/PipeRuntimeOptions.java
index 0abd49ab..c5f42020 100644
--- a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/config/PipeRuntimeOptions.java
+++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/config/PipeRuntimeOptions.java
@@ -272,12 +272,4 @@ public void setValue(final String valueString) {
value = Integer.parseInt(valueString);
}
};
-
- public static final Option EXECUTOR_CRON_HEARTBEAT_EVENT_INTERVAL_SECONDS =
- new Option("executor_cron_heartbeat_event_interval_seconds", 20L) {
- @Override
- public void setValue(final String valueString) {
- value = Long.parseLong(valueString);
- }
- };
}
diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/config/TaskRuntimeOptions.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/config/TaskRuntimeOptions.java
index dc35f339..35dda43c 100644
--- a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/config/TaskRuntimeOptions.java
+++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/config/TaskRuntimeOptions.java
@@ -72,4 +72,20 @@ public void setValue(final String valueString) {
value = addHomeDir(valueString);
}
};
+
+ public static final Option EXECUTOR_CRON_HEARTBEAT_EVENT_INTERVAL_SECONDS =
+ new Option("executor_cron_heartbeat_event_interval_seconds", 20L) {
+ @Override
+ public void setValue(final String valueString) {
+ value = Long.parseLong(valueString);
+ }
+ };
+
+ public static final Option TASK_PROGRESS_REPORT_INTERVAL =
+ new Option("task_progress_report_interval", 60) {
+ @Override
+ public void setValue(String valueString) {
+ value = Integer.parseInt(valueString);
+ }
+ };
}
diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/persistence/DBConstant.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/persistence/DBConstant.java
index 5e18ce13..6928d62e 100644
--- a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/persistence/DBConstant.java
+++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/persistence/DBConstant.java
@@ -41,6 +41,7 @@ public class DBConstant {
+ " source_attribute BLOB NOT NULL,\n"
+ " processor_attribute BLOB NOT NULL,\n"
+ " sink_attribute BLOB NOT NULL,\n"
+ + " task_progress BLOB NOT NULL,\n"
+ " create_time TEXT NOT NULL\n"
+ ");";
diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/persistence/TaskPersistence.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/persistence/TaskPersistence.java
index 573f5b4f..2cf0c6cf 100644
--- a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/persistence/TaskPersistence.java
+++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/persistence/TaskPersistence.java
@@ -20,19 +20,18 @@
package org.apache.iotdb.collector.persistence;
import org.apache.iotdb.collector.config.TaskRuntimeOptions;
+import org.apache.iotdb.collector.runtime.progress.ProgressIndex;
import org.apache.iotdb.collector.runtime.task.TaskStateEnum;
+import org.apache.iotdb.collector.runtime.task.event.ProgressReportEvent;
import org.apache.iotdb.collector.service.RuntimeService;
+import org.apache.iotdb.collector.utils.SerializationUtil;
-import org.apache.tsfile.utils.PublicBAOS;
-import org.apache.tsfile.utils.ReadWriteIOUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import javax.ws.rs.core.Response;
-import java.io.DataOutputStream;
import java.io.IOException;
-import java.nio.ByteBuffer;
import java.nio.file.Files;
import java.nio.file.Path;
import java.nio.file.Paths;
@@ -41,9 +40,9 @@
import java.sql.ResultSet;
import java.sql.SQLException;
import java.sql.Timestamp;
-import java.util.HashMap;
import java.util.Map;
import java.util.Objects;
+import java.util.Optional;
public class TaskPersistence extends Persistence {
@@ -68,9 +67,9 @@ protected void initDatabaseFileIfPossible() {
@Override
protected void initTableIfPossible() {
- try (final Connection connection = getConnection()) {
- final PreparedStatement statement =
- connection.prepareStatement(DBConstant.CREATE_TASK_TABLE_SQL);
+ try (final Connection connection = getConnection();
+ final PreparedStatement statement =
+ connection.prepareStatement(DBConstant.CREATE_TASK_TABLE_SQL)) {
statement.executeUpdate();
} catch (final SQLException e) {
LOGGER.warn("Failed to create task database", e);
@@ -82,10 +81,9 @@ public void tryResume() {
final String queryAllTaskSQL =
"SELECT task_id, task_state, source_attribute, processor_attribute, sink_attribute, create_time FROM task";
- try (final Connection connection = getConnection()) {
- final PreparedStatement statement = connection.prepareStatement(queryAllTaskSQL);
- final ResultSet taskResultSet = statement.executeQuery();
-
+ try (final Connection connection = getConnection();
+ final PreparedStatement statement = connection.prepareStatement(queryAllTaskSQL);
+ final ResultSet taskResultSet = statement.executeQuery()) {
while (taskResultSet.next()) {
final String taskId = taskResultSet.getString(1);
final TaskStateEnum taskState = TaskStateEnum.values()[taskResultSet.getInt(2)];
@@ -96,9 +94,9 @@ public void tryResume() {
tryRecoverTask(
taskId,
taskState,
- deserialize(sourceAttribute),
- deserialize(processorAttribute),
- deserialize(sinkAttribute));
+ SerializationUtil.deserialize(sourceAttribute),
+ SerializationUtil.deserialize(processorAttribute),
+ SerializationUtil.deserialize(sinkAttribute));
}
} catch (final SQLException e) {
LOGGER.warn("Failed to resume task persistence message, because {}", e.getMessage());
@@ -125,21 +123,6 @@ public void tryRecoverTask(
}
}
- private Map deserialize(final byte[] buffer) {
- final Map attribute = new HashMap<>();
- final ByteBuffer attributeBuffer = ByteBuffer.wrap(buffer);
-
- final int size = ReadWriteIOUtils.readInt(attributeBuffer);
- for (int i = 0; i < size; i++) {
- final String key = ReadWriteIOUtils.readString(attributeBuffer);
- final String value = ReadWriteIOUtils.readString(attributeBuffer);
-
- attribute.put(key, value);
- }
-
- return attribute;
- }
-
public void tryPersistenceTask(
final String taskId,
final TaskStateEnum taskState,
@@ -147,21 +130,22 @@ public void tryPersistenceTask(
final Map processorAttribute,
final Map sinkAttribute) {
final String insertSQL =
- "INSERT INTO task(task_id, task_state , source_attribute, processor_attribute, sink_attribute, create_time) values(?, ?, ?, ?, ?, ?)";
+ "INSERT INTO task(task_id, task_state , source_attribute, processor_attribute, sink_attribute,task_progress, create_time) values(?, ?,?, ?, ?, ?, ?)";
- try (final Connection connection = getConnection()) {
- final PreparedStatement statement = connection.prepareStatement(insertSQL);
+ try (final Connection connection = getConnection();
+ final PreparedStatement statement = connection.prepareStatement(insertSQL)) {
- final byte[] sourceAttributeBuffer = serialize(sourceAttribute);
- final byte[] processorAttributeBuffer = serialize(processorAttribute);
- final byte[] sinkAttributeBuffer = serialize(sinkAttribute);
+ final byte[] sourceAttributeBuffer = SerializationUtil.serialize(sourceAttribute);
+ final byte[] processorAttributeBuffer = SerializationUtil.serialize(processorAttribute);
+ final byte[] sinkAttributeBuffer = SerializationUtil.serialize(sinkAttribute);
statement.setString(1, taskId);
statement.setInt(2, taskState.getTaskState());
statement.setBytes(3, sourceAttributeBuffer);
statement.setBytes(4, processorAttributeBuffer);
statement.setBytes(5, sinkAttributeBuffer);
- statement.setString(6, String.valueOf(new Timestamp(System.currentTimeMillis())));
+ statement.setBytes(6, null);
+ statement.setString(7, String.valueOf(new Timestamp(System.currentTimeMillis())));
statement.executeUpdate();
LOGGER.info("successfully persisted task {} info", taskId);
@@ -170,25 +154,11 @@ public void tryPersistenceTask(
}
}
- private byte[] serialize(final Map attribute) throws IOException {
- try (final PublicBAOS byteArrayOutputStream = new PublicBAOS();
- final DataOutputStream outputStream = new DataOutputStream(byteArrayOutputStream)) {
- ReadWriteIOUtils.write(attribute.size(), outputStream);
- for (final Map.Entry entry : attribute.entrySet()) {
- ReadWriteIOUtils.write(entry.getKey(), outputStream);
- ReadWriteIOUtils.write(entry.getValue(), outputStream);
- }
-
- return ByteBuffer.wrap(byteArrayOutputStream.getBuf(), 0, byteArrayOutputStream.size())
- .array();
- }
- }
-
public void tryDeleteTask(final String taskId) {
final String deleteSQL = "DELETE FROM task WHERE task_id = ?";
- try (final Connection connection = getConnection()) {
- final PreparedStatement statement = connection.prepareStatement(deleteSQL);
+ try (final Connection connection = getConnection();
+ final PreparedStatement statement = connection.prepareStatement(deleteSQL)) {
statement.setString(1, taskId);
statement.executeUpdate();
@@ -201,15 +171,55 @@ public void tryDeleteTask(final String taskId) {
public void tryAlterTaskState(final String taskId, final TaskStateEnum taskState) {
final String alterSQL = "UPDATE task SET task_state = ? WHERE task_id = ?";
- try (final Connection connection = getConnection()) {
- final PreparedStatement statement = connection.prepareStatement(alterSQL);
+ try (final Connection connection = getConnection();
+ final PreparedStatement statement = connection.prepareStatement(alterSQL)) {
statement.setInt(1, taskState.getTaskState());
statement.setString(2, taskId);
statement.executeUpdate();
LOGGER.info("successfully altered task {}", taskId);
- } catch (SQLException e) {
+ } catch (final SQLException e) {
LOGGER.warn("Failed to alter task persistence message, because {}", e.getMessage());
}
}
+
+ public void tryReportTaskProgress(final ProgressReportEvent reportEvent) {
+ if (reportEvent.getInstancesProgress() == null
+ || reportEvent.getInstancesProgress().isEmpty()) {
+ return;
+ }
+
+ final String reportSQL = "UPDATE task SET task_progress = ? WHERE task_id = ?";
+
+ try (final Connection connection = getConnection();
+ final PreparedStatement statement = connection.prepareStatement(reportSQL)) {
+ statement.setBytes(
+ 1, SerializationUtil.serializeInstances(reportEvent.getInstancesProgress()));
+ statement.setString(2, reportEvent.getTaskId());
+ statement.executeUpdate();
+ } catch (final SQLException | IOException e) {
+ LOGGER.warn("Failed to report task progress because {}", e.getMessage());
+ }
+ }
+
+ public Optional