You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@inlong.apache.org by do...@apache.org on 2021/07/05 14:08:08 UTC

[incubator-inlong] branch INLONG-613 updated: [INLONG-636] add agent directory

This is an automated email from the ASF dual-hosted git repository.

dockerzhang pushed a commit to branch INLONG-613
in repository https://gitbox.apache.org/repos/asf/incubator-inlong.git


The following commit(s) were added to refs/heads/INLONG-613 by this push:
     new a18b077  [INLONG-636] add agent directory
a18b077 is described below

commit a18b077af0632fa43168a68245cf2b72a3f2b217
Author: stingpeng <st...@tencent.com>
AuthorDate: Mon Jul 5 20:38:22 2021 +0800

    [INLONG-636] add agent directory
---
 inlong-agent/agent-common/pom.xml                  | 108 ++++++
 .../apache/inlong/agent/cache/LocalFileCache.java  |  90 +++++
 .../apache/inlong/agent/checkpoint/Checkpoint.java |  31 ++
 .../apache/inlong/agent/common/AbstractDaemon.java |  97 +++++
 .../inlong/agent/common/AgentThreadFactory.java    |  44 +++
 .../org/apache/inlong/agent/common/Service.java    |  42 +++
 .../inlong/agent/conf/AbstractConfiguration.java   | 268 ++++++++++++++
 .../inlong/agent/conf/AgentConfiguration.java      | 118 ++++++
 .../org/apache/inlong/agent/conf/JobProfile.java   |  85 +++++
 .../apache/inlong/agent/conf/ProfileFetcher.java   |  39 ++
 .../apache/inlong/agent/conf/TriggerProfile.java   |  55 +++
 .../inlong/agent/constants/AgentConstants.java     | 162 ++++++++
 .../inlong/agent/constants/CommonConstants.java    | 136 +++++++
 .../inlong/agent/constants/JobConstants.java       |  88 +++++
 .../org/apache/inlong/agent/db/BerkeleyDbImp.java  | 234 ++++++++++++
 .../java/org/apache/inlong/agent/db/CommandDb.java |  82 +++++
 .../org/apache/inlong/agent/db/CommandEntity.java  |  46 +++
 .../main/java/org/apache/inlong/agent/db/Db.java   | 109 ++++++
 .../org/apache/inlong/agent/db/JobProfileDb.java   | 182 +++++++++
 .../org/apache/inlong/agent/db/KeyValueEntity.java | 105 ++++++
 .../org/apache/inlong/agent/db/LocalProfile.java   |  71 ++++
 .../org/apache/inlong/agent/db/RocksDbImp.java     | 123 +++++++
 .../org/apache/inlong/agent/db/StateSearchKey.java |  32 ++
 .../apache/inlong/agent/db/TriggerProfileDb.java   |  79 ++++
 .../inlong/agent/message/DefaultMessage.java       |  53 +++
 .../apache/inlong/agent/message/EndMessage.java    |  39 ++
 .../apache/inlong/agent/message/ProxyMessage.java  |  74 ++++
 .../inlong/agent/metrics/AgentDynamicMBean.java    | 129 +++++++
 .../org/apache/inlong/agent/metrics/Metric.java    |  63 ++++
 .../inlong/agent/metrics/MetricException.java      |  29 ++
 .../inlong/agent/metrics/MetricSnapshot.java       |  22 ++
 .../org/apache/inlong/agent/metrics/Metrics.java   |  52 +++
 .../inlong/agent/metrics/MetricsRegister.java      | 146 ++++++++
 .../apache/inlong/agent/metrics/MutableMetric.java |  25 ++
 .../java/org/apache/inlong/agent/metrics/Tag.java  |  40 ++
 .../inlong/agent/metrics/counter/Counter.java      |  36 ++
 .../inlong/agent/metrics/counter/CounterInt.java   |  41 +++
 .../inlong/agent/metrics/counter/CounterLong.java  |  44 +++
 .../apache/inlong/agent/metrics/gauge/Gauge.java   |  50 +++
 .../inlong/agent/metrics/gauge/GaugeInt.java       |  57 +++
 .../inlong/agent/metrics/gauge/GaugeLong.java      |  57 +++
 .../inlong/agent/metrics/meta/MetricMeta.java      |  79 ++++
 .../inlong/agent/metrics/meta/MetricsMeta.java     |  61 +++
 .../org/apache/inlong/agent/plugin/Channel.java    |  54 +++
 .../org/apache/inlong/agent/plugin/Filter.java     |  30 ++
 .../org/apache/inlong/agent/plugin/Message.java    |  42 +++
 .../org/apache/inlong/agent/plugin/Reader.java     |  49 +++
 .../java/org/apache/inlong/agent/plugin/Sink.java  |  38 ++
 .../org/apache/inlong/agent/plugin/Source.java     |  35 ++
 .../java/org/apache/inlong/agent/plugin/Stage.java |  38 ++
 .../org/apache/inlong/agent/plugin/Trigger.java    |  57 +++
 .../org/apache/inlong/agent/plugin/Validator.java  |  32 ++
 .../inlong/agent/state/AbstractStateWrapper.java   |  93 +++++
 .../java/org/apache/inlong/agent/state/State.java  |  36 ++
 .../apache/inlong/agent/state/StateCallback.java   |  26 ++
 .../inlong/agent/state/StateTransferException.java |  26 ++
 .../apache/inlong/agent/utils/AgentDbUtils.java    | 242 ++++++++++++
 .../org/apache/inlong/agent/utils/AgentUtils.java  | 363 ++++++++++++++++++
 .../org/apache/inlong/agent/utils/ByteUtil.java    | 237 ++++++++++++
 .../apache/inlong/agent/AgentBaseTestsHelper.java  |  65 ++++
 .../agent/common/TestAgentThreadFactory.java       |  42 +++
 .../apache/inlong/agent/common/TestAgentUtils.java |  88 +++++
 .../inlong/agent/conf/TestConfiguration.java       |  73 ++++
 .../apache/inlong/agent/db/TestBerkeleyDBImp.java  | 135 +++++++
 .../apache/inlong/agent/metrics/TestMetrics.java   | 113 ++++++
 .../src/test/resources/agent.properties            |  25 ++
 .../agent-common/src/test/resources/job.json       |  11 +
 .../agent-common/src/test/resources/job.properties |  23 ++
 .../src/test/resources/log4j.properties            |  21 ++
 inlong-agent/agent-core/pom.xml                    |  64 ++++
 .../org/apache/inlong/agent/core/AgentMain.java    | 121 ++++++
 .../org/apache/inlong/agent/core/AgentManager.java | 206 +++++++++++
 .../apache/inlong/agent/core/conf/ConfigJetty.java | 118 ++++++
 .../inlong/agent/core/conf/ConfigServlet.java      | 107 ++++++
 .../inlong/agent/core/conf/ResponseResult.java     |  49 +++
 .../java/org/apache/inlong/agent/core/job/Job.java | 105 ++++++
 .../apache/inlong/agent/core/job/JobManager.java   | 249 +++++++++++++
 .../apache/inlong/agent/core/job/JobMetrics.java   |  47 +++
 .../apache/inlong/agent/core/job/JobWrapper.java   | 130 +++++++
 .../org/apache/inlong/agent/core/task/Task.java    |  77 ++++
 .../apache/inlong/agent/core/task/TaskManager.java | 255 +++++++++++++
 .../apache/inlong/agent/core/task/TaskMetrics.java |  55 +++
 .../agent/core/task/TaskPositionManager.java       | 148 ++++++++
 .../apache/inlong/agent/core/task/TaskWrapper.java | 198 ++++++++++
 .../inlong/agent/core/trigger/TriggerManager.java  | 248 +++++++++++++
 .../inlong/agent/core/AgentBaseTestsHelper.java    |  65 ++++
 .../apache/inlong/agent/core/TestAgentMain.java    |  48 +++
 .../inlong/agent/core/TestCompletableFuture.java   | 114 ++++++
 .../apache/inlong/agent/core/TestConfigJetty.java  | 125 +++++++
 .../apache/inlong/agent/task/TestTaskWrapper.java  | 185 ++++++++++
 .../agent-core/src/test/resources/agent.properties |  22 ++
 .../agent-core/src/test/resources/log4j.properties |  20 +
 inlong-agent/agent-plugins/pom.xml                 | 127 +++++++
 .../inlong/agent/plugin/channel/MemoryChannel.java | 111 ++++++
 .../inlong/agent/plugin/except/FileException.java  |  25 ++
 .../agent/plugin/fetcher/ManagerFetcher.java       | 409 +++++++++++++++++++++
 .../plugin/fetcher/ManagerResultFormatter.java     | 136 +++++++
 .../plugin/fetcher/constants/CommandConstants.java |  57 +++
 .../plugin/fetcher/constants/FetcherConstants.java |  54 +++
 .../agent/plugin/fetcher/dtos/CmdConfig.java       |  29 ++
 .../agent/plugin/fetcher/dtos/CommandInfoDto.java  |  31 ++
 .../plugin/fetcher/dtos/ConfirmAgentIpRequest.java |  30 ++
 .../agent/plugin/fetcher/dtos/DataConfig.java      |  43 +++
 .../agent/plugin/fetcher/dtos/JobProfileDto.java   |  70 ++++
 .../plugin/fetcher/dtos/ManagerReturnDto.java      |  28 ++
 .../agent/plugin/fetcher/dtos/TaskRequsetDto.java  |  30 ++
 .../agent/plugin/fetcher/dtos/TaskResult.java      |  43 +++
 .../agent/plugin/fetcher/enums/ManagerOpEnum.java  |  62 ++++
 .../agent/plugin/filter/DateFormatRegex.java       | 134 +++++++
 .../agent/plugin/message/PackProxyMessage.java     | 141 +++++++
 .../inlong/agent/plugin/message/SequentialID.java  |  92 +++++
 .../inlong/agent/plugin/metrics/PluginMetric.java  |  57 +++
 .../inlong/agent/plugin/sinks/ConsoleSink.java     |  51 +++
 .../inlong/agent/plugin/sinks/ProxySink.java       | 200 ++++++++++
 .../inlong/agent/plugin/sinks/PulsarSink.java      | 237 ++++++++++++
 .../inlong/agent/plugin/sinks/SenderManager.java   | 241 ++++++++++++
 .../agent/plugin/sources/DataBaseSource.java       |  67 ++++
 .../agent/plugin/sources/TextFileSource.java       |  67 ++++
 .../agent/plugin/sources/reader/SqlReader.java     | 206 +++++++++++
 .../plugin/sources/reader/TextFileReader.java      | 164 +++++++++
 .../inlong/agent/plugin/trigger/CronTrigger.java   |  53 +++
 .../agent/plugin/trigger/DirectoryTrigger.java     | 270 ++++++++++++++
 .../plugin/trigger/FormatDateDirectoryTrigger.java |  25 ++
 .../inlong/agent/plugin/trigger/PathPattern.java   | 181 +++++++++
 .../inlong/agent/plugin/utils/HttpManager.java     | 127 +++++++
 .../inlong/agent/plugin/utils/PluginUtils.java     | 156 ++++++++
 .../agent/plugin/validator/PatternValidator.java   |  34 ++
 .../agent/plugin/validator/PrefixValidator.java    |  34 ++
 .../inlong/agent/plugin/AgentBaseTestsHelper.java  |  71 ++++
 .../org/apache/inlong/agent/plugin/MiniAgent.java  |  64 ++++
 .../apache/inlong/agent/plugin/TestFileAgent.java  | 133 +++++++
 .../agent/plugin/filter/TestDateFormatRegex.java   |  79 ++++
 .../apache/inlong/agent/plugin/sinks/MockSink.java |  62 ++++
 .../agent/plugin/sources/TestTextFileReader.java   | 165 +++++++++
 .../agent/plugin/sources/TestTextFileSource.java   |  77 ++++
 .../agent/plugin/trigger/TestWatchDirTrigger.java  | 146 ++++++++
 .../inlong/agent/plugin/utils/TestUtils.java       |  81 ++++
 .../agent/plugin/validators/TestValidators.java    |  41 +++
 .../src/test/resources/fileAgent.trigger.json      |  23 ++
 .../src/test/resources/fileAgentJob.json           |  20 +
 .../src/test/resources/log4j.properties            |  21 ++
 .../agent-plugins/src/test/resources/test/1.txt    |   2 +
 .../agent-plugins/src/test/resources/test/2.txt    |   2 +
 .../agent-plugins/src/test/resources/test/a.txt    |   2 +
 inlong-agent/agent-release/package.xml             |  61 +++
 inlong-agent/agent-release/pom.xml                 |  75 ++++
 inlong-agent/bin/agent-env.sh                      |  52 +++
 inlong-agent/bin/agent.sh                          | 121 ++++++
 inlong-agent/conf/agent.properties                 | 103 ++++++
 inlong-agent/conf/log4j.properties                 |  57 +++
 inlong-agent/pom.xml                               | 278 ++++++++++++++
 pom.xml                                            |   1 +
 152 files changed, 13832 insertions(+)

diff --git a/inlong-agent/agent-common/pom.xml b/inlong-agent/agent-common/pom.xml
new file mode 100755
index 0000000..863d3d6
--- /dev/null
+++ b/inlong-agent/agent-common/pom.xml
@@ -0,0 +1,108 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+    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.
+-->
+<project xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+    xmlns="http://maven.apache.org/POM/4.0.0"
+    xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <parent>
+        <groupId>org.apache.inlong</groupId>
+        <artifactId>inlong-agent</artifactId>
+        <version>0.9.0-incubating-SNAPSHOT</version>
+    </parent>
+    <artifactId>agent-common</artifactId>
+    <modelVersion>4.0.0</modelVersion>
+    <name>Apache InLong - Agent Common</name>
+
+    <dependencies>
+        <dependency>
+            <artifactId>commons-dbutils</artifactId>
+            <groupId>commons-dbutils</groupId>
+        </dependency>
+        <dependency>
+            <artifactId>je</artifactId>
+            <groupId>com.sleepycat</groupId>
+        </dependency>
+        <dependency>
+            <artifactId>commons-lang3</artifactId>
+            <groupId>org.apache.commons</groupId>
+        </dependency>
+        <dependency>
+            <artifactId>gson</artifactId>
+            <groupId>com.google.code.gson</groupId>
+        </dependency>
+        <dependency>
+            <artifactId>slf4j-api</artifactId>
+            <groupId>org.slf4j</groupId>
+        </dependency>
+        <dependency>
+            <groupId>org.rocksdb</groupId>
+            <artifactId>rocksdbjni</artifactId>
+        </dependency>
+        <dependency>
+            <artifactId>slf4j-log4j12</artifactId>
+            <groupId>org.slf4j</groupId>
+        </dependency>
+
+        <dependency>
+            <groupId>commons-codec</groupId>
+            <artifactId>commons-codec</artifactId>
+        </dependency>
+
+        <dependency>
+            <artifactId>commons-cli</artifactId>
+            <groupId>commons-cli</groupId>
+        </dependency>
+
+        <dependency>
+            <artifactId>commons-io</artifactId>
+            <groupId>commons-io</groupId>
+        </dependency>
+
+
+
+        <dependency>
+            <artifactId>junit</artifactId>
+            <groupId>junit</groupId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.projectlombok</groupId>
+            <artifactId>lombok</artifactId>
+            <version>RELEASE</version>
+            <scope>compile</scope>
+        </dependency>
+    </dependencies>
+    <build>
+        <plugins>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-jar-plugin</artifactId>
+                <version>3.2.0</version>
+                <executions>
+                    <execution>
+                        <phase>compile</phase>
+                        <goals>
+                            <goal>test-jar</goal>
+                        </goals>
+                    </execution>
+                </executions>
+            </plugin>
+        </plugins>
+    </build>
+</project>
\ No newline at end of file
diff --git a/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/cache/LocalFileCache.java b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/cache/LocalFileCache.java
new file mode 100644
index 0000000..f6ccacc
--- /dev/null
+++ b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/cache/LocalFileCache.java
@@ -0,0 +1,90 @@
+/*
+ * 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.inlong.agent.cache;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import org.apache.commons.io.FileUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * use local file as cache.
+ */
+public class LocalFileCache {
+    private static final Logger LOGGER = LoggerFactory.getLogger(LocalFileCache.class);
+
+    private final File file;
+    private final long cacheTime;
+    private final ReadWriteLock lock = new ReentrantReadWriteLock();
+
+    /**
+     * file with cache time in ms
+     * @param file - file
+     * @param cacheTime - cache time.
+     */
+    public LocalFileCache(File file, long cacheTime) {
+        this.file = file;
+        this.cacheTime = cacheTime;
+    }
+
+    public String getCacheInfo() {
+        // lock before reading cache.
+        lock.readLock().lock();
+        String result = null;
+        try {
+            result = FileUtils.readFileToString(this.file, StandardCharsets.UTF_8);
+        } catch (IOException ex) {
+            LOGGER.error("exception on reading {}", this.file, ex);
+        } finally {
+            lock.readLock().unlock();
+        }
+        return result;
+    }
+
+    /**
+     * write string to cache
+     * @param info
+     */
+    public void writeToCache(String info) {
+        lock.writeLock().lock();
+        try {
+            FileUtils.writeStringToFile(this.file, info, StandardCharsets.UTF_8);
+        } catch (IOException ex) {
+            LOGGER.error("exception on writing {}", this.file, ex);
+        } finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    /**
+     * check whether cache is expired.
+     * @return true if cache is expired else false.
+     */
+    public boolean cacheIsExpired() {
+        lock.readLock().lock();
+        try {
+            return System.currentTimeMillis() - file.lastModified() > cacheTime;
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+}
diff --git a/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/checkpoint/Checkpoint.java b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/checkpoint/Checkpoint.java
new file mode 100644
index 0000000..bbdb3f1
--- /dev/null
+++ b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/checkpoint/Checkpoint.java
@@ -0,0 +1,31 @@
+/*
+ * 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.inlong.agent.checkpoint;
+
+public interface Checkpoint {
+
+    /**
+     * commit checkpoint.
+     */
+    void commit();
+
+    /**
+     * rollback checkpoint.
+     */
+    void rollback();
+}
diff --git a/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/common/AbstractDaemon.java b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/common/AbstractDaemon.java
new file mode 100644
index 0000000..1affb4e
--- /dev/null
+++ b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/common/AbstractDaemon.java
@@ -0,0 +1,97 @@
+/*
+ * 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.inlong.agent.common;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Providing work threads management, those threads run
+ * periodically until agent is stopped.
+ */
+public abstract class AbstractDaemon implements Service {
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(AbstractDaemon.class);
+
+    /** worker thread pool, share it **/
+    private static final ExecutorService WORKER_SERVICES =
+        new ThreadPoolExecutor(0, Integer.MAX_VALUE,
+        60L, TimeUnit.SECONDS,
+        new SynchronousQueue<Runnable>(), new AgentThreadFactory("AbstractDaemon"));
+    private final List<CompletableFuture<?>> workerFutures;
+    private boolean runnable = true;
+
+    public AbstractDaemon() {
+        this.workerFutures = new ArrayList<>();
+    }
+
+    /**
+     * Whether threads can in running state with while loop.
+     *
+     * @return - true if threads can run
+     */
+    public boolean isRunnable() {
+        return runnable;
+    }
+
+    /**
+     * Stop running threads.
+     */
+    public void stopRunningThreads() {
+        runnable = false;
+    }
+
+    /**
+     * Submit work thread to thread pool.
+     *
+     * @param worker - work thread
+     */
+    public void submitWorker(Runnable worker) {
+        CompletableFuture<?> future = CompletableFuture.runAsync(worker, WORKER_SERVICES);
+        workerFutures.add(future);
+        LOGGER.info("{} running worker number is {}", this.getClass().getName(),
+                workerFutures.size());
+    }
+
+    /**
+     * Wait for threads finish.
+     */
+    @Override
+    public void join() {
+        for (CompletableFuture<?> future : workerFutures) {
+            future.join();
+        }
+    }
+
+    /**
+     * Stop thread pool and running threads if they're in the running state.
+     */
+    public void waitForTerminate() {
+        // stop running threads.
+        if (isRunnable()) {
+            stopRunningThreads();
+        }
+    }
+}
diff --git a/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/common/AgentThreadFactory.java b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/common/AgentThreadFactory.java
new file mode 100644
index 0000000..dd48468
--- /dev/null
+++ b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/common/AgentThreadFactory.java
@@ -0,0 +1,44 @@
+/*
+ * 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.inlong.agent.common;
+
+
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class AgentThreadFactory implements ThreadFactory {
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(AgentThreadFactory.class);
+
+    private final AtomicInteger mThreadNum = new AtomicInteger(1);
+
+    private final String threadType;
+
+    public AgentThreadFactory(String threadType) {
+        this.threadType = threadType;
+    }
+
+    @Override
+    public Thread newThread(Runnable r) {
+        Thread t = new Thread(r, threadType + "-running-thread-" + mThreadNum.getAndIncrement());
+        LOGGER.debug("{} created", t.getName());
+        return t;
+    }
+}
\ No newline at end of file
diff --git a/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/common/Service.java b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/common/Service.java
new file mode 100755
index 0000000..aea5b58
--- /dev/null
+++ b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/common/Service.java
@@ -0,0 +1,42 @@
+/*
+ * 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.inlong.agent.common;
+
+/**
+ * Service lifecycle interface.
+ */
+public interface Service {
+
+    /**
+     * start service
+     * @throws Exception
+     */
+    void start() throws Exception;
+
+    /**
+     * stop service
+     * @throws Exception
+     */
+    void stop() throws Exception;
+
+    /**
+     * join and wait until getting signal
+     * @throws Exception
+     */
+    void join() throws Exception;
+}
diff --git a/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/conf/AbstractConfiguration.java b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/conf/AbstractConfiguration.java
new file mode 100644
index 0000000..1a9bb6c
--- /dev/null
+++ b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/conf/AbstractConfiguration.java
@@ -0,0 +1,268 @@
+/*
+ * 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.inlong.agent.conf;
+
+import com.google.gson.JsonArray;
+import com.google.gson.JsonElement;
+import com.google.gson.JsonObject;
+import com.google.gson.JsonParser;
+import com.google.gson.JsonPrimitive;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.Reader;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.inlong.agent.utils.AgentUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public abstract class AbstractConfiguration {
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(AbstractConfiguration.class);
+    private static final JsonParser JSON_PARSER = new JsonParser();
+
+    private final Map<String, JsonPrimitive> configStorage = new HashMap<>();
+
+    /** get config file by class loader **/
+    private ClassLoader classLoader;
+
+    public AbstractConfiguration() {
+        classLoader = Thread.currentThread().getContextClassLoader();
+        if (classLoader == null) {
+            classLoader = AgentConfiguration.class.getClassLoader();
+        }
+    }
+
+    /**
+     * Check whether all required keys exist
+     * @return true if all key exist else false.
+     */
+    public abstract boolean allRequiredKeyExist();
+
+    /**
+     * support load config file from json/properties file.
+     *
+     * @param fileName -  file name
+     * @param isJson - whether is json file
+     */
+    private void loadResource(String fileName, boolean isJson) {
+        Reader reader = null;
+        try {
+            InputStream inputStream = classLoader.getResourceAsStream(fileName);
+            if (inputStream != null) {
+                reader = new InputStreamReader(inputStream, StandardCharsets.UTF_8);
+                if (isJson) {
+                    JsonElement tmpElement = JSON_PARSER.parse(reader).getAsJsonObject();
+                    updateConfig(new HashMap<>(10), 0, tmpElement);
+                } else {
+                    Properties properties = new Properties();
+                    properties.load(reader);
+                    properties.forEach((key, value) -> configStorage.put((String) key,
+                        new JsonPrimitive((String) value)));
+                }
+            }
+        } catch (Exception ioe) {
+            LOGGER.error("error init {}", fileName, ioe);
+        } finally {
+            AgentUtils.finallyClose(reader);
+        }
+    }
+
+    /**
+     * load config from json string.
+     *
+     * @param jsonStr - json string
+     */
+    public void loadJsonStrResource(String jsonStr) {
+        JsonElement tmpElement = JSON_PARSER.parse(jsonStr);
+        updateConfig(new HashMap<>(10), 0, tmpElement);
+    }
+
+    /**
+     * load config file from CLASS_PATH. config file is json file.
+     *
+     * @param fileName - file name
+     */
+    void loadJsonResource(String fileName) {
+        loadResource(fileName, true);
+    }
+
+    void loadPropertiesResource(String fileName) {
+        loadResource(fileName, false);
+    }
+
+    /**
+     * Convert json string to map
+     *
+     * @param keyDeptPath - map
+     * @param dept - json dept
+     * @param tmpElement - json element
+     */
+    void updateConfig(HashMap<Integer, String> keyDeptPath, int dept, JsonElement tmpElement) {
+        if (tmpElement instanceof JsonObject) {
+            JsonObject tmpJsonObject = tmpElement.getAsJsonObject();
+            for (String key : tmpJsonObject.keySet()) {
+                keyDeptPath.put(dept, key);
+                updateConfig(keyDeptPath, dept + 1, tmpJsonObject.get(key));
+            }
+        } else if (tmpElement instanceof JsonArray) {
+            JsonArray tmpJsonArray = tmpElement.getAsJsonArray();
+            String lastKey = keyDeptPath.getOrDefault(dept - 1, "");
+            for (int index = 0; index < tmpJsonArray.size(); index++) {
+                keyDeptPath.put(dept - 1, lastKey + "[" + index + "]");
+                updateConfig(keyDeptPath, dept, tmpJsonArray.get(index));
+            }
+        } else if (tmpElement instanceof JsonPrimitive) {
+            List<String> builder = new ArrayList<>();
+            for (int index = 0; index < dept; index++) {
+                builder.add(keyDeptPath.getOrDefault(index, ""));
+            }
+            String keyChain = StringUtils.join(builder, ".");
+            if (!StringUtils.isBlank(keyChain)) {
+                configStorage.put(keyChain, tmpElement.getAsJsonPrimitive());
+            }
+        }
+    }
+
+    /**
+     * get int from config
+     *
+     * @param key - key
+     * @param defaultValue - default value
+     * @return value
+     */
+    public int getInt(String key, int defaultValue) {
+        JsonElement value = configStorage.get(key);
+        return value == null ? defaultValue : value.getAsInt();
+    }
+
+    /**
+     * get int from config
+     *
+     * @param key - key
+     * @return value
+     * @throws NullPointerException npe
+     */
+    public int getInt(String key) {
+        JsonElement value = configStorage.get(key);
+        if (value == null) {
+            throw new NullPointerException("null value for key " + key);
+        }
+        return value.getAsInt();
+    }
+
+    /**
+     * get long
+     *
+     * @param key - key
+     * @param defaultValue - default value
+     * @return long
+     */
+    public long getLong(String key, long defaultValue) {
+        JsonElement value = configStorage.get(key);
+        return value == null ? defaultValue : value.getAsLong();
+    }
+
+    /**
+     * get boolean
+     *
+     * @param key - key
+     * @param defaultValue - default value
+     * @return boolean
+     */
+    public boolean getBoolean(String key, boolean defaultValue) {
+        JsonElement value = configStorage.get(key);
+        return value == null ? defaultValue : value.getAsBoolean();
+    }
+
+    /**
+     * get string
+     *
+     * @param key - key
+     * @param defaultValue - default value
+     * @return string
+     */
+    public String get(String key, String defaultValue) {
+        JsonElement value = configStorage.get(key);
+        return value == null ? defaultValue : value.getAsString();
+    }
+
+    /**
+     * get string or throw npe
+     *
+     * @param key - key
+     * @return string
+     * @throws NullPointerException if value is null, throw npe
+     */
+    public String get(String key) {
+        JsonElement value = configStorage.get(key);
+        if (value == null) {
+            throw new NullPointerException("null value for key " + key);
+        }
+        return value.getAsString();
+    }
+
+    /**
+     * whether key exists
+     *
+     * @param key - key
+     * @return - true if key exists else not
+     */
+    public boolean hasKey(String key) {
+        return configStorage.containsKey(key);
+    }
+
+    /**
+     * set key/value
+     *
+     * @param key - key
+     * @param value - value
+     */
+    public void set(String key, String value) {
+        configStorage.put(key, new JsonPrimitive(value));
+    }
+
+    public void setInt(String key, int value) {
+        configStorage.put(key, new JsonPrimitive(value));
+    }
+
+    public void setLong(String key, long value) {
+        configStorage.put(key, new JsonPrimitive(value));
+    }
+
+    public void setBoolean(String key, boolean value) {
+        configStorage.put(key, new JsonPrimitive(value));
+    }
+
+    Map<String, JsonPrimitive> getConfigStorage() {
+        return configStorage;
+    }
+
+    List<String> getStorageList() {
+        List<String> result = new ArrayList<>();
+        for (Map.Entry<String, JsonPrimitive> entry : configStorage.entrySet()) {
+            result.add(entry.getKey() + "=" + entry.getValue().getAsString());
+        }
+        return result;
+    }
+}
diff --git a/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/conf/AgentConfiguration.java b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/conf/AgentConfiguration.java
new file mode 100644
index 0000000..71a566a
--- /dev/null
+++ b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/conf/AgentConfiguration.java
@@ -0,0 +1,118 @@
+/*
+ * 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.inlong.agent.conf;
+
+import java.io.File;
+import java.text.SimpleDateFormat;
+import java.util.ArrayList;
+import java.util.Date;
+import java.util.List;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import org.apache.commons.io.FileUtils;
+import org.apache.inlong.agent.constants.AgentConstants;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * agent configuration. Only one instance in the process.
+ * Basically it use properties file to store configurations.
+ */
+public class AgentConfiguration extends AbstractConfiguration {
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(AgentConfiguration.class);
+
+    private static final String DEFAULT_CONFIG_FILE = "agent.properties";
+    private static final String TMP_CONFIG_FILE = ".tmp.agent.properties";
+
+    private static final ArrayList<String> LOCAL_RESOURCES = new ArrayList<>();
+
+    private static final ReadWriteLock LOCK = new ReentrantReadWriteLock();
+
+    static {
+        LOCAL_RESOURCES.add(DEFAULT_CONFIG_FILE);
+    }
+
+    private static volatile AgentConfiguration agentConf = null;
+
+    /**
+     * load config from agent file.
+     */
+    private AgentConfiguration() {
+        for (String fileName : LOCAL_RESOURCES) {
+            super.loadPropertiesResource(fileName);
+        }
+    }
+
+    /**
+     * singleton for agent configuration.
+     * @return - static instance of AgentConfiguration
+     */
+    public static AgentConfiguration getAgentConf() {
+        if (agentConf == null) {
+            synchronized (AgentConfiguration.class) {
+                if (agentConf == null) {
+                    agentConf = new AgentConfiguration();
+                }
+            }
+        }
+        return agentConf;
+    }
+
+    private String getNextBackupFileName() {
+        SimpleDateFormat format = new SimpleDateFormat("yyyyMMddHHmmss");
+        String dateStr = format.format(new Date(System.currentTimeMillis()));
+        return DEFAULT_CONFIG_FILE + "." + dateStr;
+    }
+
+    /**
+     * flush config to local files.
+     */
+    public void flushToLocalPropertiesFile() {
+        LOCK.writeLock().lock();
+        // TODO: flush to local file as properties file.
+        try {
+            String agentConfParent = get(
+                AgentConstants.AGENT_CONF_PARENT, AgentConstants.DEFAULT_AGENT_CONF_PARENT);
+            File sourceFile = new File(agentConfParent, DEFAULT_CONFIG_FILE);
+            File targetFile = new File(agentConfParent, getNextBackupFileName());
+            File tmpFile = new File(agentConfParent, TMP_CONFIG_FILE);
+            if (sourceFile.exists()) {
+                FileUtils.copyFile(sourceFile, targetFile);
+            }
+            List<String> tmpCache = getStorageList();
+            FileUtils.writeLines(tmpFile, tmpCache);
+
+            FileUtils.copyFile(tmpFile, sourceFile);
+            boolean result = tmpFile.delete();
+            if (!result) {
+                LOGGER.warn("cannot delete file {}", tmpFile);
+            }
+        } catch (Exception ex) {
+            LOGGER.error("error while flush agent conf to local", ex);
+        } finally {
+            LOCK.writeLock().unlock();
+        }
+
+    }
+
+    @Override
+    public boolean allRequiredKeyExist() {
+        return true;
+    }
+}
diff --git a/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/conf/JobProfile.java b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/conf/JobProfile.java
new file mode 100644
index 0000000..bf3fbc4
--- /dev/null
+++ b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/conf/JobProfile.java
@@ -0,0 +1,85 @@
+/*
+ * 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.inlong.agent.conf;
+
+import com.google.gson.Gson;
+import org.apache.inlong.agent.constants.JobConstants;
+
+/**
+ * job profile which contains details describing properties of one job.
+ *
+ */
+public class JobProfile extends AbstractConfiguration {
+
+    private final Gson gson = new Gson();
+
+    /**
+     * parse json string to configuration instance。
+     *
+     * @param jsonStr
+     * @return job configuration
+     */
+    public static JobProfile parseJsonStr(String jsonStr) {
+        JobProfile conf = new JobProfile();
+        conf.loadJsonStrResource(jsonStr);
+        return conf;
+    }
+
+    /**
+     * parse properties file
+     *
+     * @param fileName - file name.
+     * @return jobConfiguration.
+     */
+    public static JobProfile parsePropertiesFile(String fileName) {
+        JobProfile conf = new JobProfile();
+        conf.loadPropertiesResource(fileName);
+        return conf;
+    }
+
+    /**
+     * pase json file.
+     * @param fileName - json file name.
+     * @return jobConfiguration.
+     */
+    public static JobProfile parseJsonFile(String fileName) {
+        JobProfile conf = new JobProfile();
+        conf.loadJsonResource(fileName);
+        return conf;
+    }
+
+    /**
+     * check whether required keys exists.
+     *
+     * @return return true if all required keys exists else false.
+     */
+    @Override
+    public boolean allRequiredKeyExist() {
+        return hasKey(JobConstants.JOB_ID) && hasKey(JobConstants.JOB_SOURCE)
+                && hasKey(JobConstants.JOB_SINK) && hasKey(JobConstants.JOB_CHANNEL) && hasKey(
+            JobConstants.JOB_NAME);
+    }
+
+    public String toJsonStr() {
+        return gson.toJson(getConfigStorage());
+    }
+
+    public String getInstanceId() {
+        return get(JobConstants.JOB_INSTANCE_ID);
+    }
+}
diff --git a/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/conf/ProfileFetcher.java b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/conf/ProfileFetcher.java
new file mode 100755
index 0000000..59f3111
--- /dev/null
+++ b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/conf/ProfileFetcher.java
@@ -0,0 +1,39 @@
+/*
+ * 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.inlong.agent.conf;
+
+import java.util.List;
+import org.apache.inlong.agent.common.Service;
+
+/**
+ * fetch profile from other system, communicate with json format string
+ */
+public interface ProfileFetcher extends Service {
+
+    /**
+     * get job profiles
+     * @return - job profile list
+     */
+    List<JobProfile> getJobProfiles();
+
+    /**
+     * get trigger profiles
+     * @return - trigger profile lisy
+     */
+    List<TriggerProfile>  getTriggerProfiles();
+}
diff --git a/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/conf/TriggerProfile.java b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/conf/TriggerProfile.java
new file mode 100755
index 0000000..4a0e48a
--- /dev/null
+++ b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/conf/TriggerProfile.java
@@ -0,0 +1,55 @@
+/*
+ * 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.inlong.agent.conf;
+
+import org.apache.inlong.agent.constants.JobConstants;
+
+/**
+ * profile used in trigger. Trigger profile is a special job profile
+ */
+public class TriggerProfile extends JobProfile {
+
+    @Override
+    public boolean allRequiredKeyExist() {
+        return hasKey(JobConstants.JOB_TRIGGER) && super.allRequiredKeyExist();
+    }
+
+    public static TriggerProfile parseJsonStr(String jsonStr) {
+        TriggerProfile conf = new TriggerProfile();
+        conf.loadJsonStrResource(jsonStr);
+        return conf;
+    }
+
+    public String getTriggerId() {
+        return get(JobConstants.JOB_ID);
+    }
+
+    public static TriggerProfile parseJobProfile(JobProfile jobProfile) {
+        TriggerProfile conf = new TriggerProfile();
+        conf.loadJsonStrResource(jobProfile.toJsonStr());
+        return conf;
+    }
+
+    public Integer getOpType() {
+        return getInt(JobConstants.JOB_OP);
+    }
+
+    public String getDeliveryTime() {
+        return get(JobConstants.JOB_DELIVERY_TIME);
+    }
+}
diff --git a/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/constants/AgentConstants.java b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/constants/AgentConstants.java
new file mode 100755
index 0000000..22bfec1
--- /dev/null
+++ b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/constants/AgentConstants.java
@@ -0,0 +1,162 @@
+/*
+ * 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.inlong.agent.constants;
+
+import org.apache.inlong.agent.utils.AgentUtils;
+
+public class AgentConstants {
+
+    public static final String AGENT_HOME = "agent.home";
+    public static final String DEFAULT_AGENT_HOME = System.getProperty("agent.home");
+
+    public static final String AGENT_LOCAL_CACHE = "agent.local.cache";
+    public static final String DEFAULT_AGENT_LOCAL_CACHE = ".local";
+
+    public static final String AGENT_LOCAL_CACHE_TIMEOUT = "agent.local.cache.timeout";
+    /** cache timeout in minutes. **/
+    public static final int DEFAULT_AGENT_LOCAL_CACHE_TIMEOUT = 30;
+
+    public static final String AGENT_LOCAL_STORE_PATH = "agent.localStore.path";
+    public static final String DEFAULT_AGENT_LOCAL_STORE_PATH = ".bdb";
+
+    public static final String AGENT_ROCKS_DB_PATH = "agent.rocks.db.path";
+    public static final String DEFAULT_AGENT_ROCKS_DB_PATH = ".rocksdb";
+
+    public static final String AGENT_UNIQ_ID = "agent.uniq.id";
+    // default use local ip as uniq id for agent.
+    public static final String DEFAULT_AGENT_UNIQ_ID = AgentUtils.getLocalIp();
+
+    public static final String AGENT_DB_INSTANCE_NAME = "agent.db.instance.name";
+    public static final String DEFAULT_AGENT_DB_INSTANCE_NAME = "agent";
+
+    public static final String AGENT_DB_CLASSNAME = "agent.db.classname";
+    public static final String DEFAULT_AGENT_DB_CLASSNAME = "org.apache.inlong.agent.db.BerkeleyDbImp";
+
+    // default is empty.
+    public static final String AGENT_FETCHER_CLASSNAME = "agent.fetcher.classname";
+
+    public static final String AGENT_CONF_PARENT = "agent.conf.parent";
+    public static final String DEFAULT_AGENT_CONF_PARENT = "conf";
+
+    public static final String AGENT_LOCAL_STORE_READONLY = "agent.localStore.readonly";
+    public static final boolean DEFAULT_AGENT_LOCAL_STORE_READONLY = false;
+
+    public static final String AGENT_HTTP_PORT = "agent.http.port";
+    public static final int DEFAULT_AGENT_HTTP_PORT = 8008;
+
+    public static final String AGENT_ENABLE_HTTP = "agent.http.enable";
+    public static final boolean DEFAULT_AGENT_ENABLE_HTTP = false;
+
+    public static final String TRIGGER_FETCH_INTERVAL = "trigger.fetch.interval";
+    public static final int DEFAULT_TRIGGER_FETCH_INTERVAL = 1;
+
+    public static final String TRIGGER_MAX_RUNNING_NUM = "trigger.max.running.num";
+    public static final int DEFAULT_TRIGGER_MAX_RUNNING_NUM = 4096;
+
+    public static final String AGENT_LOCAL_STORE_TRANSACTIONAL = "agent.localStore.transactional";
+    public static final boolean DEFAULT_AGENT_LOCAL_STORE_TRANSACTIONAL = true;
+
+    public static final String AGENT_LOCAL_STORE_LOCK_TIMEOUT = "agent.localStore.lockTimeout";
+    public static final int DEFAULT_AGENT_LOCAL_STORE_LOCK_TIMEOUT = 10000;
+
+    public static final String AGENT_LOCAL_STORE_NO_SYNC_VOID = "agent.localStore.noSyncVoid";
+    public static final boolean DEFAULT_AGENT_LOCAL_STORE_NO_SYNC_VOID = false;
+
+    public static final String AGENT_LOCAL_STORE_WRITE_NO_SYNC_VOID =
+            "agent.localStore.WriteNoSyncVoid";
+    public static final boolean DEFAULT_AGENT_LOCAL_STORE_WRITE_NO_SYNC_VOID = false;
+
+    public static final String AGENT_FETCH_CENTER_INTERVAL_SECONDS = "agent.fetchCenter.interval";
+    public static final int DEFAULT_AGENT_FETCH_CENTER_INTERVAL_SECONDS = 5;
+
+    public static final String AGENT_TRIGGER_CHECK_INTERVAL_SECONDS = "agent.trigger.check.interval";
+    public static final int DEFAULT_AGENT_TRIGGER_CHECK_INTERVAL_SECONDS = 1;
+
+    public static final String THREAD_POOL_AWAIT_TIME = "thread.pool.await.time";
+    // time in ms
+    public static final long DEFAULT_THREAD_POOL_AWAIT_TIME = 300;
+
+    public static final String JOB_THREAD_PENDING_MAX = "job.thread.pending.max";
+    public static final int DEFAULT_JOB_THREAD_PENDING_MAX = 40;
+
+    public static final String JOB_THREAD_RUNNING_CORE = "job.thread.running.core";
+    public static final int DEFAULT_JOB_THREAD_RUNNING_CORE = 4;
+
+
+    public static final String JOB_MONITOR_INTERVAL = "job.monitor.interval";
+    public static final int DEFAULT_JOB_MONITOR_INTERVAL = 5;
+
+    public static final String JOB_THREAD_RUNNING_MAX = "job.thread.running.max";
+    public static final int DEFAULT_JOB_THREAD_RUNNING_MAX = 20;
+
+    public static final String JOB_RUNNING_THREAD_KEEP_ALIVE = "job.running.thread.keepAlive";
+    public static final long DEFAULT_JOB_RUNNING_THREAD_KEEP_ALIVE = 60L;
+
+    public static final String JOB_FINISH_CHECK_INTERVAL = "job.finish.checkInterval";
+    public static final long DEFAULT_JOB_FINISH_CHECK_INTERVAL = 6L;
+
+    public static final String TASK_PENDING_MAX = "task.pending.max";
+    public static final int DEFAULT_TASK_PENDING_MAX = 100;
+
+    public static final String TASK_RUNNING_THREAD_CORE_SIZE = "task.running.thread.coreSize";
+    public static final int DEFAULT_TASK_RUNNING_THREAD_CORE_SIZE = 4;
+
+    public static final String TASK_RUNNING_THREAD_MAX_SIZE = "task.running.thread.maxSize";
+    public static final int DEFAULT_TASK_RUNNING_THREAD_MAX_SIZE =
+            Runtime.getRuntime().availableProcessors() * 2;
+
+    public static final String TASK_RUNNING_THREAD_KEEP_ALIVE = "task.running.thread.keepAlive";
+    public static final long DEFAULT_TASK_RUNNING_THREAD_KEEP_ALIVE = 60L;
+
+    public static final String TASK_RETRY_MAX_CAPACITY = "task.retry.maxCapacity";
+    public static final int DEFAULT_TASK_RETRY_MAX_CAPACITY = 10000;
+
+    public static final String TASK_MONITOR_INTERVAL = "task.monitor.interval";
+    public static final int DEFAULT_TASK_MONITOR_INTERVAL = 6;
+
+    public static final String TASK_RETRY_SUBMIT_WAIT_SECONDS = "task.retry.submit.waitSeconds";
+    public static final int DEFAULT_TASK_RETRY_SUBMIT_WAIT_SECONDS = 5;
+
+    public static final String TASK_MAX_RETRY_TIME = "task.maxRetry.time";
+    public static final int DEFAULT_TASK_MAX_RETRY_TIME = 3;
+
+    public static final String TASK_PUSH_MAX_SECOND = "task.push.maxSecond";
+    public static final int DEFAULT_TASK_PUSH_MAX_SECOND = 2;
+
+    public static final String TASK_PULL_MAX_SECOND = "task.pull.maxSecond";
+    public static final int DEFAULT_TASK_PULL_MAX_SECOND = 2;
+
+    public static final String CHANNEL_MEMORY_CAPACITY = "channel.memory.capacity";
+    public static final int DEFAULT_CHANNEL_MEMORY_CAPACITY = 10000;
+
+    public static final String TRIGGER_CHECK_INTERVAL = "trigger.check.interval";
+    public static final int DEFAULT_TRIGGER_CHECK_INTERVAL = 2;
+
+    public static final String WORKER_POOL_AWAIT_TIME = "worker.pool.await.time";
+    public static final long DEFAULT_WORKER_POOL_AWAIT_TIME = 10;
+
+    public static final String JOB_DB_CACHE_TIME = "job.db.cache.time";
+    // cache for 3 days.
+    public static final long DEFAULT_JOB_DB_CACHE_TIME = 3 * 24 * 60 * 60 * 1000;
+
+    public static final String JOB_DB_CACHE_CHECK_INTERVAL = "job.db.cache.check.interval";
+    public static final int DEFAULT_JOB_DB_CACHE_CHECK_INTERVAL = 60 * 60;
+
+    public static final String AGENT_LOCAL_IP = "agent.local.ip";
+
+}
diff --git a/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/constants/CommonConstants.java b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/constants/CommonConstants.java
new file mode 100644
index 0000000..7e3fc6d
--- /dev/null
+++ b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/constants/CommonConstants.java
@@ -0,0 +1,136 @@
+/*
+ * 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.inlong.agent.constants;
+
+import org.apache.inlong.agent.utils.AgentUtils;
+
+public class CommonConstants {
+
+    public static final String PROXY_NET_TAG = "proxy.net.tag";
+    public static final String DEFAULT_PROXY_NET_TAG = "";
+
+    public static final String PROXY_BID = "proxy.bid";
+    public static final String POSITION_SUFFIX = ".position";
+
+    public static final String PROXY_LOCAL_HOST = "proxy.localHost";
+    public static final String DEFAULT_PROXY_LOCALHOST = AgentUtils.getLocalIp();
+
+    public static final String PROXY_IS_LOCAL_VISIT = "proxy.isLocalVisit";
+    public static final boolean DEFAULT_PROXY_IS_LOCAL_VISIT = true;
+
+    public static final String PROXY_TOTAL_ASYNC_PROXY_SIZE = "proxy.total.async.proxy.size";
+    public static final int DEFAULT_PROXY_TOTAL_ASYNC_PROXY_SIZE = 200 * 1024 * 1024;
+
+    public static final String PROXY_ALIVE_CONNECTION_NUM = "proxy.alive.connection.num";
+    public static final int DEFAULT_PROXY_ALIVE_CONNECTION_NUM = 10;
+
+    public static final String PROXY_MSG_TYPE = "proxy.msgType";
+    public static final int DEFAULT_PROXY_MSG_TYPE = 7;
+
+    public static final String PROXY_IS_COMPRESS = "proxy.is.compress";
+    public static final boolean DEFAULT_PROXY_IS_COMPRESS = true;
+
+    public static final String PROXY_MAX_SENDER_PER_BID = "proxy.max.sender.per.pid";
+    public static final int DEFAULT_PROXY_MAX_SENDER_PER_PID = 10;
+
+    // max size of message list
+    public static final String PROXY_PACKAGE_MAX_SIZE = "proxy.package.maxSize";
+    // max size of single batch in bytes, default is 200KB.
+    public static final int DEFAULT_PROXY_PACKAGE_MAX_SIZE = 200000;
+
+    public static final String PROXY_TID_QUEUE_MAX_NUMBER = "proxy.tid.queue.maxNumber";
+    public static final int DEFAULT_PROXY_TID_QUEUE_MAX_NUMBER = 10000;
+
+    public static final String PROXY_PACKAGE_MAX_TIMEOUT_MS = "proxy.package.maxTimeout.ms";
+    public static final int DEFAULT_PROXY_PACKAGE_MAX_TIMEOUT_MS = 4 * 1000;
+
+    public static final String PROXY_BATCH_FLUSH_INTERVAL = "proxy.batch.flush.interval";
+    public static final int DEFAULT_PROXY_BATCH_FLUSH_INTERVAL = 2 * 1000;
+
+    public static final String PROXY_SENDER_MAX_TIMEOUT = "proxy.sender.maxTimeout";
+    // max timeout in seconds.
+    public static final int DEFAULT_PROXY_SENDER_MAX_TIMEOUT = 20;
+
+    public static final String PROXY_SENDER_MAX_RETRY = "proxy.sender.maxRetry";
+    public static final int DEFAULT_PROXY_SENDER_MAX_RETRY = 5;
+
+    public static final String PROXY_IS_FILE = "proxy.isFile";
+    public static final boolean DEFAULT_IS_FILE = false;
+
+    public static final String PROXY_RETRY_SLEEP = "proxy.retry.sleep";
+    public static final long DEFAULT_PROXY_RETRY_SLEEP = 500;
+
+    public static final String PROXY_KEY_BID = "bid";
+    public static final String PROXY_KEY_ID = "id";
+    public static final String PROXY_KEY_AGENT_IP = "agentip";
+    public static final String PROXY_OCEANUS_F = "f";
+    public static final String PROXY_OCEANUS_BL = "bl";
+
+    // config for pulsar
+    // pulsar host port like http://host1:port1
+    public static final String PULSAR_SERVERS = "pulsar.servers";
+    // pulsar topic name
+    public static final String PULSAR_TOPIC = "pulsar.topic";
+    // whether async sending data
+    public static final String PULSAR_PRODUCER_ASYNC = "pulsar.producer.async";
+    public static final boolean DEFAULT_PULSAR_PRODUCER_ASYNC = true;
+
+    public static final String PULSAR_PRODUCER_MAX_PENDING_COUNT = "pulsar.producer.maxPending.count";
+    public static final int DEFAULT_PULSAR_PRODUCER_MAX_PENDING_COUNT = 10000;
+
+    public static final String PULSAR_PRODUCER_THREAD_NUM = "pulsar.producer.thread.num";
+    public static final int DEFAULT_PULSAR_PRODUCER_THREAD_NUM = 1;
+
+    public static final String PULSAR_PRODUCER_ENABLE_BATCH = "pulsar.producer.enable.batch";
+    public static final boolean DEFAULT_PULSAR_PRODUCER_ENABLE_BATCH = true;
+
+    public static final String PULSAR_SINK_POLL_TIMEOUT = "pulsar.sink.poll.timeout";
+    // time in ms
+    public static final long DEFAULT_PULSAR_SINK_POLL_TIMEOUT = 1000;
+
+    public static final String PULSAR_SINK_CACHE_CAPACITY = "pulsar.sink.cache.capacity";
+    public static final int DEFAULT_PULSAR_SINK_CACHE_CAPACITY = 100000;
+
+    public static final String PULSAR_PRODUCER_COMPRESS_TYPE = "pulsar.producer.compress.type";
+    public static final String DEFAULT_PULSAR_PRODUCER_COMPRESS_TYPE = "snappy";
+
+    public static final String PULSAR_PRODUCER_BATCH_MAXSIZE = "pulsar.producer.batch.maxsize";
+    public static final int DEFAULT_PULSAR_PRODUCER_BATCH_MAXSIZE = 1024 * 1024;
+
+    public static final String PULSAR_PRODUCER_BATCH_MAXCOUNT = "pulsar.producer.batch.maxcount";
+    public static final int DEFAULT_PULSAR_PRODUCER_BATCH_MAXCOUNT = 1000;
+
+    public static final String PULSAR_PRODUCER_BLOCK_QUEUE = "pulsar.producer.block.queue";
+    public static final boolean DEFAULT_PULSAR_PRODUCER_BLOCK_QUEUE = true;
+
+
+    public static final String FILE_MAX_NUM = "file.max.num";
+    public static final int DEFAULT_FILE_MAX_NUM = 4096;
+
+    public static final String TRIGGER_ID_PREFIX = "trigger_";
+
+
+    public static final String COMMAND_STORE_INSTANCE_NAME = "commandStore";
+
+
+    public static final String AGENT_OS_NAME = "os.name";
+    public static final String AGENT_NIX_OS = "nix";
+    public static final String AGENT_NUX_OS = "nux";
+    public static final String AGENT_COLON = ":";
+
+}
diff --git a/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/constants/JobConstants.java b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/constants/JobConstants.java
new file mode 100755
index 0000000..b131f3a
--- /dev/null
+++ b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/constants/JobConstants.java
@@ -0,0 +1,88 @@
+/*
+ * 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.inlong.agent.constants;
+
+
+/**
+ * Basic config for a single job
+ */
+public class JobConstants extends CommonConstants {
+
+    // job id
+    public static final String JOB_ID = "job.id";
+    public static final String JOB_INSTANCE_ID = "job.instance.id";
+    public static final String JOB_IP = "job.ip";
+    public static final String JOB_RETRY = "job.retry";
+
+
+    public static final String JOB_SOURCE = "job.source";
+    public static final String JOB_SINK = "job.sink";
+    public static final String JOB_CHANNEL = "job.channel";
+    public static final String JOB_TRIGGER = "job.trigger";
+    public static final String JOB_NAME = "job.name";
+    public static final String JOB_LINE_FILTER_PATTERN = "job.pattern";
+    public static final String DEFAULT_JOB_NAME = "default";
+    public static final String JOB_DESCRIPTION = "job.description";
+    public static final String DEFAULT_JOB_DESCRIPTION = "default job description";
+    public static final String DEFAULT_JOB_LINE_FILTER = "";
+
+    // job type, delete/add
+    public static final String JOB_TYPE = "job.type";
+
+    public static final String JOB_CHECKPOINT = "job.checkpoint";
+
+    // offset for time
+    public static final String JOB_FILE_TIME_OFFSET = "job.file.time.offset";
+
+    public static final String DEFAULT_JOB_FILE_TIME_OFFSET = "0d";
+
+    public static final String JOB_FILE_MAX_WAIT = "job.file.max.wait";
+    // time in min
+    public static final int DEFAULT_JOB_FILE_MAX_WAIT = 1;
+
+    public static final String JOB_DIR_FILTER_PATTERN = "job.dir.pattern";
+
+    public static final String JOB_DIR_FILTER_PATH = "job.dir.path";
+
+    public static final String JOB_ID_PREFIX = "job_";
+
+    public static final String JOB_STORE_TIME = "job.store.time";
+
+    public static final String JOB_OP = "job.op";
+
+    public static final String TRIGGER_ONLY_ONE_JOB = "job.standalone";
+
+    // field splitter
+    public static final String JOB_FIELD_SPLITTER = "job.splitter";
+
+    public static final String JOB_ADDITION_STR = "job.additionStr";
+
+    // job delivery time
+    public static final String JOB_DELIVERY_TIME = "job.deliveryTime";
+
+    // job time reading file
+    public static final String JOB_DATA_TIME = "job.dataTime";
+
+    public static final String JOB_CYCLE_UNIT = "job.cycleUnit";
+
+    /**
+     * when job is retried, the retry time should be provided
+     */
+    public static final String JOB_RETRY_TIME = "job.retryTime";
+
+}
diff --git a/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/db/BerkeleyDbImp.java b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/db/BerkeleyDbImp.java
new file mode 100644
index 0000000..407ac89
--- /dev/null
+++ b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/db/BerkeleyDbImp.java
@@ -0,0 +1,234 @@
+/*
+ * 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.inlong.agent.db;
+
+import static java.util.Objects.requireNonNull;
+
+import com.sleepycat.je.Environment;
+import com.sleepycat.je.EnvironmentConfig;
+import com.sleepycat.persist.EntityCursor;
+import com.sleepycat.persist.EntityStore;
+import com.sleepycat.persist.PrimaryIndex;
+import com.sleepycat.persist.SecondaryIndex;
+import com.sleepycat.persist.StoreConfig;
+import java.io.File;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import org.apache.inlong.agent.conf.AgentConfiguration;
+import org.apache.inlong.agent.constants.AgentConstants;
+import org.apache.inlong.agent.constants.CommonConstants;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * DB implement based on berkeley db.
+ */
+public class BerkeleyDbImp implements Db {
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(BerkeleyDbImp.class);
+
+    private final EntityStore jobStore;
+    private final EntityStore commandStore;
+    private final PrimaryIndex<String, KeyValueEntity> primaryIndex;
+    private final SecondaryIndex<StateSearchKey, String, KeyValueEntity> secondaryIndex;
+    private final PrimaryIndex<String, CommandEntity> commandPrimaryIndex;
+    private final SecondaryIndex<String, String, KeyValueEntity> fileNameSecondaryIndex;
+    private final SecondaryIndex<Boolean, String, CommandEntity> commandSecondaryIndex;
+
+    private final AgentConfiguration agentConf;
+
+    public BerkeleyDbImp() {
+        this.agentConf = AgentConfiguration.getAgentConf();
+        StoreConfig storeConfig = initStoreConfig();
+        Environment environment = initEnv();
+        String instanceName = agentConf.get(
+            AgentConstants.AGENT_DB_INSTANCE_NAME, AgentConstants.DEFAULT_AGENT_DB_INSTANCE_NAME);
+        this.jobStore = new EntityStore(environment, instanceName, storeConfig);
+        this.commandStore = new EntityStore(environment, CommonConstants.COMMAND_STORE_INSTANCE_NAME, storeConfig);
+        commandPrimaryIndex = this.commandStore.getPrimaryIndex(String.class, CommandEntity.class);
+        commandSecondaryIndex = commandStore.getSecondaryIndex(
+            commandPrimaryIndex, Boolean.class, "isAcked");
+        primaryIndex = this.jobStore.getPrimaryIndex(String.class, KeyValueEntity.class);
+        secondaryIndex = this.jobStore.getSecondaryIndex(primaryIndex, StateSearchKey.class,
+                "stateSearchKey");
+        fileNameSecondaryIndex = this.jobStore.getSecondaryIndex(primaryIndex,
+            String.class, "fileName");
+    }
+
+    /**
+     * init store by config
+     *
+     * @return store config
+     */
+    private StoreConfig initStoreConfig() {
+        return new StoreConfig()
+                .setReadOnly(agentConf.getBoolean(
+                        AgentConstants.AGENT_LOCAL_STORE_READONLY,
+                    AgentConstants.DEFAULT_AGENT_LOCAL_STORE_READONLY))
+                .setAllowCreate(!agentConf.getBoolean(
+                        AgentConstants.AGENT_LOCAL_STORE_READONLY,
+                    AgentConstants.DEFAULT_AGENT_LOCAL_STORE_READONLY))
+                .setTransactional(agentConf.getBoolean(
+                        AgentConstants.AGENT_LOCAL_STORE_TRANSACTIONAL,
+                    AgentConstants.DEFAULT_AGENT_LOCAL_STORE_TRANSACTIONAL));
+    }
+
+    /**
+     * init local bdb path and get it.
+     * @return local path.
+     */
+    private File tryToInitAndGetPath() {
+        String storePath = agentConf.get(
+            AgentConstants.AGENT_LOCAL_STORE_PATH, AgentConstants.DEFAULT_AGENT_LOCAL_STORE_PATH);
+        String parentPath = agentConf.get(
+            AgentConstants.AGENT_HOME, AgentConstants.DEFAULT_AGENT_HOME);
+        File finalPath = new File(parentPath, storePath);
+        try {
+            boolean result = finalPath.mkdirs();
+            LOGGER.info("try to create local path {}, result is {}", finalPath, result);
+        } catch (Exception ex) {
+            throw new RuntimeException(ex);
+        }
+        return finalPath;
+    }
+
+    /**
+     * init env by config
+     *
+     * @return env config
+     */
+    private Environment initEnv() {
+        EnvironmentConfig envConfig = new EnvironmentConfig()
+                .setReadOnly(agentConf.getBoolean(
+                        AgentConstants.AGENT_LOCAL_STORE_READONLY, AgentConstants.DEFAULT_AGENT_LOCAL_STORE_READONLY))
+                .setAllowCreate(!agentConf.getBoolean(
+                        AgentConstants.AGENT_LOCAL_STORE_READONLY, AgentConstants.DEFAULT_AGENT_LOCAL_STORE_READONLY))
+                .setTransactional(agentConf.getBoolean(
+                        AgentConstants.AGENT_LOCAL_STORE_TRANSACTIONAL,
+                    AgentConstants.DEFAULT_AGENT_LOCAL_STORE_TRANSACTIONAL))
+                .setLockTimeout(
+                        agentConf.getInt(AgentConstants.AGENT_LOCAL_STORE_LOCK_TIMEOUT,
+                                AgentConstants.DEFAULT_AGENT_LOCAL_STORE_LOCK_TIMEOUT),
+                        TimeUnit.MILLISECONDS);
+        envConfig.setTxnNoSyncVoid(agentConf.getBoolean(
+            AgentConstants.AGENT_LOCAL_STORE_NO_SYNC_VOID,
+                AgentConstants.DEFAULT_AGENT_LOCAL_STORE_NO_SYNC_VOID));
+        envConfig.setTxnWriteNoSyncVoid(agentConf.getBoolean(
+            AgentConstants.AGENT_LOCAL_STORE_WRITE_NO_SYNC_VOID,
+                AgentConstants.DEFAULT_AGENT_LOCAL_STORE_WRITE_NO_SYNC_VOID));
+        return new Environment(tryToInitAndGetPath(), envConfig);
+    }
+
+    @Override
+    public KeyValueEntity get(String key) {
+        requireNonNull(key);
+        return primaryIndex.get(key);
+    }
+
+
+    @Override
+    public CommandEntity getCommand(String commandId) {
+        requireNonNull(commandId);
+        return commandPrimaryIndex.get(commandId);
+    }
+
+
+    @Override
+    public CommandEntity putCommand(CommandEntity entity) {
+        requireNonNull(entity);
+        return commandPrimaryIndex.put(entity);
+    }
+
+    @Override
+    public void set(KeyValueEntity entity) {
+        requireNonNull(entity);
+        primaryIndex.put(entity);
+    }
+
+    @Override
+    public KeyValueEntity put(KeyValueEntity entity) {
+        requireNonNull(entity);
+        return primaryIndex.put(entity);
+    }
+
+    @Override
+    public KeyValueEntity remove(String key) {
+        requireNonNull(key);
+        KeyValueEntity entity = primaryIndex.get(key);
+        primaryIndex.delete(key);
+        return entity;
+    }
+
+    @Override
+    public List<KeyValueEntity> search(StateSearchKey searchKey) {
+        requireNonNull(searchKey);
+        List<KeyValueEntity> ret = new ArrayList<>();
+        try (EntityCursor<KeyValueEntity> children = secondaryIndex.subIndex(searchKey)
+                .entities()) {
+            for (KeyValueEntity entity : children) {
+                ret.add(entity);
+            }
+        }
+        return ret;
+    }
+
+    @Override
+    public List<CommandEntity> searchCommands(boolean isAcked) {
+        requireNonNull(isAcked);
+        List<CommandEntity> ret = new ArrayList<>();
+        try (EntityCursor<CommandEntity> children = commandSecondaryIndex.subIndex(isAcked)
+            .entities()) {
+            for (CommandEntity entity : children) {
+                ret.add(entity);
+            }
+        }
+        return ret;
+    }
+
+    @Override
+    public KeyValueEntity searchOne(StateSearchKey searchKey) {
+        requireNonNull(searchKey);
+        return secondaryIndex.get(searchKey);
+    }
+
+    @Override
+    public KeyValueEntity searchOne(String fileName) {
+        requireNonNull(fileName);
+        return fileNameSecondaryIndex.get(fileName);
+    }
+
+    @Override
+    public List<KeyValueEntity> findAll(String prefix) {
+        requireNonNull(prefix);
+        List<KeyValueEntity> ret = new ArrayList<>();
+        try (EntityCursor<KeyValueEntity> children = primaryIndex.entities()) {
+            for (KeyValueEntity entity : children) {
+                if (entity.getKey().startsWith(prefix)) {
+                    ret.add(entity);
+                }
+            }
+        }
+        return ret;
+    }
+
+    @Override
+    public void close() {
+        jobStore.close();
+    }
+}
diff --git a/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/db/CommandDb.java b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/db/CommandDb.java
new file mode 100644
index 0000000..c343e69
--- /dev/null
+++ b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/db/CommandDb.java
@@ -0,0 +1,82 @@
+/*
+ * 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.inlong.agent.db;
+
+import java.util.List;
+import org.apache.inlong.agent.conf.TriggerProfile;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class CommandDb {
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(CommandDb.class);
+    public static final int MANAGER_SUCCESS_CODE = 0;
+    public static final int MANAGER_FAIL_CODE = 1;
+    private final Db db;
+
+    public CommandDb(Db db) {
+        this.db = db;
+    }
+
+    /**
+     * store manager command to db
+     * @param commandEntity
+     */
+    public void storeCommand(CommandEntity commandEntity) {
+        db.putCommand(commandEntity);
+    }
+
+    /**
+     * get those commands not ack to manager
+     * @return
+     */
+    public List<CommandEntity> getUnackedCommands() {
+        return db.searchCommands(false);
+    }
+
+
+    /**
+     * save normal command result for trigger
+     * @param profile
+     * @param success
+     */
+    public void saveNormalCmds(TriggerProfile profile, boolean success) {
+        CommandEntity entity = new CommandEntity();
+        entity.setId(CommandEntity.generateCommanid(profile.getTriggerId(), profile.getOpType()));
+        entity.setTaskId(profile.getTriggerId());
+        entity.setDeliveryTime(profile.getDeliveryTime());
+        entity.setCommandResult(success ? MANAGER_SUCCESS_CODE : MANAGER_FAIL_CODE);
+        entity.setAcked(false);
+        storeCommand(entity);
+    }
+
+    /**
+     * save special command result for trigger (retry\makeup\check)
+     * @param id
+     * @param taskId
+     * @param success
+     */
+    public void saveSpecialCmds(Integer id, Integer taskId, boolean success) {
+        CommandEntity entity = new CommandEntity();
+        entity.setId(String.valueOf(id));
+        entity.setTaskId(String.valueOf(taskId));
+        entity.setAcked(false);
+        entity.setCommandResult(success ? MANAGER_SUCCESS_CODE : MANAGER_FAIL_CODE);
+        storeCommand(entity);
+    }
+}
diff --git a/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/db/CommandEntity.java b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/db/CommandEntity.java
new file mode 100644
index 0000000..68fac8b
--- /dev/null
+++ b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/db/CommandEntity.java
@@ -0,0 +1,46 @@
+/*
+ * 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.inlong.agent.db;
+
+import com.sleepycat.persist.model.Entity;
+import com.sleepycat.persist.model.PrimaryKey;
+import com.sleepycat.persist.model.Relationship;
+import com.sleepycat.persist.model.SecondaryKey;
+import lombok.AllArgsConstructor;
+import lombok.Data;
+import lombok.NoArgsConstructor;
+
+
+@Entity(version = 1)
+@Data
+@AllArgsConstructor
+@NoArgsConstructor
+public class CommandEntity {
+    @PrimaryKey
+    private String id;
+    private int commandResult;
+    @SecondaryKey(relate = Relationship.MANY_TO_ONE)
+    private boolean isAcked;
+    private String taskId;
+    private String deliveryTime;
+
+
+    public static String generateCommanid(String taskId, int opType) {
+        return taskId + opType;
+    }
+}
diff --git a/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/db/Db.java b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/db/Db.java
new file mode 100644
index 0000000..81023db
--- /dev/null
+++ b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/db/Db.java
@@ -0,0 +1,109 @@
+/*
+ * 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.inlong.agent.db;
+
+import java.io.Closeable;
+import java.util.List;
+import javax.management.openmbean.KeyAlreadyExistsException;
+
+/**
+ * local storage for key/value.
+ */
+public interface Db extends Closeable {
+
+
+    abstract KeyValueEntity get(String key);
+
+    /**
+     * get command by command id
+     * @param commandId
+     * @return
+     */
+    CommandEntity getCommand(String commandId);
+
+    /**
+     * put command entity in db
+     * @param entity
+     * @return
+     */
+    CommandEntity putCommand(CommandEntity entity);
+
+    /**
+     * store keyValue, if key has exists, throw exception.
+     *
+     * @param entity - key/value
+     * @throws NullPointerException key should not be null
+     * @throws KeyAlreadyExistsException key already exists
+     */
+    void set(KeyValueEntity entity);
+
+    /**
+     * store keyValue, if key has exists, overwrite it.
+     *
+     * @param entity - key/value
+     * @return null or old value which is overwritten.
+     * @throws NullPointerException key should not be null.
+     */
+    KeyValueEntity put(KeyValueEntity entity);
+
+    /**
+     * remove keyValue by key.
+     *
+     * @param key - key
+     * @return key/value
+     * @throws NullPointerException key should not be null.
+     */
+    KeyValueEntity remove(String key);
+
+    /**
+     * search keyValue list by search key.
+     *
+     * @param searchKey - search keys.
+     * @return key/value list
+     * @throws NullPointerException search key should not be null.
+     */
+    List<KeyValueEntity> search(StateSearchKey searchKey);
+
+    /**
+     * search commands using ack status
+     * @param isAcked
+     * @return
+     */
+    List<CommandEntity> searchCommands(boolean isAcked);
+
+    /**
+     * search one keyValue by search key
+     * @param searchKey - search key
+     * @return null or keyValue
+     */
+    KeyValueEntity searchOne(StateSearchKey searchKey);
+
+    /**
+     * search one keyValue by fileName
+     * @param fileName
+     * @return
+     */
+    KeyValueEntity searchOne(String fileName);
+
+    /**
+     * find all by prefix key.
+     * @param prefix - prefix string
+     * @return list of k/v
+     */
+    List<KeyValueEntity> findAll(String prefix);
+}
diff --git a/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/db/JobProfileDb.java b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/db/JobProfileDb.java
new file mode 100644
index 0000000..f8bc6ea
--- /dev/null
+++ b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/db/JobProfileDb.java
@@ -0,0 +1,182 @@
+/*
+ * 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.inlong.agent.db;
+
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.inlong.agent.conf.JobProfile;
+import org.apache.inlong.agent.constants.JobConstants;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Wrapper for job conf persistence.
+ */
+public class JobProfileDb {
+    private static final Logger LOGGER = LoggerFactory.getLogger(JobProfileDb.class);
+    private final Db db;
+
+    public JobProfileDb(Db db) {
+        this.db = db;
+    }
+
+    /**
+     * get job which in accepted state
+     * @return null or job conf
+     */
+    public JobProfile getAcceptedJob() {
+        return getJob(StateSearchKey.ACCEPTED);
+    }
+
+    public List<JobProfile> getAcceptedJobs() {
+        return getJobs(StateSearchKey.ACCEPTED);
+    }
+
+    /**
+     * update job state and search it by key name
+     * @param jobInstanceId - job key name
+     * @param stateSearchKey - job state
+     */
+    public void updateJobState(String jobInstanceId, StateSearchKey stateSearchKey) {
+        KeyValueEntity entity = db.get(jobInstanceId);
+        if (entity != null) {
+            entity.setStateSearchKey(stateSearchKey);
+            db.put(entity);
+        }
+    }
+
+    /**
+     * store job profile
+     * @param jobProfile - job profile
+     */
+    public void storeJobFirstTime(JobProfile jobProfile) {
+        if (jobProfile.allRequiredKeyExist()) {
+            String keyName = jobProfile.get(JobConstants.JOB_INSTANCE_ID);
+            jobProfile.setLong(JobConstants.JOB_STORE_TIME, System.currentTimeMillis());
+            KeyValueEntity entity = new KeyValueEntity(keyName,
+                jobProfile.toJsonStr(), jobProfile.get(JobConstants.JOB_DIR_FILTER_PATTERN));
+            entity.setStateSearchKey(StateSearchKey.ACCEPTED);
+            db.put(entity);
+        }
+    }
+
+    /**
+     * update job profile
+     * @param jobProfile
+     */
+    public void updateJobProfile(JobProfile jobProfile) {
+        String instanceId = jobProfile.getInstanceId();
+        KeyValueEntity entity = db.get(instanceId);
+        if (entity == null) {
+            LOGGER.warn("job profile {} doesn't exist, update job profile fail {}", instanceId, jobProfile.toJsonStr());
+            return;
+        }
+        entity.setJsonValue(jobProfile.toJsonStr());
+        db.put(entity);
+    }
+
+    /**
+     * check whether job is finished, note that non-exist job is regarded as finished.
+     * @param jobProfile
+     * @return
+     */
+    public boolean checkJobfinished(JobProfile jobProfile) {
+        KeyValueEntity entity = db.get(jobProfile.getInstanceId());
+        if (entity == null) {
+            LOGGER.info("job profile {} doesn't exist", jobProfile.getInstanceId());
+            return true;
+        }
+        return entity.checkFinished();
+    }
+
+    public void deleteJob(String keyName) {
+        db.remove(keyName);
+    }
+
+    public JobProfile getJobProfile(String jobId) {
+        KeyValueEntity keyValueEntity = db.get(jobId);
+        if (keyValueEntity != null) {
+            return keyValueEntity.getAsJobProfile();
+        }
+        return null;
+    }
+
+    public void removeExpireJobs(long expireTime) {
+        // remove finished tasks
+        List<KeyValueEntity> successEntityList = db.search(StateSearchKey.SUCCESS);
+        List<KeyValueEntity> failedEntityList = db.search(StateSearchKey.FAILED);
+        List<KeyValueEntity> entityList = new ArrayList<>(successEntityList);
+        entityList.addAll(failedEntityList);
+        for (KeyValueEntity entity : entityList) {
+            if (entity.getKey().startsWith(JobConstants.JOB_ID_PREFIX)) {
+                JobProfile profile = entity.getAsJobProfile();
+                long storeTime = profile.getLong(JobConstants.JOB_STORE_TIME, 0);
+                long currentTime = System.currentTimeMillis();
+                if (storeTime == 0 || currentTime - storeTime > expireTime) {
+                    LOGGER.info("delete job {} because of timeout store time: {}, expire time: {}",
+                        entity.getKey(), storeTime, expireTime);
+                    deleteJob(entity.getKey());
+                }
+            }
+        }
+    }
+
+    /**
+     * get job conf by state
+     * @param stateSearchKey - state index for searching.
+     * @return
+     */
+    public JobProfile getJob(StateSearchKey stateSearchKey) {
+        KeyValueEntity entity = db.searchOne(stateSearchKey);
+        if (entity != null && entity.getKey().startsWith(JobConstants.JOB_ID_PREFIX)) {
+            return entity.getAsJobProfile();
+        }
+        return null;
+    }
+
+
+    /**
+     * get job reading specific file
+     * @param fileName
+     * @return
+     */
+    public JobProfile getJob(String fileName) {
+        KeyValueEntity entity = db.searchOne(fileName);
+        if (entity != null && entity.getKey().startsWith(JobConstants.JOB_ID_PREFIX)) {
+            return entity.getAsJobProfile();
+        }
+        return null;
+    }
+
+    /**
+     * get list of job profiles.
+     * @param stateSearchKey - state search key.
+     * @return - list of job profile.
+     */
+    public List<JobProfile> getJobs(StateSearchKey stateSearchKey) {
+        List<KeyValueEntity> entityList = db.search(stateSearchKey);
+        List<JobProfile> profileList = new ArrayList<>();
+        for (KeyValueEntity entity : entityList) {
+            if (entity.getKey().startsWith(JobConstants.JOB_ID_PREFIX)) {
+                profileList.add(entity.getAsJobProfile());
+            }
+        }
+        return profileList;
+    }
+}
diff --git a/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/db/KeyValueEntity.java b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/db/KeyValueEntity.java
new file mode 100644
index 0000000..ab0ca19
--- /dev/null
+++ b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/db/KeyValueEntity.java
@@ -0,0 +1,105 @@
+/*
+ * 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.inlong.agent.db;
+
+import com.sleepycat.persist.model.Entity;
+import com.sleepycat.persist.model.PrimaryKey;
+import com.sleepycat.persist.model.Relationship;
+import com.sleepycat.persist.model.SecondaryKey;
+import org.apache.inlong.agent.conf.JobProfile;
+import org.apache.inlong.agent.conf.TriggerProfile;
+
+/**
+ * key value entity. key is string and value is a json
+ */
+@Entity(version = 1)
+public class KeyValueEntity {
+
+    @PrimaryKey
+    private String key;
+
+    @SecondaryKey(relate = Relationship.MANY_TO_ONE)
+    private StateSearchKey stateSearchKey;
+
+    /**
+     * stores the file name that the jsonValue refers
+     */
+    @SecondaryKey(relate = Relationship.MANY_TO_ONE)
+    private String fileName;
+
+    private String jsonValue;
+
+    private KeyValueEntity() {
+
+    }
+
+    public KeyValueEntity(String key, String jsonValue, String fileName) {
+        this.key = key;
+        this.jsonValue = jsonValue;
+        this.stateSearchKey = StateSearchKey.ACCEPTED;
+        this.fileName = fileName;
+    }
+
+    public String getKey() {
+        return key;
+    }
+
+    public StateSearchKey getStateSearchKey() {
+        return stateSearchKey;
+    }
+
+    public KeyValueEntity setStateSearchKey(StateSearchKey stateSearchKey) {
+        this.stateSearchKey = stateSearchKey;
+        return this;
+    }
+
+    public String getJsonValue() {
+        return jsonValue;
+    }
+
+    public KeyValueEntity setJsonValue(String jsonValue) {
+        this.jsonValue = jsonValue;
+        return this;
+    }
+
+    /**
+     * convert keyValue to job profile
+     * @return JobConfiguration
+     */
+    public JobProfile getAsJobProfile() {
+        // convert jsonValue to jobConfiguration
+        return JobProfile.parseJsonStr(getJsonValue());
+    }
+
+    /**
+     * convert keyValue to trigger profile
+     * @return
+     */
+    public TriggerProfile getAsTriggerProfile() {
+        return TriggerProfile.parseJsonStr(getJsonValue());
+    }
+
+    /**
+     * check whether the entity is finished
+     * @return
+     */
+    public boolean checkFinished() {
+        return stateSearchKey.equals(StateSearchKey.SUCCESS)
+                || stateSearchKey.equals(StateSearchKey.FAILED);
+    }
+}
diff --git a/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/db/LocalProfile.java b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/db/LocalProfile.java
new file mode 100755
index 0000000..9439912
--- /dev/null
+++ b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/db/LocalProfile.java
@@ -0,0 +1,71 @@
+/*
+ * 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.inlong.agent.db;
+
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import org.apache.inlong.agent.conf.JobProfile;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * profile from local file
+ */
+public class LocalProfile {
+    private static final Logger LOGGER = LoggerFactory.getLogger(LocalProfile.class);
+
+    private static final String JSON_SUFFIX = ".json";
+    private static final String PROPERTIES_SUFFIX = ".properties";
+    private static final String PARENT_PATH = "/jobs";
+
+    private final Path filePath;
+
+    public LocalProfile(String parentConfPath) {
+        String fileName = parentConfPath + PARENT_PATH;
+        this.filePath = Paths.get(fileName);
+    }
+
+    public List<JobProfile> readFromLocal() {
+        List<JobProfile> profileList = new ArrayList<>();
+        try {
+            if (Files.isDirectory(this.filePath)) {
+                // list parent path and find files which name is end with .json or .properties
+                for (Iterator<Path> it = Files.list(this.filePath).iterator(); it.hasNext(); ) {
+                    String childPath = it.next().toString();
+                    JobProfile jobProfile = null;
+                    if (childPath.endsWith(JSON_SUFFIX)) {
+                        jobProfile = JobProfile.parseJsonFile(childPath);
+                    } else if (childPath.endsWith(PROPERTIES_SUFFIX)) {
+                        jobProfile = JobProfile.parsePropertiesFile(childPath);
+                    }
+                    if (jobProfile != null && jobProfile.allRequiredKeyExist()) {
+                        profileList.add(jobProfile);
+                    }
+                }
+            }
+        } catch (Exception ex) {
+            LOGGER.error("error in reading files {}", this.filePath);
+        }
+        return profileList;
+    }
+}
diff --git a/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/db/RocksDbImp.java b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/db/RocksDbImp.java
new file mode 100755
index 0000000..8babb0e
--- /dev/null
+++ b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/db/RocksDbImp.java
@@ -0,0 +1,123 @@
+/*
+ * 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.inlong.agent.db;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.List;
+import org.apache.inlong.agent.conf.AgentConfiguration;
+import org.apache.inlong.agent.constants.AgentConstants;
+import org.rocksdb.Options;
+import org.rocksdb.RocksDB;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * DB implement based on rocks db.
+ * TODO: this is low priority.
+ */
+public class RocksDbImp implements Db {
+    private static final Logger LOGGER = LoggerFactory.getLogger(RocksDbImp.class);
+
+    private final AgentConfiguration conf;
+    private final RocksDB db;
+
+    public RocksDbImp() {
+        // init rocks db
+        this.conf = AgentConfiguration.getAgentConf();
+        this.db = initEnv();
+    }
+
+    private RocksDB initEnv() {
+        String storePath = conf.get(
+            AgentConstants.AGENT_ROCKS_DB_PATH, AgentConstants.DEFAULT_AGENT_ROCKS_DB_PATH);
+        String parentPath = conf.get(AgentConstants.AGENT_HOME, AgentConstants.DEFAULT_AGENT_HOME);
+        File finalPath = new File(parentPath, storePath);
+        RocksDB.loadLibrary();
+        final Options options = new Options();
+        options.setCreateIfMissing(true);
+        try {
+            boolean result = finalPath.mkdirs();
+            LOGGER.info("create directory {}, result is {}", finalPath, result);
+            return RocksDB.open(options, finalPath.getAbsolutePath());
+        } catch (Exception ex) {
+            // cannot create local path, stop running.
+            throw new RuntimeException(ex);
+        }
+    }
+
+    @Override
+    public KeyValueEntity get(String key) {
+        return null;
+    }
+
+    @Override
+    public CommandEntity getCommand(String commandId) {
+        return null;
+    }
+
+    @Override
+    public CommandEntity putCommand(CommandEntity entity) {
+        return null;
+    }
+
+    @Override
+    public void set(KeyValueEntity entity) {
+
+    }
+
+    @Override
+    public KeyValueEntity put(KeyValueEntity entity) {
+        return null;
+    }
+
+    @Override
+    public KeyValueEntity remove(String key) {
+        return null;
+    }
+
+    @Override
+    public List<KeyValueEntity> search(StateSearchKey searchKey) {
+        return null;
+    }
+
+    @Override
+    public List<CommandEntity> searchCommands(boolean isAcked) {
+        return null;
+    }
+
+    @Override
+    public KeyValueEntity searchOne(StateSearchKey searchKey) {
+        return null;
+    }
+
+    @Override
+    public KeyValueEntity searchOne(String fileName) {
+        return null;
+    }
+
+    @Override
+    public List<KeyValueEntity> findAll(String prefix) {
+        return null;
+    }
+
+    @Override
+    public void close() throws IOException {
+
+    }
+}
diff --git a/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/db/StateSearchKey.java b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/db/StateSearchKey.java
new file mode 100755
index 0000000..e747279
--- /dev/null
+++ b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/db/StateSearchKey.java
@@ -0,0 +1,32 @@
+/*
+ * 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.inlong.agent.db;
+
+/**
+ * search key for state.
+ */
+public enum StateSearchKey {
+    // success state
+    SUCCESS,
+    // fail state
+    FAILED,
+    // accepted state
+    ACCEPTED,
+    // running state
+    RUNNING
+}
diff --git a/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/db/TriggerProfileDb.java b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/db/TriggerProfileDb.java
new file mode 100755
index 0000000..af58613
--- /dev/null
+++ b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/db/TriggerProfileDb.java
@@ -0,0 +1,79 @@
+/*
+ * 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.inlong.agent.db;
+
+
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.inlong.agent.conf.TriggerProfile;
+import org.apache.inlong.agent.constants.CommonConstants;
+import org.apache.inlong.agent.constants.JobConstants;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * db interface for trigger profile.
+ */
+public class TriggerProfileDb {
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(TriggerProfileDb.class);
+
+    private final Db db;
+
+    public TriggerProfileDb(Db db) {
+        this.db = db;
+    }
+
+    /**
+     * get trigger list from db.
+     * @return - list of trigger
+     */
+    public List<TriggerProfile> getTriggers() {
+        // potential performance issue, needs to find out the speed.
+        List<KeyValueEntity> result = this.db.findAll(CommonConstants.TRIGGER_ID_PREFIX);
+        List<TriggerProfile> triggerList = new ArrayList<>();
+        for (KeyValueEntity entity : result) {
+            triggerList.add(entity.getAsTriggerProfile());
+        }
+        return triggerList;
+    }
+
+    /**
+     * store trigger profile.
+     * @param trigger - trigger
+     */
+    public void storeTrigger(TriggerProfile trigger) {
+        if (trigger.allRequiredKeyExist()) {
+            String keyName = CommonConstants.TRIGGER_ID_PREFIX + trigger.get(JobConstants.JOB_ID);
+            KeyValueEntity entity = new KeyValueEntity(keyName,
+                trigger.toJsonStr(), trigger.get(JobConstants.JOB_DIR_FILTER_PATTERN));
+            KeyValueEntity oldEntity = db.put(entity);
+            if (oldEntity != null) {
+                LOGGER.warn("trigger profile {} has been replaced", oldEntity.getKey());
+            }
+        }
+    }
+
+    /**
+     * delete trigger by id.
+     * @param id
+     */
+    public void deleteTrigger(String id) {
+        db.remove(CommonConstants.TRIGGER_ID_PREFIX + id);
+    }
+}
diff --git a/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/message/DefaultMessage.java b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/message/DefaultMessage.java
new file mode 100644
index 0000000..3c735cf
--- /dev/null
+++ b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/message/DefaultMessage.java
@@ -0,0 +1,53 @@
+/*
+ * 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.inlong.agent.message;
+
+import java.nio.charset.StandardCharsets;
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.inlong.agent.plugin.Message;
+
+public class DefaultMessage implements Message {
+
+    private final byte[] body;
+    private final Map<String, String> header;
+
+    public DefaultMessage(byte[] body, Map<String, String> header) {
+        this.body = body;
+        this.header = header;
+    }
+
+    public DefaultMessage(byte[] body) {
+        this(body, new HashMap<>());
+    }
+
+    @Override
+    public byte[] getBody() {
+        return body;
+    }
+
+    @Override
+    public Map<String, String> getHeader() {
+        return header;
+    }
+
+    @Override
+    public String toString() {
+        return new String(body, StandardCharsets.UTF_8);
+    }
+}
diff --git a/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/message/EndMessage.java b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/message/EndMessage.java
new file mode 100644
index 0000000..53876e4
--- /dev/null
+++ b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/message/EndMessage.java
@@ -0,0 +1,39 @@
+/*
+ * 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.inlong.agent.message;
+
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.inlong.agent.plugin.Message;
+
+/**
+ * End message, this is an empty message only indicate
+ * that source data have been completely consumed.
+ */
+public class EndMessage implements Message {
+
+    @Override
+    public byte[] getBody() {
+        return null;
+    }
+
+    @Override
+    public Map<String, String> getHeader() {
+        return new HashMap<>(10);
+    }
+}
diff --git a/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/message/ProxyMessage.java b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/message/ProxyMessage.java
new file mode 100644
index 0000000..7afd369
--- /dev/null
+++ b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/message/ProxyMessage.java
@@ -0,0 +1,74 @@
+/*
+ * 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.inlong.agent.message;
+
+import java.util.Map;
+import org.apache.inlong.agent.plugin.Message;
+
+/**
+ * Bus message with body, header, bid and tid.
+ */
+public class ProxyMessage implements Message {
+
+    private static final String DEFAULT_TID = "__";
+
+    private final byte[] body;
+    private final Map<String, String> header;
+    private final String bid;
+    private final String tid;
+
+
+    public ProxyMessage(byte[] body, Map<String, String> header) {
+        this.body = body;
+        this.header = header;
+        this.bid = header.get("bid");
+        this.tid = header.getOrDefault("tid", DEFAULT_TID);
+    }
+
+    /**
+     * Get first line of body list
+     *
+     * @return first line of body list
+     */
+    @Override
+    public byte[] getBody() {
+        return body;
+    }
+
+    /**
+     * Get header of message
+     *
+     * @return header
+     */
+    @Override
+    public Map<String, String> getHeader() {
+        return header;
+    }
+
+    public String getBid() {
+        return bid;
+    }
+
+    public String getTid() {
+        return tid;
+    }
+
+    public static ProxyMessage parse(Message message) {
+        return new ProxyMessage(message.getBody(), message.getHeader());
+    }
+}
diff --git a/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/metrics/AgentDynamicMBean.java b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/metrics/AgentDynamicMBean.java
new file mode 100644
index 0000000..5331342
--- /dev/null
+++ b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/metrics/AgentDynamicMBean.java
@@ -0,0 +1,129 @@
+/*
+ * 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.inlong.agent.metrics;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.ConcurrentHashMap;
+import javax.management.Attribute;
+import javax.management.AttributeList;
+import javax.management.DynamicMBean;
+import javax.management.MBeanAttributeInfo;
+import javax.management.MBeanInfo;
+import org.apache.inlong.agent.metrics.meta.MetricMeta;
+import org.apache.inlong.agent.metrics.meta.MetricsMeta;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Dynamic MBean for agent
+ */
+public class AgentDynamicMBean implements DynamicMBean {
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(AgentDynamicMBean.class);
+
+    private final ConcurrentHashMap<String, MetricSnapshot<?>> snapshotAttrs = new ConcurrentHashMap<>();
+    private final MBeanInfo mBeanInfo;
+    private final List<MBeanAttributeInfo> attrs;
+    private final MetricsMeta metricsMeta;
+    private final String module;
+    private final String aspect;
+    private final String desc;
+
+    public AgentDynamicMBean(String module, String aspect, String desc,
+        MetricsMeta metricsMeta, Object source) {
+        this.module = module;
+        this.aspect = aspect;
+        this.desc = desc;
+        this.metricsMeta = metricsMeta;
+        this.attrs = new ArrayList<>();
+        this.mBeanInfo = metricsMetaToInfo();
+        formatSnapshotList(source);
+    }
+
+    private void formatSnapshotList(Object source) {
+        for (MetricMeta metricMeta : this.metricsMeta.getMetricMetaList()) {
+            try {
+                snapshotAttrs.put(metricMeta.getName(),
+                    (MetricSnapshot<?>) metricMeta.getField().get(source));
+            } catch (Exception ex) {
+                LOGGER.error("exception while adding snapshot list", ex);
+            }
+        }
+    }
+
+
+    private MBeanInfo metricsMetaToInfo() {
+        // overwrite name, desc from MetricsMeta if not null.
+        String name = this.module == null ? metricsMeta.getName() : this.module;
+        String description = this.desc == null ? metricsMeta.getDesc() : this.desc;
+
+        for (MetricMeta fieldMetricMeta : metricsMeta.getMetricMetaList()) {
+            attrs.add(new MBeanAttributeInfo(fieldMetricMeta.getName(),
+                fieldMetricMeta.getType(), fieldMetricMeta.getDesc(), true, false, false));
+        }
+        return new MBeanInfo(name, description, attrs.toArray(new MBeanAttributeInfo[0]),
+            null, null, null);
+    }
+
+    @Override
+    public Object getAttribute(String attribute) {
+        MetricSnapshot<?> snapshot = snapshotAttrs.get(attribute);
+        return new Attribute(attribute, snapshot.snapshot());
+    }
+
+    @Override
+    public void setAttribute(Attribute attribute) {
+        throw new UnsupportedOperationException("Metrics are read-only.");
+    }
+
+    @Override
+    public AttributeList getAttributes(String[] attributes) {
+        AttributeList attributeList = new AttributeList();
+        for (String attrKey : attributes) {
+            MetricSnapshot<?> snapshot = snapshotAttrs.get(attrKey);
+            if (snapshot != null) {
+                attributeList.add(new Attribute(attrKey, snapshot.snapshot()));
+            }
+        }
+        return attributeList;
+    }
+
+    @Override
+    public AttributeList setAttributes(AttributeList attributes) {
+        throw new UnsupportedOperationException("Metrics are read-only.");
+    }
+
+    @Override
+    public Object invoke(String actionName, Object[] params, String[] signature) {
+        throw new UnsupportedOperationException("Not supported yet.");
+    }
+
+    @Override
+    public MBeanInfo getMBeanInfo() {
+        return mBeanInfo;
+    }
+
+    public String getModule() {
+        return module;
+    }
+
+    public String getAspect() {
+        return aspect;
+    }
+}
diff --git a/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/metrics/Metric.java b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/metrics/Metric.java
new file mode 100644
index 0000000..194bc41
--- /dev/null
+++ b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/metrics/Metric.java
@@ -0,0 +1,63 @@
+/*
+ * 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.inlong.agent.metrics;
+
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
+
+/**
+ * metric for field
+ */
+@Retention(RetentionPolicy.RUNTIME)
+@Target({ElementType.FIELD, ElementType.METHOD})
+public @interface Metric {
+
+    /**
+     * Type of metric
+     *
+     * @return metric type
+     */
+    Type type() default Type.DEFAULT;
+
+    /**
+     * Doc of metric
+     *
+     * @return metric doc
+     */
+    String desc() default "";
+
+    enum Type {
+        DEFAULT("java.lang.String"),
+        COUNTER_INT("java.lang.Integer"),
+        COUNTER_LONG("java.lang.Long"),
+        GAUGE_INT("java.lang.Integer"),
+        GAUGE_LONG("java.lang.Long"),
+        TAG("java.lang.String");
+        private final String value;
+
+        private Type(String value) {
+            this.value = value;
+        }
+
+        public String getValue() {
+            return value;
+        }
+    }
+}
diff --git a/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/metrics/MetricException.java b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/metrics/MetricException.java
new file mode 100644
index 0000000..ada7edd
--- /dev/null
+++ b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/metrics/MetricException.java
@@ -0,0 +1,29 @@
+/*
+ * 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.inlong.agent.metrics;
+
+public class MetricException extends RuntimeException {
+
+    public MetricException(String message, Exception ex) {
+        super(message, ex);
+    }
+
+    public MetricException(String message) {
+        super(message);
+    }
+}
diff --git a/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/metrics/MetricSnapshot.java b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/metrics/MetricSnapshot.java
new file mode 100644
index 0000000..bd729c8
--- /dev/null
+++ b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/metrics/MetricSnapshot.java
@@ -0,0 +1,22 @@
+/*
+ * 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.inlong.agent.metrics;
+
+public interface MetricSnapshot<T> {
+    T snapshot();
+}
diff --git a/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/metrics/Metrics.java b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/metrics/Metrics.java
new file mode 100644
index 0000000..f3ae8e7
--- /dev/null
+++ b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/metrics/Metrics.java
@@ -0,0 +1,52 @@
+/*
+ * 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.inlong.agent.metrics;
+
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
+
+/**
+ * metric
+ */
+@Target({ElementType.TYPE})
+@Retention(RetentionPolicy.RUNTIME)
+public @interface Metrics {
+
+    /**
+     * Metrics name
+     *
+     * @return
+     */
+    String name() default "";
+
+    /**
+     * Metrics context
+     *
+     * @return
+     */
+    String context() default "";
+
+    /**
+     * Metrics description
+     *
+     * @return
+     */
+    String desc() default "";
+}
diff --git a/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/metrics/MetricsRegister.java b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/metrics/MetricsRegister.java
new file mode 100644
index 0000000..88278c9
--- /dev/null
+++ b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/metrics/MetricsRegister.java
@@ -0,0 +1,146 @@
+/*
+ * 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.inlong.agent.metrics;
+
+import java.lang.annotation.Annotation;
+import java.lang.management.ManagementFactory;
+import java.lang.reflect.Field;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.ConcurrentHashMap;
+import javax.management.MBeanServer;
+import javax.management.ObjectName;
+import org.apache.inlong.agent.metrics.counter.CounterInt;
+import org.apache.inlong.agent.metrics.counter.CounterLong;
+import org.apache.inlong.agent.metrics.gauge.GaugeInt;
+import org.apache.inlong.agent.metrics.gauge.GaugeLong;
+import org.apache.inlong.agent.metrics.meta.MetricMeta;
+import org.apache.inlong.agent.metrics.meta.MetricsMeta;
+import org.apache.inlong.agent.utils.AgentUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * register for metrics.
+ */
+public class MetricsRegister {
+    private static final Logger LOGGER = LoggerFactory.getLogger(MetricsRegister.class);
+
+    private static final String DOMAIN_PREFIX = "Agent:";
+    private static final String MODULE_PREFIX = "module=";
+    private static final String ASPECT_PREFIX = "aspect=";
+    private static final String COMMA_SPLITTER = ",";
+
+    // object name should be uniq
+    private static final ConcurrentHashMap<String, ObjectName> CACHED_NAME = new ConcurrentHashMap<>();
+
+    private MetricsRegister() {
+    }
+
+    /**
+     * register object name for metric
+     * @param agentDynamicMBean agent mbean
+     */
+    private static void innerRegister(AgentDynamicMBean agentDynamicMBean) {
+        final MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
+        String nameStr = DOMAIN_PREFIX + MODULE_PREFIX + agentDynamicMBean.getModule()
+                + COMMA_SPLITTER + ASPECT_PREFIX + agentDynamicMBean.getAspect();
+        try {
+            ObjectName tmpName = new ObjectName(nameStr);
+            ObjectName objectName = CACHED_NAME.putIfAbsent(nameStr, tmpName);
+            if (objectName == null) {
+                mbs.registerMBean(agentDynamicMBean, tmpName);
+            }
+        } catch (Exception ex) {
+            LOGGER.error("exception while register mbean", ex);
+        }
+    }
+
+    public static void register(String module, String aspect, String desc, Object source) {
+        List<MetricMeta> metricMetaList = handleFieldAnnotation(source);
+        MetricsMeta metricsMeta = handleClassAnnotation(source, metricMetaList);
+        if (metricsMeta != null) {
+            innerRegister(new AgentDynamicMBean(module, aspect, desc, metricsMeta, source));
+        } else {
+            LOGGER.error("Cannot find Metrics annotation in {}, invalid metric", source);
+        }
+    }
+
+
+    /**
+     * handle class level annotation
+     */
+    private static MetricsMeta handleClassAnnotation(Object source,
+            List<MetricMeta> metricMetaList) {
+        for (Annotation annotation : source.getClass().getAnnotations()) {
+            if (annotation instanceof Metrics) {
+                return MetricsMeta.build((Metrics) annotation, metricMetaList);
+            }
+        }
+        return null;
+    }
+
+
+    private static boolean initFieldByType(Object source, Field field) {
+        try {
+            if (field.getType() == CounterInt.class) {
+                field.set(source, new CounterInt());
+                return true;
+            } else if (field.getType() == CounterLong.class) {
+                field.set(source, new CounterLong());
+                return true;
+            } else if (field.getType() == GaugeInt.class) {
+                field.set(source, new GaugeInt());
+                return true;
+            } else if (field.getType() == GaugeLong.class) {
+                field.set(source, new GaugeLong());
+                return true;
+            } else if (field.getType() == Tag.class) {
+                field.set(source, new Tag());
+                return true;
+            } else {
+                throw new MetricException("field type error " + field.getType().toString());
+            }
+        } catch (MetricException ex) {
+            throw ex;
+        } catch (Exception ex) {
+            throw new MetricException("Error setting field " + field
+                    + " annotated with metric", ex);
+        }
+    }
+
+    /**
+     * handle field annotation
+     */
+    private static List<MetricMeta> handleFieldAnnotation(Object source) {
+        List<MetricMeta> result = new ArrayList<>();
+        for (Field field : AgentUtils.getDeclaredFieldsIncludingInherited(source.getClass())) {
+            field.setAccessible(true);
+            for (Annotation fieldAnnotation : field.getAnnotations()) {
+                if (fieldAnnotation instanceof Metric) {
+                    if (initFieldByType(source, field)) {
+                        result.add(MetricMeta.build((Metric) fieldAnnotation, field));
+                    }
+                    break;
+                }
+            }
+        }
+        return result;
+    }
+}
+
diff --git a/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/metrics/MutableMetric.java b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/metrics/MutableMetric.java
new file mode 100644
index 0000000..f3112f6
--- /dev/null
+++ b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/metrics/MutableMetric.java
@@ -0,0 +1,25 @@
+/*
+ * 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.inlong.agent.metrics;
+
+/**
+ * Metric with number
+ */
+public interface MutableMetric extends MetricSnapshot<Number> {
+
+}
diff --git a/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/metrics/Tag.java b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/metrics/Tag.java
new file mode 100755
index 0000000..1682f33
--- /dev/null
+++ b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/metrics/Tag.java
@@ -0,0 +1,40 @@
+/*
+ * 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.inlong.agent.metrics;
+
+public class Tag implements MetricSnapshot<String> {
+
+    private String name;
+
+    /**
+     * set string name for tag.
+     * @param name - tag name
+     */
+    public void setName(String name) {
+        this.name = name;
+    }
+
+    public String getName() {
+        return name;
+    }
+
+    @Override
+    public String snapshot() {
+        return name;
+    }
+}
diff --git a/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/metrics/counter/Counter.java b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/metrics/counter/Counter.java
new file mode 100644
index 0000000..d035a69
--- /dev/null
+++ b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/metrics/counter/Counter.java
@@ -0,0 +1,36 @@
+/*
+ * 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.inlong.agent.metrics.counter;
+
+import org.apache.inlong.agent.metrics.MutableMetric;
+
+public interface Counter extends MutableMetric {
+
+    /**
+     * +1
+     */
+    void incr();
+
+    /**
+     * +delta
+     *
+     * @param delta > 0
+     */
+    void incr(int delta);
+
+}
diff --git a/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/metrics/counter/CounterInt.java b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/metrics/counter/CounterInt.java
new file mode 100644
index 0000000..ef1215b
--- /dev/null
+++ b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/metrics/counter/CounterInt.java
@@ -0,0 +1,41 @@
+/*
+ * 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.inlong.agent.metrics.counter;
+
+import java.util.concurrent.atomic.AtomicInteger;
+
+public class CounterInt implements Counter {
+
+    private AtomicInteger value = new AtomicInteger(0);
+
+    @Override
+    public void incr() {
+        value.incrementAndGet();
+    }
+
+    @Override
+    public void incr(int delta) {
+        assert delta > 0;
+        value.getAndAdd(delta);
+    }
+
+    @Override
+    public Integer snapshot() {
+        return value.get();
+    }
+}
diff --git a/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/metrics/counter/CounterLong.java b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/metrics/counter/CounterLong.java
new file mode 100644
index 0000000..fe9decd
--- /dev/null
+++ b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/metrics/counter/CounterLong.java
@@ -0,0 +1,44 @@
+/*
+ * 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.inlong.agent.metrics.counter;
+
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ * atomic long for counter
+ */
+public class CounterLong implements Counter {
+
+    private final AtomicLong value = new AtomicLong();
+
+    @Override
+    public void incr() {
+        value.incrementAndGet();
+    }
+
+    @Override
+    public void incr(int delta) {
+        assert delta > 0;
+        value.getAndAdd(delta);
+    }
+
+    @Override
+    public Long snapshot() {
+        return value.get();
+    }
+}
diff --git a/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/metrics/gauge/Gauge.java b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/metrics/gauge/Gauge.java
new file mode 100644
index 0000000..9427111
--- /dev/null
+++ b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/metrics/gauge/Gauge.java
@@ -0,0 +1,50 @@
+/*
+ * 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.inlong.agent.metrics.gauge;
+
+
+import org.apache.inlong.agent.metrics.MutableMetric;
+
+public interface Gauge<T> extends MutableMetric {
+
+    void set(T num);
+
+    /**
+     * +1
+     */
+    void incr();
+
+    /**
+     * + delt
+     *
+     * @param delta > 0
+     */
+    void incr(int delta);
+
+    /**
+     * -1
+     */
+    void decr();
+
+    /**
+     * -delta
+     *
+     * @param delta > 0
+     */
+    void decr(int delta);
+}
diff --git a/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/metrics/gauge/GaugeInt.java b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/metrics/gauge/GaugeInt.java
new file mode 100644
index 0000000..0246289
--- /dev/null
+++ b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/metrics/gauge/GaugeInt.java
@@ -0,0 +1,57 @@
+/*
+ * 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.inlong.agent.metrics.gauge;
+
+import java.util.concurrent.atomic.AtomicInteger;
+
+public class GaugeInt implements Gauge<Integer> {
+
+    private final AtomicInteger value = new AtomicInteger(0);
+
+    @Override
+    public void set(Integer num) {
+        value.set(num);
+    }
+
+    @Override
+    public void incr() {
+        value.incrementAndGet();
+    }
+
+    @Override
+    public void incr(int delta) {
+        assert delta > 0;
+        value.getAndAdd(delta);
+    }
+
+    @Override
+    public void decr() {
+        value.decrementAndGet();
+    }
+
+    @Override
+    public void decr(int delta) {
+        assert delta > 0;
+        value.getAndAdd(-delta);
+    }
+
+    @Override
+    public Integer snapshot() {
+        return value.get();
+    }
+}
diff --git a/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/metrics/gauge/GaugeLong.java b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/metrics/gauge/GaugeLong.java
new file mode 100644
index 0000000..4bf01bf
--- /dev/null
+++ b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/metrics/gauge/GaugeLong.java
@@ -0,0 +1,57 @@
+/*
+ * 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.inlong.agent.metrics.gauge;
+
+import java.util.concurrent.atomic.AtomicLong;
+
+public class GaugeLong implements Gauge<Long> {
+
+    private final AtomicLong value = new AtomicLong(0);
+
+    @Override
+    public void set(Long num) {
+        value.set(num);
+    }
+
+    @Override
+    public void incr() {
+        value.incrementAndGet();
+    }
+
+    @Override
+    public void incr(int delta) {
+        assert delta > 0;
+        value.getAndAdd(delta);
+    }
+
+    @Override
+    public void decr() {
+        value.decrementAndGet();
+    }
+
+    @Override
+    public void decr(int delta) {
+        assert delta > 0;
+        value.getAndAdd(-delta);
+    }
+
+    @Override
+    public Long snapshot() {
+        return value.get();
+    }
+}
diff --git a/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/metrics/meta/MetricMeta.java b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/metrics/meta/MetricMeta.java
new file mode 100644
index 0000000..adf8890
--- /dev/null
+++ b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/metrics/meta/MetricMeta.java
@@ -0,0 +1,79 @@
+/*
+ * 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.inlong.agent.metrics.meta;
+
+import static org.apache.inlong.agent.metrics.Metric.Type.COUNTER_INT;
+import static org.apache.inlong.agent.metrics.Metric.Type.COUNTER_LONG;
+import static org.apache.inlong.agent.metrics.Metric.Type.DEFAULT;
+import static org.apache.inlong.agent.metrics.Metric.Type.GAUGE_INT;
+import static org.apache.inlong.agent.metrics.Metric.Type.GAUGE_LONG;
+
+import java.lang.reflect.Field;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.inlong.agent.metrics.Metric;
+import org.apache.inlong.agent.metrics.counter.CounterInt;
+import org.apache.inlong.agent.metrics.counter.CounterLong;
+import org.apache.inlong.agent.metrics.gauge.GaugeInt;
+import org.apache.inlong.agent.metrics.gauge.GaugeLong;
+
+/**
+ * this class is related to {@link Metric}
+ */
+public class MetricMeta {
+
+    private String name;
+    private String type;
+    private String desc;
+    private Field field;
+
+    public static MetricMeta build(Metric annotation, Field field) {
+        MetricMeta metricMeta = new MetricMeta();
+        metricMeta.name = StringUtils.capitalize(field.getName());
+        metricMeta.desc = annotation.desc();
+        metricMeta.type = DEFAULT.getValue();
+        metricMeta.field = field;
+        Class<?> clz = field.getType();
+        if (clz.isAssignableFrom(CounterLong.class)) {
+            metricMeta.type = COUNTER_LONG.getValue();
+        } else if (clz.isAssignableFrom(CounterInt.class)) {
+            metricMeta.type = COUNTER_INT.getValue();
+        } else if (clz.isAssignableFrom(GaugeInt.class)) {
+            metricMeta.type = GAUGE_INT.getValue();
+        } else if (clz.isAssignableFrom(GaugeLong.class)) {
+            metricMeta.type = GAUGE_LONG.getValue();
+        }
+        return metricMeta;
+    }
+
+    public String getType() {
+        return type;
+    }
+
+    public String getName() {
+        return name;
+    }
+
+    public String getDesc() {
+        return desc;
+    }
+
+    public Field getField() {
+        return field;
+    }
+}
+
diff --git a/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/metrics/meta/MetricsMeta.java b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/metrics/meta/MetricsMeta.java
new file mode 100644
index 0000000..03a697f
--- /dev/null
+++ b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/metrics/meta/MetricsMeta.java
@@ -0,0 +1,61 @@
+/*
+ * 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.inlong.agent.metrics.meta;
+
+import java.util.List;
+import org.apache.inlong.agent.metrics.Metrics;
+
+/**
+ * This class is related to {@link Metrics}
+ */
+public class MetricsMeta {
+
+    private String context;
+    private String desc;
+    private String name;
+    private List<MetricMeta> metricMetaList;
+
+    private MetricsMeta() {
+    }
+
+    public static MetricsMeta build(Metrics metrics,
+            List<MetricMeta> metricMetaList) {
+        MetricsMeta metricsMeta = new MetricsMeta();
+        metricsMeta.context = metrics.context();
+        metricsMeta.desc = metrics.desc();
+        metricsMeta.name = metrics.name();
+        metricsMeta.metricMetaList = metricMetaList;
+        return metricsMeta;
+    }
+
+    public String getContext() {
+        return context;
+    }
+
+    public String getDesc() {
+        return desc;
+    }
+
+    public List<MetricMeta> getMetricMetaList() {
+        return metricMetaList;
+    }
+
+    public String getName() {
+        return name;
+    }
+}
diff --git a/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/plugin/Channel.java b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/plugin/Channel.java
new file mode 100644
index 0000000..5b192ce
--- /dev/null
+++ b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/plugin/Channel.java
@@ -0,0 +1,54 @@
+/*
+ * 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.inlong.agent.plugin;
+
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Channel is used as data buffer between source and sink.
+ */
+public interface Channel extends Stage {
+
+
+    /**
+     * write message
+     *
+     * @param message - message
+     */
+    void push(Message message);
+
+    /**
+     * write message with timeout
+     *
+     * @param message
+     * @param timeout
+     * @param unit
+     * @return
+     */
+    boolean push(Message message, long timeout, TimeUnit unit);
+
+    /**
+     * read message with timeout
+     *
+     * @param timeout
+     * @param unit
+     * @return
+     */
+    Message pull(long timeout, TimeUnit unit);
+
+}
diff --git a/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/plugin/Filter.java b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/plugin/Filter.java
new file mode 100644
index 0000000..43d403c
--- /dev/null
+++ b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/plugin/Filter.java
@@ -0,0 +1,30 @@
+/*
+ * 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.inlong.agent.plugin;
+
+/**
+ * filter for source split
+ */
+public interface Filter {
+
+    /**
+     * whether reader is legal.
+     * @return true if reader is legal else false.
+     */
+    boolean match();
+}
diff --git a/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/plugin/Message.java b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/plugin/Message.java
new file mode 100644
index 0000000..87c5582
--- /dev/null
+++ b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/plugin/Message.java
@@ -0,0 +1,42 @@
+/*
+ * 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.inlong.agent.plugin;
+
+import java.util.Map;
+
+/**
+ * Message used in inner-data transfer, message is divided into
+ * two parts, header and body. header is the attributes of message
+ * and body is the content of message.
+ */
+public interface Message {
+
+    /**
+     * Data content of message.
+     *
+     * @return bytes body
+     */
+    byte[] getBody();
+
+    /**
+     * Data attribute of message
+     *
+     * @return map header
+     */
+    Map<String, String> getHeader();
+}
diff --git a/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/plugin/Reader.java b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/plugin/Reader.java
new file mode 100755
index 0000000..ea3dabf
--- /dev/null
+++ b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/plugin/Reader.java
@@ -0,0 +1,49 @@
+/*
+ * 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.inlong.agent.plugin;
+
+/**
+ * Reader reads data and provides condition whether the reading action is finished. It's called at
+ * Task level.
+ */
+public interface Reader extends Stage {
+
+    /**
+     * Read message
+     *
+     * @return - message
+     */
+    Message read();
+
+    /**
+     * Whether finish reading
+     * @return
+     */
+    boolean isFinished();
+
+    /**
+     * Return the reader's reading file name
+     * @return
+     */
+    String getReadFile();
+
+    /**
+     * set readTimeout
+     */
+    void setReadTimeout(long mill);
+}
diff --git a/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/plugin/Sink.java b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/plugin/Sink.java
new file mode 100755
index 0000000..4c5d42e
--- /dev/null
+++ b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/plugin/Sink.java
@@ -0,0 +1,38 @@
+/*
+ * 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.inlong.agent.plugin;
+
+/**
+ * Sink data to remote data center
+ */
+public interface Sink extends Stage {
+
+    /**
+     * Write data into data center
+     *
+     * @param message - message
+     */
+    void write(Message message);
+
+
+    /**
+     * set source file name where the message is generated
+     * @param sourceFileName
+     */
+    void setSourceFile(String sourceFileName);
+}
diff --git a/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/plugin/Source.java b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/plugin/Source.java
new file mode 100755
index 0000000..a1da459
--- /dev/null
+++ b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/plugin/Source.java
@@ -0,0 +1,35 @@
+/*
+ * 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.inlong.agent.plugin;
+
+import java.util.List;
+import org.apache.inlong.agent.conf.JobProfile;
+
+/**
+ * Source can be split into multiple reader.
+ */
+public interface Source {
+
+    /**
+     * Split source into a list of readers.
+     *
+     * @param conf job conf
+     * @return - list of reader
+     */
+    List<Reader> split(JobProfile conf);
+}
diff --git a/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/plugin/Stage.java b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/plugin/Stage.java
new file mode 100755
index 0000000..2a9a96c
--- /dev/null
+++ b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/plugin/Stage.java
@@ -0,0 +1,38 @@
+/*
+ * 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.inlong.agent.plugin;
+
+import org.apache.inlong.agent.conf.JobProfile;
+
+/**
+ * Stage definition.
+ */
+public interface Stage {
+
+    /**
+     * Init job.
+     *
+     * @param jobConf - job config
+     */
+    void init(JobProfile jobConf);
+
+    /**
+     * Destroy job.
+     */
+    void destroy();
+}
diff --git a/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/plugin/Trigger.java b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/plugin/Trigger.java
new file mode 100755
index 0000000..02160b5
--- /dev/null
+++ b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/plugin/Trigger.java
@@ -0,0 +1,57 @@
+/*
+ * 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.inlong.agent.plugin;
+
+import java.io.IOException;
+import org.apache.inlong.agent.conf.JobProfile;
+import org.apache.inlong.agent.conf.TriggerProfile;
+
+/**
+ * Trigger interface, which generates job in condition.
+ */
+public interface Trigger {
+
+    /**
+     * init trigger by trigger profile
+     * @param profile
+     * @throws IOException
+     */
+    void init(TriggerProfile profile) throws IOException;
+
+    /**
+     * run trigger.
+     */
+    void run();
+
+    /**
+     * destroy trigger.
+     */
+    void destroy();
+
+    /**
+     * fetch job profile from trigger
+     * @return - job profile
+     */
+    JobProfile fetchJobProfile();
+
+    /**
+     * get trigger profile
+     * @return
+     */
+    TriggerProfile getTriggerProfile();
+}
diff --git a/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/plugin/Validator.java b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/plugin/Validator.java
new file mode 100644
index 0000000..56c749d
--- /dev/null
+++ b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/plugin/Validator.java
@@ -0,0 +1,32 @@
+/*
+ * 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.inlong.agent.plugin;
+
+/**
+ * For every message, there might be validators to filter required ones
+ */
+public interface Validator {
+
+    /**
+     *
+     * @param messageLine
+     * @return
+     */
+    boolean validate(String messageLine);
+
+}
diff --git a/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/state/AbstractStateWrapper.java b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/state/AbstractStateWrapper.java
new file mode 100644
index 0000000..edbdc83
--- /dev/null
+++ b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/state/AbstractStateWrapper.java
@@ -0,0 +1,93 @@
+/*
+ * 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.inlong.agent.state;
+
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.commons.lang3.tuple.ImmutablePair;
+import org.apache.commons.lang3.tuple.Pair;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public abstract class AbstractStateWrapper implements Runnable {
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(AbstractStateWrapper.class);
+
+    private final Map<Pair<State, State>, StateCallback> callBacks = new HashMap<>();
+    private volatile State currentState = State.ACCEPTED;
+
+    public AbstractStateWrapper() {
+        addCallbacks();
+    }
+
+    /**
+     * add callback for state change
+     */
+    public abstract void addCallbacks();
+
+
+    public AbstractStateWrapper addCallback(State begin, State end, StateCallback callback) {
+        callBacks.put(new ImmutablePair<>(begin, end), callback);
+        return this;
+    }
+
+    /**
+     * change state and execute callback functions
+     *
+     * @param nextState - next state
+     */
+    public synchronized void doChangeState(State nextState) {
+        LOGGER.debug("state change, current state is {}, next state is {}", currentState, nextState);
+        Pair<State, State> statePair = new ImmutablePair<>(currentState, nextState);
+        StateCallback callback = callBacks.get(statePair);
+        // change state before callback.
+        currentState = nextState;
+        if (callback != null) {
+            callback.call(currentState, nextState);
+        }
+    }
+
+    /**
+     * determine the exception
+     *
+     * @return
+     */
+    public boolean isException() {
+        State tmpState = currentState;
+        return State.KILLED.equals(tmpState) || State.FAILED.equals(tmpState) || State.FATAL.equals(tmpState);
+    }
+
+    public boolean isFinished() {
+        State tmpState = currentState;
+        return State.FATAL.equals(tmpState) || State.SUCCEEDED.equals(tmpState) || State.KILLED.equals(tmpState);
+    }
+
+    public boolean isSuccess() {
+        return State.SUCCEEDED.equals(currentState);
+    }
+
+    public boolean isFailed() {
+        return State.FAILED.equals(currentState);
+    }
+
+    public boolean isFatal() {
+        State tmpState = currentState;
+        return State.FATAL.equals(tmpState) || State.KILLED.equals(tmpState);
+    }
+
+}
diff --git a/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/state/State.java b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/state/State.java
new file mode 100755
index 0000000..682dffd
--- /dev/null
+++ b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/state/State.java
@@ -0,0 +1,36 @@
+/*
+ * 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.inlong.agent.state;
+
+/**
+ * job/task state
+ */
+public enum State {
+    // accepted state
+    ACCEPTED,
+    // running
+    RUNNING,
+    // succeeded
+    SUCCEEDED,
+    // failed
+    FAILED,
+    // killed
+    KILLED,
+    // fatal after retry failed
+    FATAL
+}
diff --git a/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/state/StateCallback.java b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/state/StateCallback.java
new file mode 100755
index 0000000..31a2521
--- /dev/null
+++ b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/state/StateCallback.java
@@ -0,0 +1,26 @@
+/*
+ * 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.inlong.agent.state;
+
+/**
+ * callbacks
+ */
+public interface StateCallback {
+
+    void call(State before, State after);
+}
diff --git a/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/state/StateTransferException.java b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/state/StateTransferException.java
new file mode 100755
index 0000000..a934837
--- /dev/null
+++ b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/state/StateTransferException.java
@@ -0,0 +1,26 @@
+/*
+ * 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.inlong.agent.state;
+
+
+public class StateTransferException extends RuntimeException {
+
+    public StateTransferException(State begin, State end) {
+        super(String.format("%s -> %s not allowed", begin, end));
+    }
+}
diff --git a/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/utils/AgentDbUtils.java b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/utils/AgentDbUtils.java
new file mode 100644
index 0000000..e866318
--- /dev/null
+++ b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/utils/AgentDbUtils.java
@@ -0,0 +1,242 @@
+/*
+ * 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.inlong.agent.utils;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.concurrent.TimeUnit;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.commons.dbutils.DbUtils;
+import org.apache.commons.lang3.StringUtils;
+
+public class AgentDbUtils {
+
+    private static final Pattern PATTERN =
+        Pattern.compile("\\$\\{(((0x)|(0X)|o|O)??[0-9a-fA-F]+?) *, "
+            + "*(((0x)|(0X)|o|O)??[0-9a-fA-F]+?) *(, *[0-9]*?)??}");
+    private static final Pattern OCT_PATTERN = Pattern.compile("^o[0-7]+?$");
+    private static final Pattern DEC_PATTERN = Pattern.compile("^[0-9]+?$");
+    private static final int HEX_MODE = 16;
+    private static final int EIGHT_MODE = 8;
+    private static final String HEX_PREFIX = "0x";
+    private static final String O_PREFIX  = "o";
+
+    /**
+     * Attempts to establish a connection to the database from the XML configurations. If failed,
+     * try use alternative standby database.
+     *
+     * @return Database connection
+     */
+    public static Connection getConnectionFailover(String driverClassName,
+            String connectionUrl,
+            String userName,
+            String password) throws Exception {
+
+        int totalRetryTimes = 3;
+        int timeInterval = 10;
+
+        connectionUrl = StringUtils.trim(connectionUrl);
+        connectionUrl = StringUtils.replace(connectionUrl, "\r", "");
+        connectionUrl = StringUtils.replace(connectionUrl, "\n", "");
+
+        /* TODO: try to decrypt password, if failed then use raw password */
+
+        /*
+         * NOTE: THIS MAY CAUSE DEADLOAK WHEN MULTIPLE THREADS CALLED AT THE
+         * SAME TIME
+         * sun.security.jca.ProviderConfig.getProvider(ProviderConfig.java:188)
+         */
+        synchronized (AgentDbUtils.class) {
+            DbUtils.loadDriver(driverClassName);
+        }
+
+        Connection conn = null;
+        int retryTimes = 0;
+        while (conn == null) {
+            try {
+                conn = DriverManager.getConnection(connectionUrl, userName, password);
+            } catch (Exception e) {
+                retryTimes++;
+                if (retryTimes >= totalRetryTimes) {
+                    throw new SQLException(
+                            "Failed to connect database after retry " + retryTimes + " times.", e);
+                }
+                TimeUnit.SECONDS.sleep(timeInterval);
+            }
+        }
+
+        conn.setAutoCommit(false);
+        return conn;
+    }
+
+    private static String format(int num, boolean lengthEquals, int length, int mode) {
+        String numStr;
+        if (mode == HEX_MODE) {
+            numStr = Integer.toHexString(num);
+            /* sub hex head '0x' */
+            length = length - 2;
+        } else if (mode == EIGHT_MODE) {
+            numStr = Integer.toOctalString(num);
+            /* sub oct head 'o' */
+            length = length - 1;
+        } else {
+            numStr = String.valueOf(num);
+        }
+
+        /* append string length for lengthEquals = true */
+        if (lengthEquals) {
+            if (numStr.length() < length) {
+                StringBuilder numberFormatStr = new StringBuilder();
+                for (int i = 0; i < length - numStr.length(); i++) {
+                    numberFormatStr.append(0);
+                }
+                numberFormatStr.append(numStr);
+                numStr = numberFormatStr.toString();
+            }
+        }
+        return numStr;
+    }
+
+    private static int parseInt(String parseStr) {
+
+        int parseValue = -1;
+
+        if (parseStr.startsWith(HEX_PREFIX)) {
+            parseStr = parseStr.substring(2).trim();
+            parseValue = Integer.parseInt(parseStr, 16);
+        } else if (parseStr.startsWith(O_PREFIX)) {
+            parseStr = parseStr.substring(1).trim();
+            parseValue = Integer.parseInt(parseStr, 8);
+        } else {
+            parseValue = Integer.parseInt(parseStr);
+        }
+
+        return parseValue;
+    }
+
+    /**
+     * Transfer string pattern into a list of real string.
+     * For example: ${1, 99} = 1, 2, 3, ... 98,
+     * 99 <br> ${01, 99} = 01, 02, ... 98, 99 <br>
+     *     ${0x0,0xff} = 1, 2, ... fe, ff <br> ${0x00,0xff}
+     * = 01, 02, ... fe, ff <br> ${O1,O10} = 1, 2,... 7, 10<br>
+     *     ${O01,O10} = 01, 02,... 07, 10<br>
+     *
+     * test_${0x00,0x12,5} = test_00, test_05, test_0a, test_0f<br>
+     *
+     * @param str source string
+     * @return string list.
+     */
+    public static String[] replaceDynamicSeq(String str) {
+        if (StringUtils.isBlank(str)) {
+            return null;
+        }
+
+        StringBuffer sb = new StringBuffer();
+        int index = 0;
+        /* find need replace number string */
+        Matcher matcher = PATTERN.matcher(str);
+        ArrayList<String> startNum = new ArrayList<String>();
+        ArrayList<String> endNum = new ArrayList<String>();
+        ArrayList<Integer> modes = new ArrayList<Integer>();
+        ArrayList<Integer> steps = new ArrayList<Integer>();
+        while (matcher.find()) {
+            String matchStr = matcher.group(0);
+            matchStr = StringUtils.strip(matchStr, "${");
+            matchStr = StringUtils.strip(matchStr, "}");
+            String[] patterns = matchStr.split(",");
+            String startStr = patterns[0].trim().toLowerCase();
+            String endStr = patterns[1].trim().toLowerCase();
+            int step = 1;
+            if (patterns.length >= 3) {
+                String stepStr = patterns[2].trim();
+                if (stepStr.length() > 0) {
+                    step = parseInt(stepStr);
+                }
+            }
+
+            boolean bFound = false;
+            int mode = -1;
+
+            /* match hex string */
+            if (startStr.startsWith("0x") && endStr.startsWith("0x")) {
+                bFound = true;
+                mode = 16;
+            } else if (startStr.startsWith("o") && endStr.startsWith("o")) {
+                /* match oct string */
+                Matcher startMatch = OCT_PATTERN.matcher(startStr);
+                Matcher endMatch = OCT_PATTERN.matcher(endStr);
+                if (startMatch.find() && endMatch.find()) {
+                    bFound = true;
+                    mode = 8;
+                }
+            } else {
+                /* match dec string */
+                Matcher startMatch = DEC_PATTERN.matcher(startStr);
+                Matcher endMatch = DEC_PATTERN.matcher(endStr);
+                if (startMatch.find() && endMatch.find()) {
+                    bFound = true;
+                    mode = 10;
+                }
+            }
+
+            /* if not match oct, dec, hex; not do anything */
+            /* if matched, bFound = true */
+            if (bFound) {
+                startNum.add(startStr);
+                endNum.add(endStr);
+                modes.add(mode);
+                steps.add(step);
+                matcher.appendReplacement(sb, "\\${" + (index++) + "}");
+            }
+        }
+        matcher.appendTail(sb);
+        ArrayList<String>[] tempArray = formatStartNum(startNum, endNum, modes, steps, sb);
+        return tempArray[startNum.size()].toArray(new String[0]);
+    }
+
+    private static ArrayList<String>[] formatStartNum(ArrayList<String> startNum,
+        ArrayList<String> endNum,
+        ArrayList<Integer> modes,
+        ArrayList<Integer> steps,
+        StringBuffer sb) {
+        @SuppressWarnings("unchecked")
+        ArrayList<String>[] tempArray = new ArrayList[startNum.size() + 1];
+        tempArray[0] = new ArrayList<String>();
+        tempArray[0].add(sb.toString());
+        for (int index = 0; index < startNum.size(); index++) {
+            String start = startNum.get(index);
+            String end = endNum.get(index);
+            int mode = modes.get(index);
+            int step = steps.get(index);
+            tempArray[index + 1] = new ArrayList<String>();
+            boolean lengthEquals = start.length() == end.length();
+            for (String currentPath : tempArray[index]) {
+                for (int i = parseInt(start); i <= parseInt(end); i = i + step) {
+                    tempArray[index + 1].add(currentPath.replaceAll(
+                        "\\$\\{" + index + "}",
+                        format(i, lengthEquals, end.length(), mode)));
+                }
+            }
+        }
+        return tempArray;
+    }
+}
diff --git a/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/utils/AgentUtils.java b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/utils/AgentUtils.java
new file mode 100644
index 0000000..b35e27a
--- /dev/null
+++ b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/utils/AgentUtils.java
@@ -0,0 +1,363 @@
+/*
+ * 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.inlong.agent.utils;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.InputStream;
+import java.lang.reflect.Field;
+import java.lang.reflect.Method;
+import java.net.DatagramSocket;
+import java.net.InetAddress;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.FileSystems;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+import java.time.ZonedDateTime;
+import java.time.format.DateTimeFormatter;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.TimeZone;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.lang3.tuple.Pair;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class AgentUtils {
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(AgentUtils.class);
+    private static final AtomicLong INDEX = new AtomicLong(0);
+    private static final String HEX_PREFIX = "0x";
+    public static final String EQUAL = "=";
+    public static final String M_VALUE = "m";
+    public static final String ADDITION_SPLITTER = "&";
+    public static final String BEIJING_TIME_ZONE = "GMT+8:00";
+    public static final String HOUR_PATTERN = "yyyyMMddHH";
+    public static final String DAY_PATTERN = "yyyyMMdd";
+    public static final String DEFAULT_PATTERN = "yyyyMMddHHmm";
+    public static final String DAY = "D";
+    public static final String HOUR = "H";
+    public static final String HOUR_LOW_CASE = "h";
+    public static final String MINUTE = "m";
+
+
+    /**
+     * get md5 of file.
+     * @param file - file name
+     * @return
+     */
+    public static String getFileMd5(File file) {
+        try (InputStream is = Files.newInputStream(Paths.get(file.getAbsolutePath()))) {
+            return DigestUtils.md5Hex(is);
+        } catch (Exception ex) {
+            LOGGER.warn("cannot get md5 of {}", file, ex);
+        }
+        return "";
+    }
+
+    /**
+     * finally close resources
+     *
+     * @param resource -  resource which is closable.
+     */
+    public static void finallyClose(Closeable resource) {
+        if (resource != null) {
+            try {
+                resource.close();
+            } catch (Exception ex) {
+                LOGGER.info("error while closing", ex);
+            }
+        }
+    }
+
+    /**
+     * finally close resources.
+     *
+     * @param resource -  resource which is closable.
+     */
+    public static void finallyClose(AutoCloseable resource) {
+        if (resource != null) {
+            try {
+                resource.close();
+            } catch (Exception ex) {
+                LOGGER.error("error while closing", ex);
+            }
+        }
+    }
+
+    /**
+     * Get declare fields.
+     */
+    public static List<Field> getDeclaredFieldsIncludingInherited(Class<?> clazz) {
+        List<Field> fields = new ArrayList<Field>();
+        // check whether parent exists
+        while (clazz != null) {
+            fields.addAll(Arrays.asList(clazz.getDeclaredFields()));
+            clazz = clazz.getSuperclass();
+        }
+        return fields;
+    }
+
+    /**
+     * Get declare methods.
+     *
+     * @param clazz - class of field from method return
+     * @return list of methods
+     */
+    public static List<Method> getDeclaredMethodsIncludingInherited(Class<?> clazz) {
+        List<Method> methods = new ArrayList<Method>();
+        while (clazz != null) {
+            methods.addAll(Arrays.asList(clazz.getDeclaredMethods()));
+            clazz = clazz.getSuperclass();
+        }
+        return methods;
+    }
+
+    /**
+     * get random int of [seed, seed * 2]
+     * @param seed
+     * @return
+     */
+    public static int getRandomBySeed(int seed) {
+        return ThreadLocalRandom.current().nextInt(0, seed) + seed;
+    }
+
+    public static String getLocalIp() {
+        String ip = "127.0.0.1";
+        try (DatagramSocket socket = new DatagramSocket()) {
+            socket.connect(InetAddress.getByName("8.8.8.8"), 10002);
+            ip = socket.getLocalAddress().getHostAddress();
+        } catch (Exception ex) {
+            LOGGER.error("error while get local ip", ex);
+        }
+        return ip;
+    }
+
+    /**
+     * Get uniq id with timestamp.
+     *
+     * @return uniq id.
+     */
+    public static String getUniqId(String prefix, String id) {
+        return getUniqId(prefix, id, 0L);
+    }
+
+    /**
+     * Get uniq id with timestamp and index.
+     * @param id - job id
+     * @param index - job index
+     * @return uniq id
+     */
+    public static String getUniqId(String prefix, String id, long index) {
+        long currentTime = System.currentTimeMillis() / 1000;
+        return  prefix + currentTime + "_" + id + "_" + index;
+    }
+
+    public static void silenceSleepInMs(long millisecond) {
+        try {
+            TimeUnit.MILLISECONDS.sleep(millisecond);
+        } catch (Exception ignored) {
+            LOGGER.warn("silenceSleepInMs ", ignored);
+        }
+    }
+
+    public static String parseHexStr(String delimiter) throws IllegalArgumentException {
+        if (delimiter.trim().toLowerCase().startsWith(HEX_PREFIX)) {
+            //only one char
+            byte[] byteArr = new byte[1];
+            byteArr[0] = Byte.decode(delimiter.trim());
+            return new String(byteArr, StandardCharsets.UTF_8);
+        } else {
+            throw new IllegalArgumentException("delimiter not start with " + HEX_PREFIX);
+        }
+    }
+
+    /**
+     * formatter for current time
+     * @param formatter
+     * @return
+     */
+    public static String formatCurrentTime(String formatter) {
+        return formatCurrentTime(formatter, Locale.getDefault());
+    }
+
+    public static String formatCurrentTime(String formatter, Locale locale) {
+        ZonedDateTime zoned = ZonedDateTime.now();
+        // TODO: locale seems not working
+        return DateTimeFormatter.ofPattern(formatter).withLocale(locale).format(zoned);
+    }
+
+    /**
+     * formatter with time offset
+     * @param formatter - formatter string
+     * @param day - day offset
+     * @param hour - hour offset
+     * @param min - min offset
+     * @return current time with offset
+     */
+    public static String formatCurrentTimeWithOffset(String formatter, int day, int hour, int min) {
+        ZonedDateTime zoned = ZonedDateTime.now().plusDays(day).plusHours(hour).plusMinutes(min);
+        return DateTimeFormatter.ofPattern(formatter).withLocale(Locale.getDefault()).format(zoned);
+    }
+
+
+    public static String formatCurrentTimeWithoutOffset(String formatter) {
+        ZonedDateTime zoned = ZonedDateTime.now().plusDays(0).plusHours(0).plusMinutes(0);
+        return DateTimeFormatter.ofPattern(formatter).withLocale(Locale.getDefault()).format(zoned);
+    }
+
+    /**
+     * whether all class of path name are matched
+     *
+     * @param pathStr - path string
+     * @param patternStr - regex pattern
+     * @return true if all match
+     */
+    public static boolean regexMatch(String pathStr, String patternStr) {
+        String[] pathNames = StringUtils.split(pathStr, FileSystems.getDefault().getSeparator());
+        String[] patternNames = StringUtils
+                .split(patternStr, FileSystems.getDefault().getSeparator());
+        for (int i = 0; i < pathNames.length && i < patternNames.length; i++) {
+            if (!pathNames[i].equals(patternNames[i])) {
+                Matcher matcher = Pattern.compile(patternNames[i]).matcher(pathNames[i]);
+                if (!matcher.matches()) {
+                    return false;
+                }
+            }
+        }
+        return true;
+    }
+
+    /**
+     * parse addition attr, the attributes must be send in proxy sender
+     * @param additionStr
+     * @return
+     */
+    public static Pair<String, Map<String, String>> parseAddAttr(String additionStr) {
+        Map<String, String> attr = new HashMap<>();
+        String[] split = additionStr.split(ADDITION_SPLITTER);
+        String mValue = "";
+        for (String s : split) {
+            if (!s.contains(EQUAL)) {
+                continue;
+            }
+            String[] pairs = s.split(EQUAL);
+            if (pairs[0].equalsIgnoreCase(M_VALUE)) {
+                mValue = pairs[1];
+                continue;
+            }
+            getAttrs(attr, s, pairs);
+        }
+        return Pair.of(mValue, attr);
+    }
+
+    /**
+     * the attrs in pairs can be complicated in online env
+     * @param attr
+     * @param s
+     * @param pairs
+     */
+    private static void getAttrs(Map<String, String> attr, String s, String[] pairs) {
+        // when addiction attr be like "m=10&__addcol1__worldid="
+        if (s.endsWith(EQUAL) && pairs.length == 1) {
+            attr.put(pairs[0], "");
+        } else {
+            attr.put(pairs[0], pairs[1]);
+        }
+    }
+
+    /**
+     * get addition attributes in additionStr
+     * @param additionStr
+     * @return
+     */
+    public static Map<String, String> getAdditionAttr(String additionStr) {
+        Pair<String, Map<String, String>> mValueAttrs = parseAddAttr(additionStr);
+        return mValueAttrs.getRight();
+    }
+
+
+    /**
+     * get m value in additionStr
+     * @param addictiveAttr
+     * @return
+     */
+    public static String getmValue(String addictiveAttr) {
+        Pair<String, Map<String, String>> mValueAttrs = parseAddAttr(addictiveAttr);
+        return mValueAttrs.getLeft();
+    }
+
+    /**
+     * time str convert to mill sec
+     * @param time
+     * @param cycleUnit
+     * @return
+     */
+    public static long timeStrConvertToMillSec(String time, String cycleUnit) {
+        long defaultTime = System.currentTimeMillis();
+        if (time.isEmpty() || cycleUnit.isEmpty()) {
+            return defaultTime;
+        }
+        String pattern = DEFAULT_PATTERN;
+        switch (cycleUnit) {
+            case DAY:
+                pattern = DAY_PATTERN;
+                time = time.substring(0, 8);
+                break;
+            case HOUR:
+            case HOUR_LOW_CASE:
+                pattern = HOUR_PATTERN;
+                time = time.substring(0, 10);
+                break;
+            case MINUTE:
+                break;
+            default:
+                LOGGER.error("cycle unit {} is illegal, please check!", cycleUnit);
+                break;
+        }
+        return parseTimeToMillSec(time, pattern);
+    }
+
+    private static long parseTimeToMillSec(String time, String pattern) {
+        try {
+            SimpleDateFormat df = new SimpleDateFormat(pattern);
+            df.setTimeZone(TimeZone.getTimeZone(BEIJING_TIME_ZONE));
+            return df.parse(time).getTime();
+        } catch (ParseException e) {
+            LOGGER.error("convert time string {} to millSec error", time);
+        }
+        return System.currentTimeMillis();
+    }
+
+
+
+
+
+}
diff --git a/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/utils/ByteUtil.java b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/utils/ByteUtil.java
new file mode 100644
index 0000000..bbc378f
--- /dev/null
+++ b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/utils/ByteUtil.java
@@ -0,0 +1,237 @@
+/*
+ * 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.inlong.agent.utils;
+
+import java.util.ArrayList;
+import java.util.List;
+
+public class ByteUtil {
+    /**
+     * Splits the source array into multiple array segments using the given separator, up to a
+     * maximum of count items. This will naturally produce copied byte arrays for each of the split
+     * segments. To identify the split ranges without the array copies, see {@link
+     * ByteUtil#splitRanges(byte[], byte[])}.
+     *
+     * @param source
+     * @param separator
+     * @return
+     */
+    public static byte[][] split(byte[] source, byte[] separator) {
+        return split(source, separator, -1);
+    }
+
+    /**
+     * Splits the source array into multiple array segments using the given separator, up to a
+     * maximum of count items. This will naturally produce copied byte arrays for each of the split
+     * segments. To identify the split ranges without the array copies, see {@link
+     * ByteUtil#splitRanges(byte[], byte[])}.
+     *
+     * @param source
+     * @param separator
+     * @return
+     */
+    public static byte[][] split(byte[] source, byte[] separator, int limit) {
+        List<Range> segments = splitRanges(source, separator, limit);
+
+        byte[][] splits = new byte[segments.size()][];
+        for (int i = 0; i < segments.size(); i++) {
+            Range r = segments.get(i);
+            byte[] tmp = new byte[r.length()];
+            if (tmp.length > 0) {
+                System.arraycopy(source, r.start(), tmp, 0, r.length());
+            }
+            splits[i] = tmp;
+        }
+        return splits;
+    }
+
+    /**
+     * Returns a list of ranges identifying [start, end) -- closed, open -- positions within the
+     * source byte array that would be split using the separator byte array.
+     */
+    public static List<Range> splitRanges(byte[] source, byte[] separator) {
+        return splitRanges(source, separator, -1);
+    }
+
+    /**
+     * Returns a list of ranges identifying [start, end) -- closed, open -- positions within the
+     * source byte array that would be split using the separator byte array.
+     *
+     * @param source the source data
+     * @param separator the separator pattern to look for
+     * @param limit the maximum number of splits to identify in the source
+     */
+    public static List<Range> splitRanges(byte[] source, byte[] separator, int limit) {
+        List<Range> segments = new ArrayList<Range>();
+        int start = 0;
+        itersource:
+        for (int i = 0; i < source.length; i++) {
+            for (int j = 0; j < separator.length; j++) {
+                if (source[i + j] != separator[j]) {
+                    continue itersource;
+                }
+            }
+            // all separator elements matched
+            if (limit > 0 && segments.size() >= (limit - 1)) {
+                // everything else goes in one final segment
+                break;
+            }
+
+            segments.add(new Range(start, i));
+            start = i + separator.length;
+            // i will be incremented again in outer for loop
+            i += separator.length - 1;
+        }
+        // add in remaining to a final range
+        if (start <= source.length) {
+            segments.add(new Range(start, source.length));
+        }
+        return segments;
+    }
+
+    /**
+     * Returns a single byte array containing all of the individual component arrays separated by
+     * the separator array.
+     *
+     * @param separator
+     * @param components
+     * @return
+     */
+    public static byte[] join(byte[] separator, byte[]... components) {
+        if (components == null || components.length == 0) {
+            return new byte[0];
+        }
+
+        int finalSize = 0;
+        if (separator != null) {
+            finalSize = separator.length * (components.length - 1);
+        }
+        for (byte[] comp : components) {
+            finalSize += comp.length;
+        }
+
+        byte[] buf = new byte[finalSize];
+        int offset = 0;
+        for (int i = 0; i < components.length; i++) {
+            System.arraycopy(components[i], 0, buf, offset, components[i].length);
+            offset += components[i].length;
+            if (i < (components.length - 1) && separator != null && separator.length > 0) {
+                System.arraycopy(separator, 0, buf, offset, separator.length);
+                offset += separator.length;
+            }
+        }
+        return buf;
+    }
+
+    /**
+     * Returns the index (start position) of the first occurrence of the specified {@code target}
+     * within {@code array} starting at {@code fromIndex} , or {@code -1} if there is no such
+     * occurrence.
+     *
+     * <p>
+     * Returns the lowest index {@code k} such that {@code k >= fromIndex} and {@code
+     * java.util.Arrays.copyOfRange(array, k, k + target.length)} contains exactly the same elements
+     * as {@code target}.
+     *
+     * @param array the array to search for the sequence {@code target}
+     * @param target the array to search for as a sub-sequence of {@code array}
+     * @param fromIndex the index to start the search from in {@code array}
+     */
+    public static int indexOf(byte[] array, byte[] target, int fromIndex) {
+
+        if (array == null || target == null) {
+            return -1;
+        }
+
+        // Target cannot be beyond array boundaries
+        if (fromIndex < 0 || (fromIndex > (array.length - target.length))) {
+            return -1;
+        }
+
+        // Empty is assumed to be at the fromIndex of any non-null array (after
+        // boundary check)
+        if (target.length == 0) {
+            return fromIndex;
+        }
+
+        firstbyte:
+        for (int i = fromIndex; i < array.length - target.length + 1; i++) {
+            for (int j = 0; j < target.length; j++) {
+                if (array[i + j] != target[j]) {
+                    continue firstbyte;
+                }
+            }
+            return i;
+        }
+        return -1;
+    }
+
+    /**
+     * Returns a copy of the source byte array, starting at offset for the given length.  If the
+     * offset + length is out of bounds for the array, returns null.
+     *
+     * @param source
+     * @param offset
+     * @param length
+     * @return
+     */
+    public static byte[] safeCopy(byte[] source, int offset, int length) {
+        if (length < 0 || source.length < offset + length) {
+            return null;
+        }
+        byte[] copy = new byte[length];
+        System.arraycopy(source, offset, copy, 0, length);
+        return copy;
+    }
+
+    public static class Range {
+
+        private int startIdx;
+        private int endIdx;
+
+        /**
+         * Defines a range from start index (inclusive) to end index (exclusive).
+         *
+         * @param start Starting index position
+         * @param end Ending index position (exclusive)
+         */
+        public Range(int start, int end) {
+            if (start < 0 || end < start) {
+                throw new IllegalArgumentException(
+                    "Invalid range, required that: 0 <= start <= end; start=" + start
+                        + ", end=" + end);
+            }
+
+            this.startIdx = start;
+            this.endIdx = end;
+        }
+
+        public int start() {
+            return startIdx;
+        }
+
+        public int end() {
+            return endIdx;
+        }
+
+        public int length() {
+            return endIdx - startIdx;
+        }
+    }
+}
diff --git a/inlong-agent/agent-common/src/test/java/org/apache/inlong/agent/AgentBaseTestsHelper.java b/inlong-agent/agent-common/src/test/java/org/apache/inlong/agent/AgentBaseTestsHelper.java
new file mode 100755
index 0000000..398e7d2
--- /dev/null
+++ b/inlong-agent/agent-common/src/test/java/org/apache/inlong/agent/AgentBaseTestsHelper.java
@@ -0,0 +1,65 @@
+/*
+ * 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.inlong.agent;
+
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import org.apache.commons.io.FileUtils;
+import org.apache.inlong.agent.conf.AgentConfiguration;
+import org.apache.inlong.agent.constants.AgentConstants;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * common environment setting up for test cases.
+ */
+public class AgentBaseTestsHelper {
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(AgentBaseTestsHelper.class);
+
+    private final String className;
+    private Path testRootDir;
+
+    public AgentBaseTestsHelper(String className) {
+        this.className = className;
+    }
+
+    public AgentBaseTestsHelper setupAgentHome() {
+        testRootDir = Paths
+                .get("/tmp", AgentBaseTestsHelper.class.getSimpleName(), className);
+        teardownAgentHome();
+        boolean result = testRootDir.toFile().mkdirs();
+        LOGGER.info("try to create {}, result is {}", testRootDir, result);
+        AgentConfiguration.getAgentConf().set(AgentConstants.AGENT_HOME, testRootDir.toString());
+        return this;
+    }
+
+    public Path getTestRootDir() {
+        return testRootDir;
+    }
+
+    public void teardownAgentHome() {
+        if (testRootDir != null) {
+            try {
+                FileUtils.deleteDirectory(testRootDir.toFile());
+            } catch (Exception ignored) {
+                LOGGER.warn("deleteDirectory error ", ignored);
+            }
+        }
+    }
+}
diff --git a/inlong-agent/agent-common/src/test/java/org/apache/inlong/agent/common/TestAgentThreadFactory.java b/inlong-agent/agent-common/src/test/java/org/apache/inlong/agent/common/TestAgentThreadFactory.java
new file mode 100755
index 0000000..5ccdb93
--- /dev/null
+++ b/inlong-agent/agent-common/src/test/java/org/apache/inlong/agent/common/TestAgentThreadFactory.java
@@ -0,0 +1,42 @@
+/*
+ * 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.inlong.agent.common;
+
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import org.junit.Assert;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class TestAgentThreadFactory {
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(TestAgentThreadFactory.class);
+
+    @Test
+    public void testThreadRename() throws Exception {
+        ExecutorService executor = Executors
+                .newSingleThreadExecutor(new AgentThreadFactory("test"));
+        Future<?> result = executor.submit(() -> {
+            Assert.assertEquals("test-running-thread-1", Thread.currentThread().getName());
+            LOGGER.info("thread finished");
+        });
+        result.get();
+    }
+}
diff --git a/inlong-agent/agent-common/src/test/java/org/apache/inlong/agent/common/TestAgentUtils.java b/inlong-agent/agent-common/src/test/java/org/apache/inlong/agent/common/TestAgentUtils.java
new file mode 100755
index 0000000..b587960
--- /dev/null
+++ b/inlong-agent/agent-common/src/test/java/org/apache/inlong/agent/common/TestAgentUtils.java
@@ -0,0 +1,88 @@
+/*
+ * 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.inlong.agent.common;
+
+import java.util.Map;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.inlong.agent.utils.AgentDbUtils;
+import org.apache.inlong.agent.utils.AgentUtils;
+import org.junit.Assert;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class TestAgentUtils {
+    private static final Logger LOGGER = LoggerFactory.getLogger(TestAgentUtils.class);
+
+    @Test
+    public void testReplaceDynamicSequence() throws Exception {
+        String[] result = AgentDbUtils.replaceDynamicSeq("${1, 99}");
+        assert result != null;
+        String[] expectResult = new String[99];
+        for (int index = 1; index < 100; index++) {
+            expectResult[index - 1] = String.valueOf(index);
+        }
+        Assert.assertArrayEquals(expectResult, result);
+
+        result = AgentDbUtils.replaceDynamicSeq("${0x0, 0xf}");
+        expectResult = new String[]{"0", "1", "2", "3", "4", "5", "6", "7", "8", "9", "a",
+            "b", "c", "d", "e", "f"};
+        Assert.assertArrayEquals(expectResult, result);
+
+        result = AgentDbUtils.replaceDynamicSeq("${O01,O10}");
+        expectResult = new String[]{"01", "02", "03", "04", "05",
+            "06", "07", "10"};
+        Assert.assertArrayEquals(expectResult, result);
+    }
+
+    @Test
+    public void testDateFormatter() {
+        String time = AgentUtils.formatCurrentTime("yyyyMMdd HH:mm:ss");
+        LOGGER.info("agent time is {}", time);
+    }
+
+    @Test
+    public void testDateFormatterWithOffset() {
+        String timeOffset = "-1d";
+        String number = StringUtils.substring(timeOffset, 0, timeOffset.length() - 1);
+        String mark = StringUtils.substring(timeOffset, timeOffset.length() - 1);
+        Assert.assertEquals("-1", number);
+        Assert.assertEquals("d", mark);
+        String time = AgentUtils.formatCurrentTimeWithOffset("yyyyMMdd HH:mm:ss", 1, 1, 1);
+        LOGGER.info("agent time is {}", time);
+        time = AgentUtils.formatCurrentTimeWithOffset("yyyyMMdd HH:mm:ss", -1, -1, -1);
+        LOGGER.info("agent time is {}", time);
+    }
+
+
+    @Test
+    public void testParseAddictiveStr() {
+        String addStr = "m=10&__addcol1__worldid=&t=1";
+        Map<String, String> attr = AgentUtils.getAdditionAttr(addStr);
+        Assert.assertEquals("", attr.get("__addcol1__worldid"));
+        Assert.assertEquals("1", attr.get("t"));
+    }
+
+    @Test
+    public void testTimeConvertToMillsec() {
+        Assert.assertEquals(1620316800000L, AgentUtils.timeStrConvertToMillSec("202105071554", "D"));
+        Assert.assertEquals(1620370800000L, AgentUtils.timeStrConvertToMillSec("202105071554", "H"));
+        Assert.assertEquals(1620374040000L, AgentUtils.timeStrConvertToMillSec("202105071554", "M"));
+    }
+
+}
diff --git a/inlong-agent/agent-common/src/test/java/org/apache/inlong/agent/conf/TestConfiguration.java b/inlong-agent/agent-common/src/test/java/org/apache/inlong/agent/conf/TestConfiguration.java
new file mode 100755
index 0000000..0f1b93f
--- /dev/null
+++ b/inlong-agent/agent-common/src/test/java/org/apache/inlong/agent/conf/TestConfiguration.java
@@ -0,0 +1,73 @@
+/*
+ * 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.inlong.agent.conf;
+
+import static org.junit.Assert.assertEquals;
+
+import org.apache.inlong.agent.constants.JobConstants;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestConfiguration {
+
+    private static AgentConfiguration agentConf;
+    private static JobProfile jobConf;
+    private static JobProfile jobJsonConf;
+
+    @BeforeClass
+    public static void setup() {
+        agentConf = AgentConfiguration.getAgentConf();
+        jobConf = JobProfile.parsePropertiesFile("job.properties");
+        jobJsonConf = JobProfile.parseJsonFile("job.json");
+    }
+
+    @Test
+    public void testGetInt() throws Exception {
+        assertEquals(10, agentConf.getInt("agent.maxSize", 15));
+        assertEquals(15, agentConf.getInt("key.not.exists", 15));
+    }
+
+    @Test
+    public void testGetLong() throws Exception {
+        assertEquals(200L, agentConf.getLong("agent.maxBuff", 15));
+        agentConf.setLong("agent.maxBuff", 20);
+        assertEquals(20L, agentConf.getLong("agent.maxBuff", 15));
+        assertEquals(15L, agentConf.getLong("key.not.exists", 15));
+    }
+
+    @Test
+    public void testString() throws Exception {
+        assertEquals("manager", agentConf.get("agent.conf.resource", "none"));
+        agentConf.set("agent.conf.resource", "selfDefine");
+        assertEquals("selfDefine", agentConf.get("agent.conf.resource", "none"));
+        assertEquals("none", agentConf.get("key.not.exists", "none"));
+    }
+
+    @Test
+    public void testJobConf() throws Exception {
+        Assert.assertTrue(jobConf.allRequiredKeyExist());
+        assertEquals("1", jobConf.get(JobConstants.JOB_ID));
+        assertEquals("test", jobConf.get(JobConstants.JOB_NAME));
+
+        Assert.assertTrue(jobJsonConf.allRequiredKeyExist());
+        assertEquals("1", jobJsonConf.get(JobConstants.JOB_ID));
+        assertEquals("test", jobJsonConf.get(JobConstants.JOB_NAME));
+    }
+
+}
diff --git a/inlong-agent/agent-common/src/test/java/org/apache/inlong/agent/db/TestBerkeleyDBImp.java b/inlong-agent/agent-common/src/test/java/org/apache/inlong/agent/db/TestBerkeleyDBImp.java
new file mode 100755
index 0000000..f386468
--- /dev/null
+++ b/inlong-agent/agent-common/src/test/java/org/apache/inlong/agent/db/TestBerkeleyDBImp.java
@@ -0,0 +1,135 @@
+/*
+ * 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.inlong.agent.db;
+
+import java.util.List;
+import org.apache.inlong.agent.AgentBaseTestsHelper;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+
+public class TestBerkeleyDBImp {
+
+    private static BerkeleyDbImp db;
+    private static AgentBaseTestsHelper helper;
+
+    @BeforeClass
+    public static void setup() throws Exception {
+        helper = new AgentBaseTestsHelper(TestBerkeleyDBImp.class.getName()).setupAgentHome();
+        db = new BerkeleyDbImp();
+    }
+
+    @AfterClass
+    public static void teardown() throws Exception {
+        db.close();
+        helper.teardownAgentHome();
+    }
+
+    @Test
+    public void testKeyValueDB() {
+        KeyValueEntity entity = new KeyValueEntity("test1", "testA", "test");
+        db.put(entity);
+        KeyValueEntity ret = db.get("test1");
+        Assert.assertEquals("test1", ret.getKey());
+        Assert.assertEquals("testA", ret.getJsonValue());
+
+        db.remove("test1");
+        ret = db.get("test1");
+        Assert.assertNull(ret);
+
+        StateSearchKey keys = StateSearchKey.SUCCESS;
+        KeyValueEntity entity1 = new KeyValueEntity("test2", "testA", "test");
+        entity.setStateSearchKey(keys);
+        entity1.setStateSearchKey(keys);
+
+        db.set(entity);
+        db.set(entity1);
+
+        List<KeyValueEntity> entityList = db.search(keys);
+        for (KeyValueEntity keyValueEntity : entityList) {
+            Assert.assertEquals(StateSearchKey.SUCCESS, keyValueEntity.getStateSearchKey());
+        }
+        Assert.assertEquals(2, entityList.size());
+
+        entity.setJsonValue("testC");
+        KeyValueEntity oldEntity = db.put(entity);
+        Assert.assertEquals("testA", oldEntity.getJsonValue());
+
+        KeyValueEntity newEntity = db.get("test1");
+        Assert.assertEquals("testC", newEntity.getJsonValue());
+
+    }
+
+
+    @Test
+    public void testCommandDb() {
+        CommandEntity commandEntity = new CommandEntity("1", 0, false, "1", "");
+        db.putCommand(commandEntity);
+        CommandEntity command = db.getCommand("1");
+        Assert.assertEquals("1", command.getId());
+        List<CommandEntity> commandEntities = db.searchCommands(false);
+        Assert.assertEquals("1", commandEntities.get(0).getId());
+    }
+
+    @Test
+    public void testSecondaryIndex() {
+        KeyValueEntity entity = new KeyValueEntity("searchKey1", "searchResult1", "test");
+        db.put(entity);
+        KeyValueEntity entity1 = new KeyValueEntity("searchKey2", "searchResult2", "test");
+        db.put(entity1);
+        KeyValueEntity entityResult = db.searchOne(StateSearchKey.ACCEPTED);
+        Assert.assertEquals("searchKey1", entityResult.getKey());
+
+        entityResult = db.searchOne(StateSearchKey.ACCEPTED);
+        Assert.assertEquals("searchKey1", entityResult.getKey());
+
+        entityResult.setStateSearchKey(StateSearchKey.RUNNING);
+        db.put(entityResult);
+
+        entityResult = db.searchOne(StateSearchKey.ACCEPTED);
+        Assert.assertEquals("searchKey2", entityResult.getKey());
+
+        List<KeyValueEntity> entityList = db.search(StateSearchKey.ACCEPTED);
+        Assert.assertEquals(1, entityList.size());
+
+        entityList = db.search(StateSearchKey.FAILED);
+        Assert.assertEquals(0, entityList.size());
+    }
+
+    @Test
+    public void testDeleteItem() {
+        KeyValueEntity entity = new KeyValueEntity("searchKey1", "searchResult1", "test");
+        db.put(entity);
+        KeyValueEntity entityResult1 = db.remove("searchKey1");
+        KeyValueEntity entityResult = db.searchOne(StateSearchKey.ACCEPTED);
+        Assert.assertEquals("searchKey1", entityResult1.getKey());
+        Assert.assertNull(entityResult);
+    }
+
+
+    @Test
+    public void testFileNameSearch() {
+        KeyValueEntity entity = new KeyValueEntity("searchKey1", "searchResult1", "test");
+        db.put(entity);
+        KeyValueEntity entityResult = db.searchOne("test");
+        Assert.assertEquals("searchKey1", entityResult.getKey());
+    }
+
+}
diff --git a/inlong-agent/agent-common/src/test/java/org/apache/inlong/agent/metrics/TestMetrics.java b/inlong-agent/agent-common/src/test/java/org/apache/inlong/agent/metrics/TestMetrics.java
new file mode 100755
index 0000000..9f6a712
--- /dev/null
+++ b/inlong-agent/agent-common/src/test/java/org/apache/inlong/agent/metrics/TestMetrics.java
@@ -0,0 +1,113 @@
+/*
+ * 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.inlong.agent.metrics;
+
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.inlong.agent.metrics.counter.CounterInt;
+import org.apache.inlong.agent.metrics.counter.CounterLong;
+import org.apache.inlong.agent.metrics.gauge.GaugeInt;
+import org.apache.inlong.agent.metrics.gauge.GaugeLong;
+import org.junit.Assert;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class TestMetrics {
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(TestMetrics.class);
+
+    @Test
+    public void testErrorMetric() throws Exception {
+        try {
+            ErrorMetric errorMetric = ErrorMetric.getMetrics();
+            Assert.fail("Error metric should fail");
+        } catch (Exception ex) {
+            LOGGER.info("error happens" + ex);
+        }
+    }
+
+    @Test
+    public void testMetric() throws Exception {
+        MetricTest metricTest = MetricTest.getMetrics();
+        Assert.assertNotNull(metricTest.counterInt);
+        Assert.assertNotNull(metricTest.counterLong);
+        Assert.assertNotNull(metricTest.gaugeInt);
+        Assert.assertNotNull(metricTest.gaugeLong);
+
+        metricTest.counterInt.incr();
+        metricTest.counterInt.incr();
+
+        Assert.assertEquals(2, metricTest.counterInt.snapshot().intValue());
+
+        metricTest.gaugeLong.incr();
+        metricTest.gaugeLong.incr();
+        metricTest.gaugeLong.decr();
+
+        Assert.assertEquals(1L, metricTest.gaugeLong.snapshot().longValue());
+    }
+
+    @Metrics
+    static class MetricTest {
+
+        private static final AtomicBoolean IS_INITED = new AtomicBoolean(false);
+        private static MetricTest metrics;
+        @Metric
+        CounterLong counterLong;
+
+        @Metric
+        CounterInt counterInt;
+
+        @Metric
+        GaugeLong gaugeLong;
+
+        @Metric
+        GaugeInt gaugeInt;
+
+        private MetricTest() {
+            MetricsRegister.register("Test", "TestMetrics", null, this);
+        }
+
+        public static MetricTest getMetrics() {
+            if (IS_INITED.compareAndSet(false, true)) {
+                metrics = new MetricTest();
+            }
+            return metrics;
+        }
+    }
+
+    @Metrics
+    static class ErrorMetric {
+
+        private static final AtomicBoolean IS_INITED = new AtomicBoolean(false);
+        private static ErrorMetric metrics;
+        @Metric
+        int errorInt;
+
+        private ErrorMetric() {
+            MetricsRegister.register("Test", "ErrorMetric", null, this);
+        }
+
+        public static ErrorMetric getMetrics() {
+            if (IS_INITED.compareAndSet(false, true)) {
+                metrics = new ErrorMetric();
+            }
+            return metrics;
+        }
+    }
+
+}
diff --git a/inlong-agent/agent-common/src/test/resources/agent.properties b/inlong-agent/agent-common/src/test/resources/agent.properties
new file mode 100755
index 0000000..2f0118e
--- /dev/null
+++ b/inlong-agent/agent-common/src/test/resources/agent.properties
@@ -0,0 +1,25 @@
+#
+# 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.
+#
+# default values of agent
+agent.version=0.0.1-SNAPSHOT
+agent.maxSize=10
+agent.maxBuff=200
+agent.conf.resource=manager
+job.thread.running.core=10
+agent.manager.vip.http.host=
+agent.manager.vip.http.port=
+agent.fetcher.classname=org.apache.inlong.agent.plugin.fetcher.ManagerFetcher
\ No newline at end of file
diff --git a/inlong-agent/agent-common/src/test/resources/job.json b/inlong-agent/agent-common/src/test/resources/job.json
new file mode 100644
index 0000000..ca53def
--- /dev/null
+++ b/inlong-agent/agent-common/src/test/resources/job.json
@@ -0,0 +1,11 @@
+
+{
+  "job": {
+    "id": 1,
+    "name": "test",
+    "source": "test.source",
+    "plugin": "test.plugin",
+    "sink": "test.sink",
+    "channel": "test.channel"
+  }
+}
\ No newline at end of file
diff --git a/inlong-agent/agent-common/src/test/resources/job.properties b/inlong-agent/agent-common/src/test/resources/job.properties
new file mode 100644
index 0000000..147e21b
--- /dev/null
+++ b/inlong-agent/agent-common/src/test/resources/job.properties
@@ -0,0 +1,23 @@
+#
+# 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.
+#
+
+job.id = 1
+job.name = test
+job.plugin = test.plugin
+job.source = test.source
+job.sink = test.sink
+job.channel = test.channel
\ No newline at end of file
diff --git a/inlong-agent/agent-common/src/test/resources/log4j.properties b/inlong-agent/agent-common/src/test/resources/log4j.properties
new file mode 100644
index 0000000..25675f6
--- /dev/null
+++ b/inlong-agent/agent-common/src/test/resources/log4j.properties
@@ -0,0 +1,21 @@
+#
+# 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.
+#
+
+log4j.rootLogger=INFO, out
+log4j.appender.out=org.apache.log4j.ConsoleAppender
+log4j.appender.out.layout=org.apache.log4j.PatternLayout
+log4j.appender.out.layout.ConversionPattern=%d (%t) [%p - %l] %m%n
\ No newline at end of file
diff --git a/inlong-agent/agent-core/pom.xml b/inlong-agent/agent-core/pom.xml
new file mode 100755
index 0000000..cf30e12
--- /dev/null
+++ b/inlong-agent/agent-core/pom.xml
@@ -0,0 +1,64 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+    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.
+-->
+<project xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+    xmlns="http://maven.apache.org/POM/4.0.0"
+    xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <parent>
+        <groupId>org.apache.inlong</groupId>
+        <artifactId>inlong-agent</artifactId>
+        <version>0.9.0-incubating-SNAPSHOT</version>
+    </parent>
+    <artifactId>agent-core</artifactId>
+    <modelVersion>4.0.0</modelVersion>
+    <name>Apache InLong - Agent Core</name>
+
+    <dependencies>
+        <dependency>
+            <artifactId>junit</artifactId>
+            <groupId>junit</groupId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <artifactId>awaitility</artifactId>
+            <groupId>org.awaitility</groupId>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.eclipse.jetty</groupId>
+            <artifactId>jetty-server</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.eclipse.jetty</groupId>
+            <artifactId>jetty-servlet</artifactId>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.httpcomponents</groupId>
+            <artifactId>httpclient</artifactId>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.inlong</groupId>
+            <artifactId>agent-common</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+    </dependencies>
+</project>
\ No newline at end of file
diff --git a/inlong-agent/agent-core/src/main/java/org/apache/inlong/agent/core/AgentMain.java b/inlong-agent/agent-core/src/main/java/org/apache/inlong/agent/core/AgentMain.java
new file mode 100644
index 0000000..9fa5e09
--- /dev/null
+++ b/inlong-agent/agent-core/src/main/java/org/apache/inlong/agent/core/AgentMain.java
@@ -0,0 +1,121 @@
+/*
+ * 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.inlong.agent.core;
+
+import java.util.Iterator;
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.CommandLineParser;
+import org.apache.commons.cli.DefaultParser;
+import org.apache.commons.cli.HelpFormatter;
+import org.apache.commons.cli.Option;
+import org.apache.commons.cli.Options;
+import org.apache.inlong.agent.conf.AgentConfiguration;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Agent entrance class
+ */
+public class AgentMain {
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(AgentMain.class);
+
+    /**
+     * Print help information
+     *
+     * @param opts - options
+     */
+    private static void help(Options opts) {
+        HelpFormatter formatter = new HelpFormatter();
+        formatter.printHelp("inlong-agent", opts);
+        System.exit(0);
+    }
+
+    /**
+     * Init options
+     *
+     * @param args - argument
+     * @return command line
+     */
+    public static CommandLine initOptions(String[] args) {
+        CommandLineParser parser = new DefaultParser();
+        Options options = new Options();
+        options.addOption("h", "help", false, "show help");
+        try {
+            return parser.parse(options, args);
+        } catch (Exception ex) {
+            help(options);
+        }
+        return null;
+    }
+
+    /**
+     * Init agent conf
+     *
+     * @param cl - commandline
+     */
+    public static void initAgentConf(CommandLine cl) {
+        AgentConfiguration conf = AgentConfiguration.getAgentConf();
+        Iterator<Option> iterator = cl.iterator();
+        while (iterator != null && iterator.hasNext()) {
+            Option opt = iterator.next();
+            if (opt != null && opt.getLongOpt() != null
+                    && opt.getValue() != null && conf.hasKey(opt.getLongOpt())) {
+                conf.set(opt.getLongOpt(), opt.getValue().trim());
+            }
+        }
+    }
+
+    /**
+     * Stopping agent gracefully if get killed.
+     *
+     * @param manager - agent manager
+     */
+    private static void stopManagerIfKilled(AgentManager manager) {
+        Runtime.getRuntime().addShutdownHook(new Thread(() -> {
+            try {
+                LOGGER.info("stopping agent gracefully");
+                manager.stop();
+            } catch (Exception ex) {
+                LOGGER.error("exception while stopping threads", ex);
+            }
+        }));
+    }
+
+    /**
+     * Main entrance.
+     *
+     * @param args - arguments
+     * @throws Exception exceptions
+     */
+    public static void main(String[] args) throws Exception {
+        CommandLine cl = initOptions(args);
+        assert cl != null;
+        initAgentConf(cl);
+        AgentManager manager = new AgentManager();
+        try {
+            manager.start();
+            stopManagerIfKilled(manager);
+            manager.join();
+        } catch (Exception ex) {
+            LOGGER.error("exception caught", ex);
+        } finally {
+            manager.stop();
+        }
+    }
+}
diff --git a/inlong-agent/agent-core/src/main/java/org/apache/inlong/agent/core/AgentManager.java b/inlong-agent/agent-core/src/main/java/org/apache/inlong/agent/core/AgentManager.java
new file mode 100755
index 0000000..751486a
--- /dev/null
+++ b/inlong-agent/agent-core/src/main/java/org/apache/inlong/agent/core/AgentManager.java
@@ -0,0 +1,206 @@
+/*
+ * 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.inlong.agent.core;
+
+
+import static org.apache.inlong.agent.constants.AgentConstants.AGENT_CONF_PARENT;
+import static org.apache.inlong.agent.constants.AgentConstants.DEFAULT_AGENT_CONF_PARENT;
+import static org.apache.inlong.agent.constants.JobConstants.JOB_TRIGGER;
+
+import java.lang.reflect.Constructor;
+import java.util.List;
+import org.apache.inlong.agent.common.AbstractDaemon;
+import org.apache.inlong.agent.conf.AgentConfiguration;
+import org.apache.inlong.agent.conf.JobProfile;
+import org.apache.inlong.agent.conf.ProfileFetcher;
+import org.apache.inlong.agent.conf.TriggerProfile;
+import org.apache.inlong.agent.constants.AgentConstants;
+import org.apache.inlong.agent.core.conf.ConfigJetty;
+import org.apache.inlong.agent.core.job.JobManager;
+import org.apache.inlong.agent.core.task.TaskManager;
+import org.apache.inlong.agent.core.task.TaskPositionManager;
+import org.apache.inlong.agent.core.trigger.TriggerManager;
+import org.apache.inlong.agent.db.CommandDb;
+import org.apache.inlong.agent.db.Db;
+import org.apache.inlong.agent.db.JobProfileDb;
+import org.apache.inlong.agent.db.LocalProfile;
+import org.apache.inlong.agent.db.TriggerProfileDb;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Agent Manager, the bridge for job manager, task manager, db e.t.c it manages agent level
+ * operations and communicates with outside system.
+ */
+public class AgentManager extends AbstractDaemon {
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(AgentManager.class);
+    private final JobManager jobManager;
+    private final TaskManager taskManager;
+    private final TriggerManager triggerManager;
+    private final TaskPositionManager taskPositionManager;
+
+    // jetty for config operations via http.
+    private ConfigJetty configJetty;
+
+
+
+    private final ProfileFetcher fetcher;
+    private final AgentConfiguration conf;
+    private final Db db;
+    private final LocalProfile localProfile;
+    private final CommandDb commandDb;
+
+    public AgentManager() {
+        conf = AgentConfiguration.getAgentConf();
+        this.db = initDb();
+        commandDb = new CommandDb(db);
+        String parentConfPath = conf.get(AGENT_CONF_PARENT, DEFAULT_AGENT_CONF_PARENT);
+        localProfile = new LocalProfile(parentConfPath);
+        fetcher = initFetcher(this);
+        triggerManager = new TriggerManager(this, new TriggerProfileDb(db));
+        jobManager = new JobManager(this, new JobProfileDb(db));
+        taskManager = new TaskManager(this);
+        taskPositionManager = TaskPositionManager.getTaskPositionManager(this);
+        // need to be an option.
+        if (conf.getBoolean(
+            AgentConstants.AGENT_ENABLE_HTTP, AgentConstants.DEFAULT_AGENT_ENABLE_HTTP)) {
+            this.configJetty = new ConfigJetty(jobManager, triggerManager);
+        }
+    }
+
+    /**
+     * init fetch by class name
+     *
+     * @return
+     */
+    private ProfileFetcher initFetcher(AgentManager agentManager) {
+        try {
+            Constructor<?> constructor =
+                Class.forName(conf.get(AgentConstants.AGENT_FETCHER_CLASSNAME))
+                        .getDeclaredConstructor(AgentManager.class);
+            constructor.setAccessible(true);
+            return
+                (ProfileFetcher) constructor.newInstance(agentManager);
+        } catch (Exception ex) {
+            LOGGER.warn("cannot find fetcher, ignore it {}", ex.getMessage());
+        }
+        return null;
+    }
+
+    /**
+     * init db by class name
+     *
+     * @return db
+     */
+    private Db initDb() {
+        try {
+            // db is a required component, so if not init correctly,
+            // throw exception and stop running.
+            return (Db) Class.forName(conf.get(
+                AgentConstants.AGENT_DB_CLASSNAME, AgentConstants.DEFAULT_AGENT_DB_CLASSNAME))
+                .newInstance();
+        } catch (Exception ex) {
+            throw new UnsupportedClassVersionError(ex.getMessage());
+        }
+    }
+
+    public JobManager getJobManager() {
+        return jobManager;
+    }
+
+    public Db getDb() {
+        return db;
+    }
+
+    public ProfileFetcher getFetcher() {
+        return fetcher;
+    }
+
+    public CommandDb getCommandDb() {
+        return commandDb;
+    }
+
+    public TriggerManager getTriggerManager() {
+        return triggerManager;
+    }
+
+    public TaskPositionManager getTaskPositionManager() {
+        return taskPositionManager;
+    }
+
+    public TaskManager getTaskManager() {
+        return taskManager;
+    }
+
+    @Override
+    public void join() {
+        super.join();
+        jobManager.join();
+        taskManager.join();
+    }
+
+    @Override
+    public void start() throws Exception {
+        LOGGER.info("starting agent manager");
+        triggerManager.start();
+        jobManager.start();
+        taskManager.start();
+        taskPositionManager.start();
+        // read job profiles from local
+        List<JobProfile> profileList = localProfile.readFromLocal();
+        for (JobProfile profile : profileList) {
+            if (profile.hasKey(JOB_TRIGGER)) {
+                TriggerProfile triggerProfile = TriggerProfile.parseJobProfile(profile);
+                // there is no need to store this profile in triggerDB, because
+                // this profile comes from local file.
+                triggerManager.addTrigger(triggerProfile);
+            } else {
+                // job db store instance info, so it's suitable to use submitJobProfile
+                // to store instance into job db.
+                jobManager.submitJobProfile(profile);
+            }
+        }
+        if (fetcher != null) {
+            fetcher.start();
+        }
+    }
+
+    /**
+     * It should guarantee thread-safe, and can be invoked many times.
+     *
+     * @throws Exception exceptions
+     */
+    @Override
+    public void stop() throws Exception {
+        if (configJetty != null) {
+            configJetty.close();
+        }
+        if (fetcher != null) {
+            fetcher.stop();
+        }
+        // TODO: change job state which is in running state.
+        LOGGER.info("stopping agent manager");
+        // close in order: trigger -> job -> task
+        triggerManager.stop();
+        jobManager.stop();
+        taskManager.stop();
+        taskPositionManager.stop();
+        this.db.close();
+    }
+}
diff --git a/inlong-agent/agent-core/src/main/java/org/apache/inlong/agent/core/conf/ConfigJetty.java b/inlong-agent/agent-core/src/main/java/org/apache/inlong/agent/core/conf/ConfigJetty.java
new file mode 100644
index 0000000..5976eb8
--- /dev/null
+++ b/inlong-agent/agent-core/src/main/java/org/apache/inlong/agent/core/conf/ConfigJetty.java
@@ -0,0 +1,118 @@
+/*
+ * 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.inlong.agent.core.conf;
+
+import static org.apache.inlong.agent.constants.JobConstants.JOB_TRIGGER;
+
+import java.io.Closeable;
+import org.apache.inlong.agent.conf.AgentConfiguration;
+import org.apache.inlong.agent.conf.JobProfile;
+import org.apache.inlong.agent.conf.TriggerProfile;
+import org.apache.inlong.agent.constants.AgentConstants;
+import org.apache.inlong.agent.core.job.JobManager;
+import org.apache.inlong.agent.core.trigger.TriggerManager;
+import org.eclipse.jetty.server.Connector;
+import org.eclipse.jetty.server.Server;
+import org.eclipse.jetty.server.ServerConnector;
+import org.eclipse.jetty.servlet.ServletHandler;
+import org.eclipse.jetty.servlet.ServletHolder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * start http server and get job/agent config via http
+ */
+public class ConfigJetty implements Closeable {
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(ConfigJetty.class);
+
+    private final AgentConfiguration conf;
+    private final Server server;
+    private final JobManager jobManager;
+    private final TriggerManager triggerManager;
+
+    public ConfigJetty(JobManager jobManager, TriggerManager triggerManager) {
+        this.conf = AgentConfiguration.getAgentConf();
+        this.jobManager = jobManager;
+        this.triggerManager = triggerManager;
+        server = new Server();
+        try {
+            initJetty();
+        } catch (Exception ex) {
+            LOGGER.error("exception caught", ex);
+        }
+    }
+
+    private void initJetty() throws Exception {
+        ServerConnector connector = new ServerConnector(this.server);
+        connector.setPort(conf.getInt(
+            AgentConstants.AGENT_HTTP_PORT, AgentConstants.DEFAULT_AGENT_HTTP_PORT));
+        server.setConnectors(new Connector[] { connector });
+
+        ServletHandler servletHandler = new ServletHandler();
+        ServletHolder holder = new ServletHolder(new ConfigServlet(this));
+        servletHandler.addServletWithMapping(holder, "/config/*");
+        server.setHandler(servletHandler);
+        server.start();
+    }
+
+    public void storeJobConf(JobProfile jobProfile) {
+        // store job conf to bdb
+        if (jobProfile != null) {
+            // trigger job is a special kind of job
+            if (jobProfile.hasKey(JOB_TRIGGER)) {
+                triggerManager.submitTrigger(
+                    TriggerProfile.parseJsonStr(jobProfile.toJsonStr()));
+            } else {
+                jobManager.submitJobProfile(jobProfile);
+            }
+        }
+    }
+
+    public void storeAgentConf(String confJsonStr) {
+        // store agent conf to local file
+        AgentConfiguration conf = AgentConfiguration.getAgentConf();
+        conf.loadJsonStrResource(confJsonStr);
+        conf.flushToLocalPropertiesFile();
+    }
+
+    /**
+     * delete job from conf
+     * @param jobProfile
+     */
+    public void deleteJobConf(JobProfile jobProfile) {
+        if (jobProfile != null) {
+            if (jobProfile.hasKey(JOB_TRIGGER)) {
+                triggerManager.deleteTrigger(TriggerProfile.parseJobProfile(jobProfile).getTriggerId());
+            } else {
+                jobManager.deleteJob(jobProfile.getInstanceId());
+            }
+        }
+    }
+
+    @Override
+    public void close() {
+        try {
+            if (this.server != null) {
+                this.server.stop();
+            }
+        } catch (Exception ex) {
+            LOGGER.error("exception caught", ex);
+        }
+    }
+}
diff --git a/inlong-agent/agent-core/src/main/java/org/apache/inlong/agent/core/conf/ConfigServlet.java b/inlong-agent/agent-core/src/main/java/org/apache/inlong/agent/core/conf/ConfigServlet.java
new file mode 100644
index 0000000..0c0c028
--- /dev/null
+++ b/inlong-agent/agent-core/src/main/java/org/apache/inlong/agent/core/conf/ConfigServlet.java
@@ -0,0 +1,107 @@
+/*
+ * 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.inlong.agent.core.conf;
+
+import com.google.gson.Gson;
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.PrintWriter;
+import javax.servlet.http.HttpServlet;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import org.apache.commons.io.IOUtils;
+import org.apache.inlong.agent.conf.JobProfile;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ConfigServlet extends HttpServlet {
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(ConfigServlet.class);
+
+    private static final String CONTENT_TYPE = "application/json";
+    private static final String CHARSET_TYPE = "UTF-8";
+    private static final String JOB_TYPE = "job";
+    private static final String AGENT_TYPE = "agent";
+    private static final String TRIGGER_TYPE = "trigger";
+    private static final String OP_TYPE = "op";
+    private static final String ADD_OP = "add";
+    private static final String DELETE_OP = "delete";
+    private final Gson gson = new Gson();
+    private final ConfigJetty configHandler;
+
+    public ConfigServlet(ConfigJetty configHandler) {
+        this.configHandler = configHandler;
+    }
+
+    public void responseToJson(HttpServletResponse response,
+        ResponseResult result) throws IOException {
+        response.setContentType(CONTENT_TYPE);
+        response.setCharacterEncoding(CHARSET_TYPE);
+        String jsonStr = gson.toJson(result);
+        PrintWriter out = response.getWriter();
+        out.print(jsonStr);
+        out.flush();
+    }
+
+    /**
+     * handle path of "/config/job"
+     * @param jobProfileStr - job profile string
+     */
+    private void handleJob(String jobProfileStr) {
+        JobProfile jobProfile = JobProfile.parseJsonStr(jobProfileStr);
+        String op = jobProfile.get(OP_TYPE);
+        if (ADD_OP.equals(op)) {
+            configHandler.storeJobConf(jobProfile);
+        } else if (DELETE_OP.equals(op)) {
+            configHandler.deleteJobConf(jobProfile);
+        }
+    }
+
+    private void handleAgent(String agentProfileStr) {
+        // TODO: handle agent
+    }
+
+    /**
+     * handle post requests.
+     *
+     * @param req  - request
+     * @param resp - response
+     */
+    @Override
+    protected void doPost(HttpServletRequest req, HttpServletResponse resp) throws IOException {
+        String pathInfo = req.getPathInfo();
+        ResponseResult responseResult = new ResponseResult(0, "");
+
+        try (BufferedReader reader = req.getReader()) {
+            String configJsonStr = IOUtils.toString(reader);
+            LOGGER.info("Getting request {}", configJsonStr);
+            // path is "/config/job"
+            if (pathInfo.endsWith(JOB_TYPE)) {
+                handleJob(configJsonStr);
+            } else if (pathInfo.endsWith(AGENT_TYPE)) {
+                handleAgent(configJsonStr);
+            } else {
+                responseResult.setCode(-1).setMessage("child path is not correct");
+            }
+        } catch (Exception ex) {
+            LOGGER.error("error while handle post", ex);
+            responseResult.setCode(-1).setMessage(ex.getMessage());
+        }
+        responseToJson(resp, responseResult);
+    }
+}
diff --git a/inlong-agent/agent-core/src/main/java/org/apache/inlong/agent/core/conf/ResponseResult.java b/inlong-agent/agent-core/src/main/java/org/apache/inlong/agent/core/conf/ResponseResult.java
new file mode 100755
index 0000000..b1c5a64
--- /dev/null
+++ b/inlong-agent/agent-core/src/main/java/org/apache/inlong/agent/core/conf/ResponseResult.java
@@ -0,0 +1,49 @@
+/*
+ * 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.inlong.agent.core.conf;
+
+/**
+ * response json for http requests.
+ */
+public class ResponseResult {
+    private int code;
+    private String message;
+
+    public int getCode() {
+        return code;
+    }
+
+    public ResponseResult(int code, String message) {
+        this.code = code;
+        this.message = message;
+    }
+
+    public ResponseResult setCode(int code) {
+        this.code = code;
+        return this;
+    }
+
+    public String getMessage() {
+        return message;
+    }
+
+    public ResponseResult setMessage(String message) {
+        this.message = message;
+        return this;
+    }
+}
diff --git a/inlong-agent/agent-core/src/main/java/org/apache/inlong/agent/core/job/Job.java b/inlong-agent/agent-core/src/main/java/org/apache/inlong/agent/core/job/Job.java
new file mode 100644
index 0000000..f524b97
--- /dev/null
+++ b/inlong-agent/agent-core/src/main/java/org/apache/inlong/agent/core/job/Job.java
@@ -0,0 +1,105 @@
+/*
+ * 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.inlong.agent.core.job;
+
+
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.inlong.agent.conf.JobProfile;
+import org.apache.inlong.agent.constants.JobConstants;
+import org.apache.inlong.agent.core.task.Task;
+import org.apache.inlong.agent.plugin.Channel;
+import org.apache.inlong.agent.plugin.Reader;
+import org.apache.inlong.agent.plugin.Sink;
+import org.apache.inlong.agent.plugin.Source;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * job meta definition, job will be split into several tasks.
+ */
+public class Job {
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(Job.class);
+
+    private final JobProfile jobConf;
+    // job name
+    private String name;
+    // job description
+    private String description;
+    private String jobInstanceId;
+
+    public Job(JobProfile jobConf) {
+        this.jobConf = jobConf;
+        this.name = jobConf.get(JobConstants.JOB_NAME, JobConstants.DEFAULT_JOB_NAME);
+        this.description = jobConf.get(
+            JobConstants.JOB_DESCRIPTION, JobConstants.DEFAULT_JOB_DESCRIPTION);
+        this.jobInstanceId = jobConf.get(JobConstants.JOB_INSTANCE_ID);
+    }
+
+    public String getName() {
+        return name;
+    }
+
+    public void setName(String name) {
+        this.name = name;
+    }
+
+    public String getDescription() {
+        return description;
+    }
+
+    public void setDescription(String description) {
+        this.description = description;
+    }
+
+    public String getJobInstanceId() {
+        return jobInstanceId;
+    }
+
+    public void setJobInstanceId(String jobInstanceId) {
+        this.jobInstanceId = jobInstanceId;
+    }
+
+    public List<Task> createTasks() {
+        List<Task> taskList = new ArrayList<>();
+        int index = 0;
+        try {
+            LOGGER.info("job id: {}, source: {}, channel: {}, sink: {}",
+                getJobInstanceId(), jobConf.get(JobConstants.JOB_SOURCE),
+                jobConf.get(JobConstants.JOB_CHANNEL),
+                jobConf.get(JobConstants.JOB_SINK));
+            Source source = (Source) Class.forName(jobConf.get(JobConstants.JOB_SOURCE)).newInstance();
+            for (Reader reader : source.split(jobConf)) {
+                Sink writer = (Sink) Class.forName(jobConf.get(JobConstants.JOB_SINK)).newInstance();
+                writer.setSourceFile(reader.getReadFile());
+                Channel channel = (Channel) Class.forName(jobConf.get(JobConstants.JOB_CHANNEL)).newInstance();
+                String taskId = String.format("%s_%d", jobInstanceId, index++);
+                taskList.add(new Task(taskId, reader, writer, channel, getJobConf()));
+            }
+        } catch (Exception ex) {
+            throw new RuntimeException(ex);
+        }
+        return taskList;
+    }
+
+    public JobProfile getJobConf() {
+        return this.jobConf;
+    }
+
+}
diff --git a/inlong-agent/agent-core/src/main/java/org/apache/inlong/agent/core/job/JobManager.java b/inlong-agent/agent-core/src/main/java/org/apache/inlong/agent/core/job/JobManager.java
new file mode 100644
index 0000000..2f9d128
--- /dev/null
+++ b/inlong-agent/agent-core/src/main/java/org/apache/inlong/agent/core/job/JobManager.java
@@ -0,0 +1,249 @@
+/*
+ * 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.inlong.agent.core.job;
+
+
+import static org.apache.inlong.agent.constants.AgentConstants.DEFAULT_JOB_DB_CACHE_CHECK_INTERVAL;
+import static org.apache.inlong.agent.constants.AgentConstants.DEFAULT_JOB_DB_CACHE_TIME;
+import static org.apache.inlong.agent.constants.AgentConstants.JOB_DB_CACHE_CHECK_INTERVAL;
+import static org.apache.inlong.agent.constants.AgentConstants.JOB_DB_CACHE_TIME;
+import static org.apache.inlong.agent.constants.JobConstants.JOB_ID;
+import static org.apache.inlong.agent.constants.JobConstants.JOB_ID_PREFIX;
+import static org.apache.inlong.agent.constants.JobConstants.JOB_INSTANCE_ID;
+
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import org.apache.inlong.agent.common.AbstractDaemon;
+import org.apache.inlong.agent.common.AgentThreadFactory;
+import org.apache.inlong.agent.conf.AgentConfiguration;
+import org.apache.inlong.agent.conf.JobProfile;
+import org.apache.inlong.agent.constants.AgentConstants;
+import org.apache.inlong.agent.core.AgentManager;
+import org.apache.inlong.agent.db.JobProfileDb;
+import org.apache.inlong.agent.db.StateSearchKey;
+import org.apache.inlong.agent.utils.AgentUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * JobManager maintains lots of jobs, and communicate between server and task manager.
+ */
+public class JobManager extends AbstractDaemon {
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(JobManager.class);
+
+    // key is job instance id.
+    private ConcurrentHashMap<String, JobWrapper> jobs;
+    // jobs which are not accepted by running pool.
+    private final ConcurrentHashMap<String, Job> pendingJobs;
+    // job thread pool
+    private final ThreadPoolExecutor runningPool;
+    private final AgentManager agentManager;
+    private final int monitorInterval;
+    private final long jobDbCacheTime;
+    private final long jobDbCacheCheckInterval;
+
+    // job profile db is only used to recover instance which is not finished running.
+    private final JobProfileDb jobConfDB;
+    private final JobMetrics jobMetrics;
+    private final AtomicLong index = new AtomicLong(0);
+
+    /**
+     * init job manager
+     *
+     * @param agentManager - agent manager
+     */
+    public JobManager(AgentManager agentManager, JobProfileDb jobConfDb) {
+        this.jobConfDB = jobConfDb;
+        this.agentManager = agentManager;
+        // job thread pool for running
+        this.runningPool = new ThreadPoolExecutor(
+            0, Integer.MAX_VALUE,
+            60L, TimeUnit.SECONDS,
+            new SynchronousQueue<>(),
+            new AgentThreadFactory("job"));
+        this.jobs = new ConcurrentHashMap<>();
+        this.pendingJobs = new ConcurrentHashMap<>();
+        AgentConfiguration conf = AgentConfiguration.getAgentConf();
+        this.monitorInterval = conf
+                .getInt(
+                    AgentConstants.JOB_MONITOR_INTERVAL, AgentConstants.DEFAULT_JOB_MONITOR_INTERVAL);
+        this.jobDbCacheTime = conf.getLong(JOB_DB_CACHE_TIME, DEFAULT_JOB_DB_CACHE_TIME);
+        this.jobDbCacheCheckInterval = conf.getLong(JOB_DB_CACHE_CHECK_INTERVAL, DEFAULT_JOB_DB_CACHE_CHECK_INTERVAL);
+        this.jobMetrics = JobMetrics.create();
+    }
+
+    /**
+     * submit job to work thread.
+     *
+     * @param job - job
+     */
+    public void addJob(Job job) {
+        try {
+            JobWrapper jobWrapper = new JobWrapper(agentManager, job);
+            this.runningPool.execute(jobWrapper);
+            JobWrapper jobWrapperRet = jobs.putIfAbsent(jobWrapper.getJob().getJobInstanceId(), jobWrapper);
+            if (jobWrapperRet != null) {
+                LOGGER.warn("{} has been added to running pool, "
+                    + "cannot be added repeatedly", job.getJobInstanceId());
+            } else {
+                jobMetrics.runningJobs.incr();
+            }
+        } catch (Exception rje) {
+            LOGGER.debug("reject job {}", job.getJobInstanceId(), rje);
+            pendingJobs.putIfAbsent(job.getJobInstanceId(), job);
+        }
+    }
+
+    /**
+     * add job profile
+     * @param profile - job profile.
+     */
+    public boolean submitJobProfile(JobProfile profile) {
+        if (profile == null || !profile.allRequiredKeyExist()) {
+            LOGGER.error("profile is null or not all required key exists {}", profile == null ? null
+                : profile.toJsonStr());
+            return false;
+        }
+        String jobId = profile.get(JOB_ID);
+        profile.set(JOB_INSTANCE_ID, AgentUtils.getUniqId(JOB_ID_PREFIX, jobId, index.incrementAndGet()));
+        LOGGER.info("submit job profile {}", profile.toJsonStr());
+        getJobConfDb().storeJobFirstTime(profile);
+        addJob(new Job(profile));
+        return true;
+    }
+
+    /**
+     * delete job profile and stop job thread
+     * @param jobInstancId
+     */
+    public void deleteJob(String jobInstancId) {
+        JobWrapper jobWrapper = jobs.remove(jobInstancId);
+        if (jobWrapper != null) {
+            LOGGER.info("delete job instance with job id {}", jobInstancId);
+            jobWrapper.cleanup();
+            getJobConfDb().deleteJob(jobInstancId);
+        }
+    }
+
+    /**
+     * start all accepted jobs.
+     */
+    private void startJobs() {
+        List<JobProfile> profileList = getJobConfDb().getAcceptedJobs();
+        for (JobProfile profile : profileList) {
+            LOGGER.info("init starting job from db {}", profile.toJsonStr());
+            addJob(new Job(profile));
+        }
+    }
+
+    public Runnable jobStateCheckThread() {
+        return () -> {
+            while (isRunnable()) {
+                try {
+                    // check pending jobs and try to submit again.
+                    for (String jobId : pendingJobs.keySet()) {
+                        Job job = pendingJobs.remove(jobId);
+                        if (job != null) {
+                            addJob(job);
+                        }
+                    }
+                    TimeUnit.SECONDS.sleep(monitorInterval);
+                } catch (Exception ex) {
+                    LOGGER.error("error caught", ex);
+                }
+            }
+        };
+    }
+
+    /**
+     * check local db and delete old tasks.
+     * @return
+     */
+    public Runnable dbStorageCheckThread() {
+        return () -> {
+            while (isRunnable()) {
+                try {
+                    jobConfDB.removeExpireJobs(jobDbCacheTime);
+                    TimeUnit.SECONDS.sleep(jobDbCacheCheckInterval);
+                } catch (Exception ex) {
+                    LOGGER.error("error caught", ex);
+                }
+            }
+        };
+    }
+
+    /**
+     * mark job as success by job id.
+     * @param jobId - job id
+     */
+    public void markJobAsSuccess(String jobId) {
+        JobWrapper wrapper = jobs.remove(jobId);
+        if (wrapper != null) {
+            jobMetrics.runningJobs.decr();
+            LOGGER.info("job instance {} is success", jobId);
+            // mark job as success.
+            jobConfDB.updateJobState(jobId, StateSearchKey.SUCCESS);
+        }
+    }
+
+    public void markJobAsFailed(String jobId) {
+        JobWrapper wrapper = jobs.remove(jobId);
+        if (wrapper != null) {
+            LOGGER.info("job instance {} is failed", jobId);
+            jobMetrics.runningJobs.decr();
+            jobMetrics.fatalJobs.incr();
+            // mark job as success.
+            jobConfDB.updateJobState(jobId, StateSearchKey.FAILED);
+        }
+    }
+
+    public JobProfileDb getJobConfDb() {
+        return jobConfDB;
+    }
+
+    /**
+     * check job existence using job file name
+     * @return
+     */
+    public boolean checkJobExsit(String fileName) {
+        return jobConfDB.getJob(fileName) != null;
+    }
+
+    public Map<String, JobWrapper> getJobs() {
+        return jobs;
+    }
+
+    @Override
+    public void start() {
+        submitWorker(jobStateCheckThread());
+        submitWorker(dbStorageCheckThread());
+        startJobs();
+    }
+
+    @Override
+    public void stop() throws Exception {
+        waitForTerminate();
+        this.runningPool.shutdown();
+    }
+}
diff --git a/inlong-agent/agent-core/src/main/java/org/apache/inlong/agent/core/job/JobMetrics.java b/inlong-agent/agent-core/src/main/java/org/apache/inlong/agent/core/job/JobMetrics.java
new file mode 100644
index 0000000..b1edd3e
--- /dev/null
+++ b/inlong-agent/agent-core/src/main/java/org/apache/inlong/agent/core/job/JobMetrics.java
@@ -0,0 +1,47 @@
+/*
+ * 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.inlong.agent.core.job;
+
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.inlong.agent.metrics.Metric;
+import org.apache.inlong.agent.metrics.Metrics;
+import org.apache.inlong.agent.metrics.MetricsRegister;
+import org.apache.inlong.agent.metrics.gauge.GaugeInt;
+
+@Metrics
+public class JobMetrics {
+    private static final JobMetrics JOB_METRICS = new JobMetrics();
+    private static final AtomicBoolean REGISTER_ONCE = new AtomicBoolean(false);
+
+    @Metric
+    GaugeInt runningJobs;
+
+    @Metric
+    GaugeInt fatalJobs;
+
+    private JobMetrics() {
+    }
+
+    static JobMetrics create() {
+        if (REGISTER_ONCE.compareAndSet(false, true)) {
+            MetricsRegister.register("Job", "STateSummary", null, JOB_METRICS);
+        }
+        return JOB_METRICS;
+    }
+}
diff --git a/inlong-agent/agent-core/src/main/java/org/apache/inlong/agent/core/job/JobWrapper.java b/inlong-agent/agent-core/src/main/java/org/apache/inlong/agent/core/job/JobWrapper.java
new file mode 100644
index 0000000..81f897c
--- /dev/null
+++ b/inlong-agent/agent-core/src/main/java/org/apache/inlong/agent/core/job/JobWrapper.java
@@ -0,0 +1,130 @@
+/*
+ * 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.inlong.agent.core.job;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import org.apache.inlong.agent.conf.AgentConfiguration;
+import org.apache.inlong.agent.constants.AgentConstants;
+import org.apache.inlong.agent.core.AgentManager;
+import org.apache.inlong.agent.core.task.Task;
+import org.apache.inlong.agent.core.task.TaskManager;
+import org.apache.inlong.agent.state.AbstractStateWrapper;
+import org.apache.inlong.agent.state.State;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * JobWrapper is used in JobManager, it defines the life cycle of
+ * running job and maintains the state of job.
+ */
+public class JobWrapper extends AbstractStateWrapper {
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(JobWrapper.class);
+
+    private final AgentConfiguration agentConf;
+    private final TaskManager taskManager;
+    private final JobManager jobManager;
+    private final Job job;
+
+    private final List<Task> allTasks;
+
+    public JobWrapper(AgentManager manager, Job job) {
+        super();
+        this.agentConf = AgentConfiguration.getAgentConf();
+        this.taskManager = manager.getTaskManager();
+        this.jobManager = manager.getJobManager();
+        this.job = job;
+        this.allTasks = new ArrayList<>();
+        doChangeState(State.ACCEPTED);
+    }
+
+    /**
+     * check states of all tasks, wait if one of them not finished.
+     */
+    private void checkAllTasksStateAndWait() throws Exception {
+        boolean isFinished = false;
+
+        long checkInterval = agentConf.getLong(
+                AgentConstants.JOB_FINISH_CHECK_INTERVAL, AgentConstants.DEFAULT_JOB_FINISH_CHECK_INTERVAL);
+        do {
+            // check whether all tasks have finished.
+            isFinished = allTasks.stream().allMatch(task -> taskManager.isTaskFinished(task.getTaskId()));
+            TimeUnit.SECONDS.sleep(checkInterval);
+        } while (!isFinished);
+        LOGGER.info("all tasks of {} has been checked", job.getJobInstanceId());
+        boolean isSuccess = allTasks.stream().allMatch(task -> taskManager.isTaskSuccess(task.getTaskId()));
+        if (isSuccess) {
+            doChangeState(State.SUCCEEDED);
+        } else {
+            doChangeState(State.FAILED);
+        }
+    }
+
+    /**
+     * submit all tasks
+     */
+    private void submitAllTasks() {
+        List<Task> tasks = job.createTasks();
+        tasks.forEach(task -> {
+            allTasks.add(task);
+            taskManager.submitTask(task);
+        });
+    }
+
+    /**
+     * get job
+     * @return job
+     */
+    public Job getJob() {
+        return job;
+    }
+
+    /**
+     * cleanup job
+     */
+    public void cleanup() {
+        allTasks.forEach(task -> taskManager.removeTask(task.getTaskId()));
+    }
+
+    @Override
+    public void run() {
+        try {
+            doChangeState(State.RUNNING);
+            submitAllTasks();
+            checkAllTasksStateAndWait();
+            cleanup();
+        } catch (Exception ex) {
+            doChangeState(State.FAILED);
+            LOGGER.error("error caught: {}, message: {}",
+                    job.getJobConf().toJsonStr(), ex.getMessage());
+        }
+    }
+
+    @Override
+    public void addCallbacks() {
+        this.addCallback(State.ACCEPTED, State.RUNNING, (before, after) -> {
+
+        }).addCallback(State.RUNNING, State.FAILED, (before, after) -> {
+            jobManager.markJobAsFailed(job.getJobInstanceId());
+        }).addCallback(State.RUNNING, State.SUCCEEDED, ((before, after) -> {
+            jobManager.markJobAsSuccess(job.getJobInstanceId());
+        }));
+    }
+}
diff --git a/inlong-agent/agent-core/src/main/java/org/apache/inlong/agent/core/task/Task.java b/inlong-agent/agent-core/src/main/java/org/apache/inlong/agent/core/task/Task.java
new file mode 100755
index 0000000..65b39db
--- /dev/null
+++ b/inlong-agent/agent-core/src/main/java/org/apache/inlong/agent/core/task/Task.java
@@ -0,0 +1,77 @@
+/*
+ * 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.inlong.agent.core.task;
+
+import org.apache.inlong.agent.conf.JobProfile;
+import org.apache.inlong.agent.plugin.Channel;
+import org.apache.inlong.agent.plugin.Reader;
+import org.apache.inlong.agent.plugin.Sink;
+
+
+/**
+ * task meta definition which contains reader -> channel -> sink and job config information
+ */
+public class Task {
+
+    private final String taskId;
+    private final Reader reader;
+    private final Sink sink;
+    private final Channel channel;
+    private final JobProfile jobConf;
+
+    public Task(String taskId, Reader reader, Sink sink, Channel channel,
+        JobProfile jobConf) {
+        this.reader = reader;
+        this.sink = sink;
+        this.taskId = taskId;
+        this.channel = channel;
+        this.jobConf = jobConf;
+    }
+
+    public boolean isReadFinished() {
+        return reader.isFinished();
+    }
+
+    public String getTaskId() {
+        return taskId;
+    }
+
+    public Reader getReader() {
+        return reader;
+    }
+
+    public Sink getSink() {
+        return sink;
+    }
+
+    public Channel getChannel() {
+        return channel;
+    }
+
+    public void init() {
+        this.channel.init(jobConf);
+        this.sink.init(jobConf);
+        this.reader.init(jobConf);
+    }
+
+    public void destroy() {
+        this.reader.destroy();
+        this.sink.destroy();
+        this.channel.destroy();
+    }
+}
diff --git a/inlong-agent/agent-core/src/main/java/org/apache/inlong/agent/core/task/TaskManager.java b/inlong-agent/agent-core/src/main/java/org/apache/inlong/agent/core/task/TaskManager.java
new file mode 100755
index 0000000..6f895d5
--- /dev/null
+++ b/inlong-agent/agent-core/src/main/java/org/apache/inlong/agent/core/task/TaskManager.java
@@ -0,0 +1,255 @@
+/*
+ * 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.inlong.agent.core.task;
+
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import org.apache.inlong.agent.common.AbstractDaemon;
+import org.apache.inlong.agent.common.AgentThreadFactory;
+import org.apache.inlong.agent.conf.AgentConfiguration;
+import org.apache.inlong.agent.constants.AgentConstants;
+import org.apache.inlong.agent.core.AgentManager;
+import org.apache.inlong.agent.core.job.JobManager;
+import org.apache.inlong.agent.utils.AgentUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Task manager maintains lots of tasks and communicate with job level components.
+ * It also provide functions to execute commands from job level like killing/submit tasks.
+ */
+public class TaskManager extends AbstractDaemon {
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(JobManager.class);
+
+    // task thread pool;
+    private final ThreadPoolExecutor runningPool;
+    private final AgentManager agentManager;
+    private final TaskMetrics taskMetrics;
+    private final ConcurrentHashMap<String, TaskWrapper> tasks;
+    private final BlockingQueue<TaskWrapper> retryTasks;
+    private final int monitorInterval;
+    private final int taskMaxCapacity;
+    private final int taskRetryMaxTime;
+    private final long waitTime;
+
+    /**
+     * Init task manager.
+     *
+     * @param agentManager - agent manager
+     */
+    public TaskManager(AgentManager agentManager) {
+        this.agentManager = agentManager;
+        this.runningPool = new ThreadPoolExecutor(
+            0, Integer.MAX_VALUE,
+            60L, TimeUnit.SECONDS,
+            new SynchronousQueue<>(),
+            new AgentThreadFactory("task"));
+        // metric for task level
+        taskMetrics = TaskMetrics.create();
+        tasks = new ConcurrentHashMap<>();
+        AgentConfiguration conf = AgentConfiguration.getAgentConf();
+        retryTasks = new LinkedBlockingQueue<>(
+                conf.getInt(
+                    AgentConstants.TASK_RETRY_MAX_CAPACITY, AgentConstants.DEFAULT_TASK_RETRY_MAX_CAPACITY));
+        monitorInterval = conf.getInt(
+            AgentConstants.TASK_MONITOR_INTERVAL, AgentConstants.DEFAULT_TASK_MONITOR_INTERVAL);
+        taskRetryMaxTime = conf
+                .getInt(AgentConstants.TASK_RETRY_SUBMIT_WAIT_SECONDS,
+                    AgentConstants.DEFAULT_TASK_RETRY_SUBMIT_WAIT_SECONDS);
+        taskMaxCapacity = conf.getInt(
+            AgentConstants.TASK_RETRY_MAX_CAPACITY, AgentConstants.DEFAULT_TASK_RETRY_MAX_CAPACITY);
+        waitTime = conf.getLong(
+            AgentConstants.THREAD_POOL_AWAIT_TIME, AgentConstants.DEFAULT_THREAD_POOL_AWAIT_TIME);
+    }
+
+    /**
+     * Get task metrics
+     *
+     * @return task metrics
+     */
+    public TaskMetrics getTaskMetrics() {
+        return taskMetrics;
+    }
+
+    public TaskWrapper getTaskWrapper(String taskId) {
+        return tasks.get(taskId);
+    }
+
+    /**
+     * submit task, wait if task queue is full.
+     *
+     * @param task - task
+     */
+    public void submitTask(Task task) {
+        TaskWrapper taskWrapper = new TaskWrapper(agentManager, task);
+        submitTask(taskWrapper);
+
+    }
+
+    public void submitTask(TaskWrapper wrapper) {
+        TaskWrapper retTaskWrapper = tasks.putIfAbsent(wrapper.getTask().getTaskId(), wrapper);
+        if (retTaskWrapper == null) {
+            // pool may be full
+            boolean notSubmitted = true;
+            while (notSubmitted) {
+                try {
+                    this.runningPool.submit(wrapper);
+                    notSubmitted = false;
+                } catch (Exception ex) {
+                    AgentUtils.silenceSleepInMs(waitTime);
+                    LOGGER.warn("reject task {}", wrapper.getTask().getTaskId(), ex);
+                }
+            }
+            taskMetrics.runningTasks.incr();
+        }
+    }
+
+    /**
+     * retry task.
+     *
+     * @param wrapper - task wrapper
+     */
+    private boolean addRetryTask(TaskWrapper wrapper) {
+        LOGGER.info("retry submit task {}", wrapper.getTask().getTaskId());
+        try {
+            boolean success = retryTasks.offer(wrapper, taskRetryMaxTime, TimeUnit.SECONDS);
+            if (!success) {
+                LOGGER.error("cannot submit to retry queue, max {}, current {}", taskMaxCapacity,
+                        retryTasks.size());
+            } else {
+                taskMetrics.retryingTasks.incr();
+            }
+            return success;
+        } catch (Exception ex) {
+            LOGGER.error("error while offer task", ex);
+        }
+        return false;
+    }
+
+    /**
+     * Check whether task is finished
+     *
+     * @param taskId - task id
+     * @return - true if task is finished otherwise false
+     */
+    public boolean isTaskFinished(String taskId) {
+        TaskWrapper wrapper = tasks.get(taskId);
+        if (wrapper != null) {
+            return wrapper.isFinished();
+        }
+        return false;
+    }
+
+    /**
+     * Check if task is success
+     *
+     * @param taskId task id
+     * @return true if task is success otherwise false
+     */
+    public boolean isTaskSuccess(String taskId) {
+        TaskWrapper wrapper = tasks.get(taskId);
+        if (wrapper != null) {
+            return wrapper.isSuccess();
+        }
+        return false;
+    }
+
+    /**
+     * Remove task and wait task to finish by task id
+     *
+     * @param taskId - task id
+     */
+    public void removeTask(String taskId) {
+        taskMetrics.runningTasks.decr();
+        TaskWrapper taskWrapper = tasks.remove(taskId);
+        if (taskWrapper != null) {
+            taskWrapper.waitForFinish();
+        }
+    }
+
+    /**
+     * kill task
+     *
+     * @param task task
+     * @return
+     */
+    public boolean killTask(Task task) {
+        // kill running tasks.
+        TaskWrapper taskWrapper = tasks.get(task.getTaskId());
+        if (taskWrapper != null) {
+            taskWrapper.kill();
+            return true;
+        }
+        return false;
+    }
+
+    /**
+     * Thread for checking whether task should retry.
+     *
+     * @return - runnable thread
+     */
+    public Runnable createTaskMonitorThread() {
+        return () -> {
+            while (isRunnable()) {
+                try {
+                    for (String taskId : tasks.keySet()) {
+                        TaskWrapper wrapper = tasks.get(taskId);
+                        if (wrapper != null && wrapper.isFailed() && wrapper.shouldRetry()) {
+                            boolean success = addRetryTask(wrapper);
+                            if (success) {
+                                removeTask(taskId);
+                            }
+                        }
+                    }
+                    while (!retryTasks.isEmpty()) {
+                        TaskWrapper taskWrapper = retryTasks.poll();
+                        if (taskWrapper != null) {
+                            taskMetrics.retryingTasks.decr();
+                            submitTask(taskWrapper);
+                        }
+                    }
+                    TimeUnit.SECONDS.sleep(monitorInterval);
+                } catch (Exception ex) {
+                    LOGGER.error("Exception caught", ex);
+                }
+            }
+        };
+    }
+
+    /**
+     * start service.
+     */
+    @Override
+    public void start() {
+        submitWorker(createTaskMonitorThread());
+    }
+
+    /**
+     * stop service.
+     */
+    @Override
+    public void stop() throws Exception {
+        waitForTerminate();
+        this.runningPool.shutdown();
+    }
+}
diff --git a/inlong-agent/agent-core/src/main/java/org/apache/inlong/agent/core/task/TaskMetrics.java b/inlong-agent/agent-core/src/main/java/org/apache/inlong/agent/core/task/TaskMetrics.java
new file mode 100755
index 0000000..5863e88
--- /dev/null
+++ b/inlong-agent/agent-core/src/main/java/org/apache/inlong/agent/core/task/TaskMetrics.java
@@ -0,0 +1,55 @@
+/*
+ * 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.inlong.agent.core.task;
+
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.inlong.agent.metrics.Metric;
+import org.apache.inlong.agent.metrics.Metrics;
+import org.apache.inlong.agent.metrics.MetricsRegister;
+import org.apache.inlong.agent.metrics.counter.CounterLong;
+import org.apache.inlong.agent.metrics.gauge.GaugeInt;
+
+/**
+ * Metric collector for task level.
+ */
+@Metrics
+public class TaskMetrics {
+
+    private static final TaskMetrics TASK_METRICS = new TaskMetrics();
+    private static final AtomicBoolean REGISTER_ONCE = new AtomicBoolean(false);
+
+    @Metric
+    GaugeInt runningTasks;
+
+    @Metric
+    GaugeInt retryingTasks;
+
+    @Metric
+    CounterLong fatalTasks;
+
+    private TaskMetrics() {
+    }
+
+    public static TaskMetrics create() {
+        // register one time.
+        if (REGISTER_ONCE.compareAndSet(false, true)) {
+            MetricsRegister.register("Task", "StateSummary", null, TASK_METRICS);
+        }
+        return TASK_METRICS;
+    }
+}
diff --git a/inlong-agent/agent-core/src/main/java/org/apache/inlong/agent/core/task/TaskPositionManager.java b/inlong-agent/agent-core/src/main/java/org/apache/inlong/agent/core/task/TaskPositionManager.java
new file mode 100644
index 0000000..5c4efef
--- /dev/null
+++ b/inlong-agent/agent-core/src/main/java/org/apache/inlong/agent/core/task/TaskPositionManager.java
@@ -0,0 +1,148 @@
+/*
+ * 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.inlong.agent.core.task;
+
+import static org.apache.inlong.agent.constants.CommonConstants.POSITION_SUFFIX;
+
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.TimeUnit;
+import org.apache.inlong.agent.common.AbstractDaemon;
+import org.apache.inlong.agent.conf.JobProfile;
+import org.apache.inlong.agent.core.AgentManager;
+import org.apache.inlong.agent.db.JobProfileDb;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * used to store task position to db, task position is stored as properties in JobProfile.
+ * where key is task read file name and value is task sink position
+ * note that this class is generated
+ */
+public class TaskPositionManager extends AbstractDaemon {
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(TaskPositionManager.class);
+    public static final int DEFAULT_FLUSH_TIMEOUT = 30;
+
+    private final AgentManager agentManager;
+    private final JobProfileDb jobConfDb;
+    private ConcurrentHashMap<String, ConcurrentHashMap<String, Long>> jobTaskPositionMap;
+
+    private static volatile TaskPositionManager taskPositionManager = null;
+
+    private TaskPositionManager(AgentManager agentManager) {
+        this.agentManager = agentManager;
+        this.jobConfDb = agentManager.getJobManager().getJobConfDb();
+        this.jobTaskPositionMap = new ConcurrentHashMap<>();
+    }
+
+    /**
+     * task position manager singleton, can only generated by agent manager
+     * @param agentManager
+     * @return
+     */
+    public static TaskPositionManager getTaskPositionManager(AgentManager agentManager) {
+        if (taskPositionManager == null) {
+            synchronized (TaskPositionManager.class) {
+                if (taskPositionManager == null) {
+                    taskPositionManager = new TaskPositionManager(agentManager);
+                }
+            }
+        }
+        return taskPositionManager;
+    }
+
+    /**
+     * get taskPositionManager singleton
+     * @return
+     */
+    public static TaskPositionManager getTaskPositionManager() {
+        if (taskPositionManager == null) {
+            throw new RuntimeException("task position manager has not been initialized by agentManager");
+        }
+        return taskPositionManager;
+    }
+
+    @Override
+    public void start() throws Exception {
+        submitWorker(taskPositionFlushThread());
+    }
+
+    private Runnable taskPositionFlushThread() {
+        return () -> {
+            while (isRunnable()) {
+                try {
+                    // check pending jobs and try to submit again.
+                    for (String jobId : jobTaskPositionMap.keySet()) {
+                        JobProfile jobProfile = jobConfDb.getJobProfile(jobId);
+                        if (jobProfile == null) {
+                            LOGGER.warn("jobProfile {} cannot be found in db, "
+                                + "might be deleted by standalone mode, now delete job position in memory", jobId);
+                            deleteJobPosition(jobId);
+                            continue;
+                        }
+                        flushJobProfile(jobId, jobProfile);
+                    }
+                    TimeUnit.SECONDS.sleep(DEFAULT_FLUSH_TIMEOUT);
+                } catch (Exception ex) {
+                    LOGGER.error("error caught", ex);
+                }
+            }
+        };
+    }
+
+
+    private void flushJobProfile(String jobId, JobProfile jobProfile) {
+        jobTaskPositionMap.get(jobId).forEach(
+            (fileName, position) -> jobProfile.setLong(fileName + POSITION_SUFFIX, position)
+        );
+        if (jobConfDb.checkJobfinished(jobProfile)) {
+            LOGGER.info("Cannot update job profile {}, delete memory job in jobTaskPosition", jobId);
+            deleteJobPosition(jobId);
+        } else {
+            jobConfDb.updateJobProfile(jobProfile);
+        }
+    }
+
+    private void deleteJobPosition(String jobId) {
+        jobTaskPositionMap.remove(jobId);
+    }
+
+    @Override
+    public void stop() throws Exception {
+        waitForTerminate();
+    }
+
+    public void updateFileSinkPosition(String jobInstanceId, String sourceFilePath, long size) {
+        ConcurrentHashMap<String, Long> filePositionTemp = new ConcurrentHashMap<>();
+        ConcurrentHashMap<String, Long> filePosition = jobTaskPositionMap.putIfAbsent(jobInstanceId, filePositionTemp);
+        if (filePosition == null) {
+            filePosition = filePositionTemp;
+        }
+        Long beforePosition = filePosition.getOrDefault(sourceFilePath, 1L);
+        filePosition.put(sourceFilePath, beforePosition + size);
+    }
+
+
+    public ConcurrentHashMap<String, Long> getTaskPositionMap(String jobId) {
+        return jobTaskPositionMap.get(jobId);
+    }
+
+    public ConcurrentHashMap<String, ConcurrentHashMap<String, Long>> getJobTaskPosition() {
+        return jobTaskPositionMap;
+    }
+}
diff --git a/inlong-agent/agent-core/src/main/java/org/apache/inlong/agent/core/task/TaskWrapper.java b/inlong-agent/agent-core/src/main/java/org/apache/inlong/agent/core/task/TaskWrapper.java
new file mode 100755
index 0000000..3f7caba
--- /dev/null
+++ b/inlong-agent/agent-core/src/main/java/org/apache/inlong/agent/core/task/TaskWrapper.java
@@ -0,0 +1,198 @@
+/*
+ * 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.inlong.agent.core.task;
+
+
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.inlong.agent.common.AgentThreadFactory;
+import org.apache.inlong.agent.conf.AgentConfiguration;
+import org.apache.inlong.agent.constants.AgentConstants;
+import org.apache.inlong.agent.core.AgentManager;
+import org.apache.inlong.agent.message.EndMessage;
+import org.apache.inlong.agent.plugin.Message;
+import org.apache.inlong.agent.state.AbstractStateWrapper;
+import org.apache.inlong.agent.state.State;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * TaskWrapper is used in taskManager, it maintains the life cycle of
+ * running task.
+ */
+public class TaskWrapper extends AbstractStateWrapper {
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(TaskWrapper.class);
+    public static final int WAIT_FINISH_TIME_OUT = 1;
+
+    private final TaskManager taskManager;
+    private final Task task;
+
+    private final AtomicInteger retryTime = new AtomicInteger(0);
+    private final int maxRetryTime;
+    private final int pushMaxWaitTime;
+    private final int pullMaxWaitTime;
+    private ExecutorService executorService;
+
+    public TaskWrapper(AgentManager manager, Task task) {
+        super();
+        this.taskManager = manager.getTaskManager();
+        this.task = task;
+        AgentConfiguration conf = AgentConfiguration.getAgentConf();
+        maxRetryTime = conf.getInt(
+            AgentConstants.TASK_MAX_RETRY_TIME, AgentConstants.DEFAULT_TASK_MAX_RETRY_TIME);
+        pushMaxWaitTime = conf.getInt(
+            AgentConstants.TASK_PUSH_MAX_SECOND, AgentConstants.DEFAULT_TASK_PUSH_MAX_SECOND);
+        pullMaxWaitTime = conf.getInt(
+            AgentConstants.TASK_PULL_MAX_SECOND, AgentConstants.DEFAULT_TASK_PULL_MAX_SECOND);
+        if (executorService == null) {
+            executorService = new ThreadPoolExecutor(0, Integer.MAX_VALUE,
+                60L, TimeUnit.SECONDS,
+                new SynchronousQueue<Runnable>(),
+                new AgentThreadFactory("task-reader-writer"));
+        }
+        doChangeState(State.ACCEPTED);
+    }
+
+    /**
+     * submit read thread
+     *
+     * @return CompletableFuture
+     */
+    private CompletableFuture<?> submitReadThread() {
+        return CompletableFuture.runAsync(() -> {
+            Message message = null;
+            while (!isException() && !task.isReadFinished()) {
+                if (message == null || task.getChannel()
+                        .push(message, pushMaxWaitTime, TimeUnit.SECONDS)) {
+                    message = task.getReader().read();
+                }
+            }
+            LOGGER.info("read end, task exception status is {}, read finish status is {}",
+                isException(), task.isReadFinished());
+            // write end message
+            task.getChannel().push(new EndMessage());
+        }, executorService);
+    }
+
+    /**
+     * submit write thread
+     *
+     * @return CompletableFuture
+     */
+    private CompletableFuture<?> submitWriteThread() {
+        return CompletableFuture.runAsync(() -> {
+            while (!isException()) {
+                Message message = task.getChannel().pull(pullMaxWaitTime, TimeUnit.SECONDS);
+                if (message instanceof EndMessage) {
+                    break;
+                }
+                task.getSink().write(message);
+            }
+        }, executorService);
+    }
+
+    /**
+     * submit reader/writer
+     */
+    private void submitThreadsAndWait() {
+        CompletableFuture<?> reader = submitReadThread();
+        CompletableFuture<?> writer = submitWriteThread();
+        CompletableFuture.allOf(reader, writer)
+                .exceptionally(ex -> {
+                    doChangeState(State.FAILED);
+                    LOGGER.error("exception caught", ex);
+                    return null;
+                }).join();
+    }
+
+    /**
+     * kill task
+     */
+    void kill() {
+        LOGGER.info("task id {} is killed", task.getTaskId());
+        doChangeState(State.KILLED);
+    }
+
+    /**
+     * In standalone mode, the job to be removed should wait until the read is finished, set
+     * timeout to WAIT_FINISH_TIME_OUT minute to wait for finishing
+     */
+    void waitForFinish() {
+        LOGGER.info("set readTime out to 1 minute task id is {}", task.getTaskId());
+        task.getReader().setReadTimeout(TimeUnit.MINUTES.toMillis(WAIT_FINISH_TIME_OUT));
+    }
+
+    /**
+     * whether task retry times exceed max retry time.
+     *
+     * @return - whether should retry
+     */
+    boolean shouldRetry() {
+        return retryTime.get() < maxRetryTime;
+    }
+
+    Task getTask() {
+        return task;
+    }
+
+    @Override
+    public void addCallbacks() {
+        this.addCallback(State.ACCEPTED, State.RUNNING, (before, after) -> {
+
+        }).addCallback(State.RUNNING, State.FAILED, (before, after) -> {
+            LOGGER.info("task {} is failed, please check it", task.getTaskId());
+            retryTime.incrementAndGet();
+            if (!shouldRetry()) {
+                doChangeState(State.FATAL);
+                taskManager.getTaskMetrics().fatalTasks.incr();
+            }
+        }).addCallback(State.FAILED, State.FATAL, (before, after) -> {
+
+        }).addCallback(State.FAILED, State.ACCEPTED, (before, after) -> {
+
+        }).addCallback(State.FAILED, State.RUNNING, ((before, after) -> {
+
+        })).addCallback(State.RUNNING, State.SUCCEEDED, (before, after) -> {
+
+        });
+    }
+
+
+    @Override
+    public void run() {
+        try {
+            LOGGER.info("start to run {}, retry time is {}", task.getTaskId(), retryTime.get());
+            doChangeState(State.RUNNING);
+            task.init();
+            submitThreadsAndWait();
+            if (!isException()) {
+                doChangeState(State.SUCCEEDED);
+            }
+            LOGGER.info("start to destroy task {}", task.getTaskId());
+            task.destroy();
+        } catch (Exception ex) {
+            LOGGER.error("error while running wrapper", ex);
+            doChangeState(State.FAILED);
+        }
+    }
+}
diff --git a/inlong-agent/agent-core/src/main/java/org/apache/inlong/agent/core/trigger/TriggerManager.java b/inlong-agent/agent-core/src/main/java/org/apache/inlong/agent/core/trigger/TriggerManager.java
new file mode 100755
index 0000000..7cf7bbc
--- /dev/null
+++ b/inlong-agent/agent-core/src/main/java/org/apache/inlong/agent/core/trigger/TriggerManager.java
@@ -0,0 +1,248 @@
+/*
+ * 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.inlong.agent.core.trigger;
+
+
+import static org.apache.inlong.agent.constants.AgentConstants.DEFAULT_TRIGGER_MAX_RUNNING_NUM;
+import static org.apache.inlong.agent.constants.AgentConstants.TRIGGER_MAX_RUNNING_NUM;
+import static org.apache.inlong.agent.constants.JobConstants.JOB_ID;
+import static org.apache.inlong.agent.constants.JobConstants.TRIGGER_ONLY_ONE_JOB;
+
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.TimeUnit;
+import org.apache.inlong.agent.common.AbstractDaemon;
+import org.apache.inlong.agent.conf.AgentConfiguration;
+import org.apache.inlong.agent.conf.JobProfile;
+import org.apache.inlong.agent.conf.TriggerProfile;
+import org.apache.inlong.agent.constants.AgentConstants;
+import org.apache.inlong.agent.constants.JobConstants;
+import org.apache.inlong.agent.core.AgentManager;
+import org.apache.inlong.agent.core.job.JobWrapper;
+import org.apache.inlong.agent.db.TriggerProfileDb;
+import org.apache.inlong.agent.plugin.Trigger;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * manager for triggers.
+ */
+public class TriggerManager extends AbstractDaemon {
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(TriggerManager.class);
+    public static final int JOB_CHECK_INTERVAL = 1;
+
+    private final AgentManager manager;
+    private final TriggerProfileDb triggerProfileDB;
+    private final ConcurrentHashMap<String, Trigger> triggerMap;
+    private final ConcurrentHashMap<String, ConcurrentHashMap<String, JobProfile>> triggerJobMap;
+    private final AgentConfiguration conf;
+    private final int triggerFetchInterval;
+    private final int maxRunningNum;
+
+    public TriggerManager(AgentManager manager, TriggerProfileDb triggerProfileDb) {
+        this.conf = AgentConfiguration.getAgentConf();
+        this.manager = manager;
+        this.triggerProfileDB = triggerProfileDb;
+        this.triggerMap = new ConcurrentHashMap<>();
+        this.triggerJobMap = new ConcurrentHashMap<>();
+        this.triggerFetchInterval = conf.getInt(AgentConstants.TRIGGER_FETCH_INTERVAL,
+                AgentConstants.DEFAULT_TRIGGER_FETCH_INTERVAL);
+        this.maxRunningNum = conf.getInt(TRIGGER_MAX_RUNNING_NUM, DEFAULT_TRIGGER_MAX_RUNNING_NUM);
+    }
+
+    /**
+     * submit trigger profile.
+     * @param triggerProfile - trigger profile
+     */
+    public boolean addTrigger(TriggerProfile triggerProfile) {
+        try {
+            Class<?> triggerClass = Class.forName(triggerProfile.get(JobConstants.JOB_TRIGGER));
+            Trigger trigger = (Trigger) triggerClass.newInstance();
+            String triggerId = triggerProfile.get(JOB_ID);
+            if (triggerMap.containsKey(triggerId)) {
+                deleteTrigger(triggerId);
+                LOGGER.warn("trigger {} is running, stop it", triggerId);
+            }
+            triggerMap.put(triggerId, trigger);
+            trigger.init(triggerProfile);
+            trigger.run();
+        } catch (Exception ex) {
+            LOGGER.error("exception caught", ex);
+            return false;
+        }
+        return true;
+    }
+
+
+    public Trigger getTrigger(String triggerId) {
+        return triggerMap.get(triggerId);
+    }
+
+    public boolean submitTrigger(TriggerProfile triggerProfile) {
+        // make sure all required key exists.
+        if (!triggerProfile.allRequiredKeyExist() || this.triggerMap.size() > maxRunningNum) {
+            LOGGER.error("trigger {} not all required key exists or size {} exceed {}",
+                triggerProfile.toJsonStr(), this.triggerMap.size(), maxRunningNum);
+            return false;
+        }
+        triggerProfileDB.storeTrigger(triggerProfile);
+        addTrigger(triggerProfile);
+        return true;
+    }
+
+
+
+
+    private Runnable jobFetchThread() {
+        return () -> {
+            while (isRunnable()) {
+                try {
+                    triggerMap.forEach((s, trigger) -> {
+                        JobProfile profile = trigger.fetchJobProfile();
+                        if (profile != null) {
+                            TriggerProfile triggerProfile = trigger.getTriggerProfile();
+                            if (triggerProfile.getBoolean(TRIGGER_ONLY_ONE_JOB, false)) {
+                                deleteRelatedJobs(triggerProfile.getTriggerId());
+                            }
+                            manager.getJobManager().submitJobProfile(profile);
+                            addToTriggerMap(profile.get(JOB_ID), profile);
+                        }
+                    });
+                    TimeUnit.SECONDS.sleep(triggerFetchInterval);
+                } catch (Exception ignored) {
+                    LOGGER.info("ignored Exception ", ignored);
+                }
+            }
+
+        };
+    }
+
+    /**
+     * delete jobs generated by the trigger
+     * @param triggerId
+     */
+    private void deleteRelatedJobs(String triggerId) {
+        LOGGER.info("start to delete related jobs in triggerId {}", triggerId);
+        ConcurrentHashMap<String, JobProfile> jobProfiles =
+            triggerJobMap.get(triggerId);
+        if (jobProfiles != null) {
+            LOGGER.info("trigger can only run one job, stop the others {}", jobProfiles.keySet());
+            jobProfiles.keySet().forEach(this::deleteJob);
+            triggerJobMap.remove(triggerId);
+        }
+    }
+
+    private void deleteJob(String jobInstanceId) {
+        manager.getJobManager().deleteJob(jobInstanceId);
+    }
+
+
+    private Runnable jobCheckThread() {
+        return () -> {
+            while (isRunnable()) {
+                try {
+                    triggerJobMap.forEach((s, jobProfiles) -> {
+                        for (String jobId : jobProfiles.keySet()) {
+                            Map<String, JobWrapper> jobs =
+                                manager.getJobManager().getJobs();
+                            if (jobs.get(jobId) == null) {
+                                triggerJobMap.remove(jobId);
+                            }
+                        }
+                    });
+                    TimeUnit.MINUTES.sleep(JOB_CHECK_INTERVAL);
+                } catch (Exception ignored) {
+                    LOGGER.info("ignored Exception ", ignored);
+                }
+            }
+
+        };
+    }
+
+    /**
+     * need to put profile in triggerJobMap
+     * @param triggerId
+     * @param profile
+     */
+    private void addToTriggerMap(String triggerId, JobProfile profile) {
+        ConcurrentHashMap<String, JobProfile> tmpList =
+            new ConcurrentHashMap<>();
+        ConcurrentHashMap<String, JobProfile> jobWrappers =
+            triggerJobMap.putIfAbsent(triggerId, tmpList);
+        if (jobWrappers == null) {
+            jobWrappers = tmpList;
+        }
+        jobWrappers.putIfAbsent(profile.getInstanceId(), profile);
+    }
+
+    /**
+     * delete trigger by trigger profile.
+     * @param triggerId - trigger profile.
+     */
+    public boolean deleteTrigger(String triggerId) {
+        LOGGER.info("delete trigger {}", triggerId);
+        Trigger trigger = triggerMap.remove(triggerId);
+        if (trigger != null) {
+            deleteRelatedJobs(triggerId);
+            trigger.destroy();
+            // delete trigger from db
+            triggerProfileDB.deleteTrigger(triggerId);
+            return true;
+        }
+        LOGGER.warn("cannot find trigger {}", triggerId);
+        return false;
+    }
+
+
+
+    /**
+     * init all triggers when daemon started.
+     */
+    private void initTriggers() throws Exception {
+        // fetch all triggers from db
+        List<TriggerProfile> profileList = triggerProfileDB.getTriggers();
+        for (TriggerProfile profile : profileList) {
+            addTrigger(profile);
+        }
+    }
+
+    private void stopTriggers() {
+        triggerMap.forEach((s, trigger) -> {
+            trigger.destroy();
+        });
+    }
+
+    @Override
+    public void start() throws Exception {
+        initTriggers();
+        submitWorker(jobFetchThread());
+        submitWorker(jobCheckThread());
+    }
+
+
+
+    @Override
+    public void stop() {
+        // stop all triggers
+        stopTriggers();
+    }
+
+
+}
diff --git a/inlong-agent/agent-core/src/test/java/org/apache/inlong/agent/core/AgentBaseTestsHelper.java b/inlong-agent/agent-core/src/test/java/org/apache/inlong/agent/core/AgentBaseTestsHelper.java
new file mode 100755
index 0000000..d92b2da
--- /dev/null
+++ b/inlong-agent/agent-core/src/test/java/org/apache/inlong/agent/core/AgentBaseTestsHelper.java
@@ -0,0 +1,65 @@
+/*
+ * 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.inlong.agent.core;
+
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import org.apache.commons.io.FileUtils;
+import org.apache.inlong.agent.conf.AgentConfiguration;
+import org.apache.inlong.agent.constants.AgentConstants;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * common environment setting up for test cases.
+ */
+public class AgentBaseTestsHelper {
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(AgentBaseTestsHelper.class);
+
+    private final String className;
+    private Path testRootDir;
+
+    public AgentBaseTestsHelper(String className) {
+        this.className = className;
+    }
+
+    public AgentBaseTestsHelper setupAgentHome() {
+        testRootDir = Paths
+                .get("/tmp", AgentBaseTestsHelper.class.getSimpleName(), className);
+        teardownAgentHome();
+        boolean result = testRootDir.toFile().mkdirs();
+        LOGGER.info("try to create {}, result is {}", testRootDir, result);
+        AgentConfiguration.getAgentConf().set(AgentConstants.AGENT_HOME, testRootDir.toString());
+        return this;
+    }
+
+    public Path getTestRootDir() {
+        return testRootDir;
+    }
+
+    public void teardownAgentHome() {
+        if (testRootDir != null) {
+            try {
+                FileUtils.deleteDirectory(testRootDir.toFile());
+            } catch (Exception ignored) {
+                LOGGER.warn("deleteDirectory error ", ignored);
+            }
+        }
+    }
+}
diff --git a/inlong-agent/agent-core/src/test/java/org/apache/inlong/agent/core/TestAgentMain.java b/inlong-agent/agent-core/src/test/java/org/apache/inlong/agent/core/TestAgentMain.java
new file mode 100755
index 0000000..1d5c3e6
--- /dev/null
+++ b/inlong-agent/agent-core/src/test/java/org/apache/inlong/agent/core/TestAgentMain.java
@@ -0,0 +1,48 @@
+/*
+ * 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.inlong.agent.core;
+
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.CommandLineParser;
+import org.apache.commons.cli.DefaultParser;
+import org.apache.commons.cli.HelpFormatter;
+import org.apache.commons.cli.Options;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestAgentMain {
+
+    @Test
+    public void testOpts() throws Exception {
+        Options options = new Options();
+        options.addOption("h", "help", false, "show help");
+        HelpFormatter formatter = new HelpFormatter();
+        formatter.printHelp("./run.sh", options);
+    }
+
+    @Test
+    public void testOptsOverride() throws Exception {
+        Options options = new Options();
+        options.addOption("t", "test", true, "test args");
+
+        CommandLineParser parser = new DefaultParser();
+        CommandLine cl = parser.parse(options, new String[]{"-test all the"});
+        Assert.assertTrue(cl.hasOption("test"));
+    }
+
+}
diff --git a/inlong-agent/agent-core/src/test/java/org/apache/inlong/agent/core/TestCompletableFuture.java b/inlong-agent/agent-core/src/test/java/org/apache/inlong/agent/core/TestCompletableFuture.java
new file mode 100755
index 0000000..d27e004
--- /dev/null
+++ b/inlong-agent/agent-core/src/test/java/org/apache/inlong/agent/core/TestCompletableFuture.java
@@ -0,0 +1,114 @@
+/*
+ * 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.inlong.agent.core;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import org.junit.Assert;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class TestCompletableFuture {
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(TestCompletableFuture.class);
+
+    @Test
+    public void testFuture() throws Exception {
+        ExecutorService service = Executors.newSingleThreadExecutor();
+        CompletableFuture<?> future = CompletableFuture.runAsync(() -> {
+            LOGGER.info("start run async");
+        }, service);
+        future.join();
+        service.shutdown();
+    }
+
+    @Test
+    public void testFutureException() throws Exception {
+        ExecutorService service = Executors.newCachedThreadPool();
+        List<CompletableFuture<?>> result = new ArrayList<>();
+        result.add(CompletableFuture.runAsync(() -> {
+            try {
+                LOGGER.info("test future1");
+                TimeUnit.SECONDS.sleep(1);
+                throw new RuntimeException("test exception1");
+            } catch (InterruptedException ex) {
+                LOGGER.error("ignore exception {}", ex.getMessage());
+            }
+        }, service));
+        result.add(CompletableFuture.runAsync(() -> {
+            try {
+                LOGGER.info("test future2");
+                TimeUnit.MILLISECONDS.sleep(200);
+                throw new RuntimeException("test exception2");
+            } catch (InterruptedException ex) {
+                LOGGER.error("ignore exception {}", ex.getMessage());
+            }
+        }, service));
+        try {
+            CompletableFuture.allOf(result.toArray(new CompletableFuture[0])).join();
+            Assert.fail("future should fail");
+        } catch (Exception ex) {
+            Assert.assertTrue(ex.getMessage().contains("test exception"));
+        }
+
+        service.shutdown();
+    }
+
+    @Test
+    public void testFutureComplete() throws Exception {
+        ExecutorService service = Executors.newCachedThreadPool();
+        CompletableFuture<?> reader = CompletableFuture.runAsync(() -> {
+            try {
+                TimeUnit.SECONDS.sleep(1);
+                throw new RuntimeException("test exception1");
+            } catch (InterruptedException ex) {
+                LOGGER.error("ignore exception {}", ex.getMessage());
+            }
+        }, service);
+
+        CompletableFuture<?> writer = CompletableFuture.runAsync(() -> {
+            try {
+                int count = 0;
+                while (count < 10) {
+                    LOGGER.info("test writer");
+                    count += 1;
+                    TimeUnit.MILLISECONDS.sleep(200);
+                }
+
+            } catch (InterruptedException ex) {
+                LOGGER.error("ignore exception {}", ex.getMessage());
+            }
+        }, service);
+
+        CompletableFuture.anyOf(reader, writer).exceptionally(ex -> {
+            Assert.assertTrue(ex.getMessage().contains("test exception1"));
+            Arrays.asList(reader, writer).forEach(future -> future.completeExceptionally(
+                    new RuntimeException("all exception")));
+            return null;
+        }).join();
+        Assert.assertTrue(reader.isCompletedExceptionally());
+        Assert.assertTrue(writer.isCompletedExceptionally());
+        service.shutdown();
+    }
+}
diff --git a/inlong-agent/agent-core/src/test/java/org/apache/inlong/agent/core/TestConfigJetty.java b/inlong-agent/agent-core/src/test/java/org/apache/inlong/agent/core/TestConfigJetty.java
new file mode 100755
index 0000000..7cac146
--- /dev/null
+++ b/inlong-agent/agent-core/src/test/java/org/apache/inlong/agent/core/TestConfigJetty.java
@@ -0,0 +1,125 @@
+/*
+ * 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.inlong.agent.core;
+
+import static org.apache.inlong.agent.constants.AgentConstants.AGENT_CONF_PARENT;
+import static org.apache.inlong.agent.constants.AgentConstants.AGENT_ENABLE_HTTP;
+import static org.apache.inlong.agent.constants.AgentConstants.AGENT_HTTP_PORT;
+import static org.apache.inlong.agent.constants.AgentConstants.DEFAULT_AGENT_HTTP_PORT;
+
+import com.google.gson.Gson;
+import org.apache.http.client.config.RequestConfig;
+import org.apache.http.client.methods.CloseableHttpResponse;
+import org.apache.http.client.methods.HttpPost;
+import org.apache.http.entity.ContentType;
+import org.apache.http.entity.StringEntity;
+import org.apache.http.impl.client.CloseableHttpClient;
+import org.apache.http.impl.client.HttpClientBuilder;
+import org.apache.http.util.EntityUtils;
+import org.apache.inlong.agent.conf.AgentConfiguration;
+import org.apache.inlong.agent.core.conf.ResponseResult;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestConfigJetty {
+
+    private static CloseableHttpClient httpClient;
+    private static AgentConfiguration configuration;
+    private static final Gson gson = new Gson();
+    private static AgentManager manager;
+    private static AgentBaseTestsHelper helper;
+
+    @BeforeClass
+    public static void setup() {
+        configuration = AgentConfiguration.getAgentConf();
+        configuration.setBoolean(AGENT_ENABLE_HTTP, true);
+        helper = new AgentBaseTestsHelper(TestConfigJetty.class.getName()).setupAgentHome();
+        manager = new AgentManager();
+        RequestConfig requestConfig = RequestConfig.custom().build();
+        HttpClientBuilder httpClientBuilder = HttpClientBuilder.create();
+        httpClientBuilder.setDefaultRequestConfig(requestConfig);
+        httpClient = httpClientBuilder.build();
+    }
+
+    @Test
+    public void testJobConfig() throws Exception {
+        int webPort = configuration.getInt(AGENT_HTTP_PORT, DEFAULT_AGENT_HTTP_PORT);
+        String url = "http://localhost:" + webPort + "/config/";
+        HttpPost httpPost = new HttpPost(url);
+        CloseableHttpResponse response = httpClient.execute(httpPost);
+        String returnStr = EntityUtils.toString(response.getEntity());
+        ResponseResult responseResult = gson.fromJson(returnStr, ResponseResult.class);
+        Assert.assertEquals(-1, responseResult.getCode());
+        Assert.assertEquals("child path is not correct", responseResult.getMessage());
+
+        url = "http://localhost:" + webPort + "/config/job";
+        httpPost = new HttpPost(url);
+
+        String jsonStr = "{\n"
+            + "  \"op\": 0,\n"
+            + "  \"job\": {\n"
+            + "    \"file\": {\n"
+            + "      \"max\": {\n"
+            + "        \"wait\": 1\n"
+            + "      }\n"
+            + "    },\n"
+            + "    \"trigger\": \"org.apache.inlong.agent.plugin.trigger.DirectoryTrigger\",\n"
+            + "    \"op\": 0,\n"
+            + "    \"deliveryTime\": \"12313123\",\n"
+            + "    \"dir\": {\n"
+            + "      \"path\": \"\",\n"
+            + "      \"pattern\": \"/test.[0-9]\"\n"
+            + "    },\n"
+            + "    \"id\": 1,\n"
+            + "    \"name\": \"fileAgentTest\",\n"
+            + "    \"source\": \"org.apache.inlong.agent.plugin.sources.TextFileSource\",\n"
+            + "    \"sink\": \"org.apache.inlong.agent.plugin.sinks.MockSink\",\n"
+            + "    \"channel\": \"org.apache.inlong.agent.plugin.channel.MemoryChannel\",\n"
+            + "    \"pattern\": \"test\"\n"
+            + "  }\n"
+            + "}";
+        StringEntity entity = new StringEntity(jsonStr, ContentType.APPLICATION_JSON);
+        httpPost.setEntity(entity);
+
+        response = httpClient.execute(httpPost);
+        returnStr = EntityUtils.toString(response.getEntity());
+        responseResult = gson.fromJson(returnStr, ResponseResult.class);
+        Assert.assertEquals(0, responseResult.getCode());
+    }
+
+    @Test
+    public void testAgentConfig() throws Exception {
+        int webPort = configuration.getInt(AGENT_HTTP_PORT, DEFAULT_AGENT_HTTP_PORT);
+        configuration.set(AGENT_CONF_PARENT, helper.getTestRootDir().toString());
+        String url = "http://localhost:" + webPort + "/config/agent";
+        HttpPost httpPost = new HttpPost(url);
+        CloseableHttpResponse response = httpClient.execute(httpPost);
+        String returnStr = EntityUtils.toString(response.getEntity());
+        ResponseResult responseResult = gson.fromJson(returnStr, ResponseResult.class);
+        Assert.assertEquals(0, responseResult.getCode());
+    }
+
+    @AfterClass
+    public static void teardown() throws Exception {
+        httpClient.close();
+        manager.stop();
+        helper.teardownAgentHome();
+    }
+}
diff --git a/inlong-agent/agent-core/src/test/java/org/apache/inlong/agent/task/TestTaskWrapper.java b/inlong-agent/agent-core/src/test/java/org/apache/inlong/agent/task/TestTaskWrapper.java
new file mode 100755
index 0000000..10b4829
--- /dev/null
+++ b/inlong-agent/agent-core/src/test/java/org/apache/inlong/agent/task/TestTaskWrapper.java
@@ -0,0 +1,185 @@
+/*
+ * 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.inlong.agent.task;
+
+import static org.awaitility.Awaitility.await;
+
+import java.nio.charset.StandardCharsets;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.TimeUnit;
+import org.apache.inlong.agent.core.AgentBaseTestsHelper;
+import org.apache.inlong.agent.conf.JobProfile;
+import org.apache.inlong.agent.core.AgentManager;
+import org.apache.inlong.agent.core.task.Task;
+import org.apache.inlong.agent.core.task.TaskWrapper;
+import org.apache.inlong.agent.message.DefaultMessage;
+import org.apache.inlong.agent.plugin.Channel;
+import org.apache.inlong.agent.plugin.Message;
+import org.apache.inlong.agent.plugin.Reader;
+import org.apache.inlong.agent.plugin.Sink;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class TestTaskWrapper {
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(TestTaskWrapper.class);
+
+    private static AgentManager manager;
+    private static Task task;
+    private static WriterImpl writer;
+    private static ReaderImpl reader;
+    private static AgentBaseTestsHelper helper;
+
+    private static final String className = TestTaskWrapper.class.getName();
+
+    public static class MockChannel implements Channel {
+
+        private final LinkedBlockingQueue<Message> queue = new LinkedBlockingQueue<>();
+
+        @Override
+        public void push(Message message) {
+            queue.offer(message);
+        }
+
+        @Override
+        public boolean push(Message message, long timeout, TimeUnit unit) {
+            return queue.offer(message);
+        }
+
+        @Override
+        public Message pull(long timeout, TimeUnit unit) {
+            return queue.poll();
+        }
+
+        @Override
+        public void init(JobProfile jobConf) {
+
+        }
+
+        @Override
+        public void destroy() {
+            queue.clear();
+        }
+    }
+
+    @BeforeClass
+    public static void setup() throws Exception {
+        helper = new AgentBaseTestsHelper(TestTaskWrapper.class.getName()).setupAgentHome();
+        manager = new AgentManager();
+        reader = new ReaderImpl();
+        writer = new WriterImpl();
+        task = new Task("test", reader, writer,
+            new MockChannel(), JobProfile.parseJsonStr(""));
+    }
+
+    @AfterClass
+    public static void teardown() throws Exception {
+        manager.stop();
+        helper.teardownAgentHome();
+    }
+
+    @Test
+    public void testTaskRunning() throws Exception {
+        manager.getTaskManager().submitTask(task);
+        String jobId = "test";
+        TaskWrapper wrapper = manager.getTaskManager().getTaskWrapper(jobId);
+        assert wrapper != null;
+        while (!wrapper.isSuccess()) {
+            LOGGER.info("waiting for success");
+            TimeUnit.MILLISECONDS.sleep(100);
+        }
+        await().atMost(200, TimeUnit.SECONDS).until(()
+                -> reader.getCount() == writer.getWriterCount() + 1);
+        Assert.assertEquals("reader writer not equal", reader.getCount(),
+            writer.getWriterCount() + 1);
+    }
+
+    private static class ReaderImpl implements Reader {
+
+        private int count = 0;
+
+        @Override
+        public Message read() {
+            count += 1;
+            return new DefaultMessage("".getBytes(StandardCharsets.UTF_8));
+        }
+
+        @Override
+        public boolean isFinished() {
+            return count > 99999;
+        }
+
+        @Override
+        public String getReadFile() {
+            return null;
+        }
+
+        @Override
+        public void setReadTimeout(long mill) {
+
+        }
+
+        public int getCount() {
+            return count;
+        }
+
+        @Override
+        public void init(JobProfile jobConf) {
+        }
+
+        @Override
+        public void destroy() {
+
+        }
+    }
+
+    private static class WriterImpl implements Sink {
+
+        private int writerCount = 0;
+
+        @Override
+        public void write(Message message) {
+            if (message != null) {
+                writerCount += 1;
+            }
+        }
+
+        @Override
+        public void setSourceFile(String sourceFileName) {
+
+        }
+
+        public int getWriterCount() {
+            return writerCount;
+        }
+
+        @Override
+        public void init(JobProfile jobConf) {
+
+        }
+
+        @Override
+        public void destroy() {
+
+        }
+    }
+}
diff --git a/inlong-agent/agent-core/src/test/resources/agent.properties b/inlong-agent/agent-core/src/test/resources/agent.properties
new file mode 100755
index 0000000..87d6370
--- /dev/null
+++ b/inlong-agent/agent-core/src/test/resources/agent.properties
@@ -0,0 +1,22 @@
+#
+# 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.
+#
+
+# default values of agent
+agent.version=0.0.1-SNAPSHOT
+agent.maxSize=10
+agent.conf.resource=manager
+job.thread.running.core=10
diff --git a/inlong-agent/agent-core/src/test/resources/log4j.properties b/inlong-agent/agent-core/src/test/resources/log4j.properties
new file mode 100644
index 0000000..a64a3b6
--- /dev/null
+++ b/inlong-agent/agent-core/src/test/resources/log4j.properties
@@ -0,0 +1,20 @@
+#
+# 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.
+#
+log4j.rootLogger=INFO, out
+log4j.appender.out=org.apache.log4j.ConsoleAppender
+log4j.appender.out.layout=org.apache.log4j.PatternLayout
+log4j.appender.out.layout.ConversionPattern=%d (%t) [%p - %l] %m%n
\ No newline at end of file
diff --git a/inlong-agent/agent-plugins/pom.xml b/inlong-agent/agent-plugins/pom.xml
new file mode 100644
index 0000000..8f7ca65
--- /dev/null
+++ b/inlong-agent/agent-plugins/pom.xml
@@ -0,0 +1,127 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+    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.
+-->
+<project xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+    xmlns="http://maven.apache.org/POM/4.0.0"
+    xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <parent>
+        <groupId>org.apache.inlong</groupId>
+        <artifactId>inlong-agent</artifactId>
+        <version>0.9.0-incubating-SNAPSHOT</version>
+    </parent>
+    <artifactId>agent-plugins</artifactId>
+    <modelVersion>4.0.0</modelVersion>
+    <name>Apache InLong - Agent Plugins</name>
+
+    <repositories>
+        <repository>
+            <id>central</id>
+            <name>Central Repository</name>
+            <url>https://repo.maven.apache.org/maven2</url>
+            <layout>default</layout>
+            <snapshots>
+                <enabled>false</enabled>
+            </snapshots>
+        </repository>
+    </repositories>
+    <properties>
+        <powermock.version>2.0.2</powermock.version>
+    </properties>
+
+    <dependencies>
+        <dependency>
+            <artifactId>guava</artifactId>
+            <groupId>com.google.guava</groupId>
+        </dependency>
+        <dependency>
+            <groupId>org.powermock</groupId>
+            <artifactId>powermock-module-junit4</artifactId>
+            <version>${powermock.version}</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.powermock</groupId>
+            <artifactId>powermock-api-mockito</artifactId>
+            <version>1.7.4</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.inlong</groupId>
+            <artifactId>agent-core</artifactId>
+        </dependency>
+        <dependency>
+            <artifactId>slf4j-api</artifactId>
+            <groupId>org.slf4j</groupId>
+        </dependency>
+        <dependency>
+            <groupId>junit</groupId>
+            <artifactId>junit</artifactId>
+            <version>4.12</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.inlong</groupId>
+            <artifactId>inlong-dataproxy-sdk</artifactId>
+            <version>0.9.0-incubating-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <artifactId>slf4j-log4j12</artifactId>
+            <groupId>org.slf4j</groupId>
+        </dependency>
+        <dependency>
+            <artifactId>junit</artifactId>
+            <groupId>junit</groupId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <artifactId>mockito-core</artifactId>
+            <groupId>org.mockito</groupId>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <artifactId>awaitility</artifactId>
+            <groupId>org.awaitility</groupId>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.pulsar</groupId>
+            <artifactId>pulsar-client</artifactId>
+        </dependency>
+
+        <dependency>
+            <artifactId>byte-buddy</artifactId>
+            <groupId>net.bytebuddy</groupId>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.projectlombok</groupId>
+            <artifactId>lombok</artifactId>
+            <version>RELEASE</version>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.inlong</groupId>
+            <artifactId>agent-common</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+    </dependencies>
+</project>
\ No newline at end of file
diff --git a/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/channel/MemoryChannel.java b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/channel/MemoryChannel.java
new file mode 100644
index 0000000..93f5e21
--- /dev/null
+++ b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/channel/MemoryChannel.java
@@ -0,0 +1,111 @@
+/*
+ * 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.inlong.agent.plugin.channel;
+
+
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.TimeUnit;
+import org.apache.inlong.agent.conf.JobProfile;
+import org.apache.inlong.agent.constants.AgentConstants;
+import org.apache.inlong.agent.plugin.Channel;
+import org.apache.inlong.agent.plugin.Message;
+import org.apache.inlong.agent.plugin.metrics.PluginMetric;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class MemoryChannel implements Channel {
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(MemoryChannel.class);
+
+    private LinkedBlockingQueue<Message> queue;
+
+    private final PluginMetric metric = new PluginMetric();
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public void push(Message message) {
+        try {
+            if (message != null) {
+                metric.readNum.incr();
+                queue.put(message);
+                metric.readSuccessNum.incr();
+            }
+        } catch (InterruptedException ex) {
+            metric.readFailedNum.incr();
+            Thread.currentThread().interrupt();
+        }
+    }
+
+    @Override
+    public boolean push(Message message, long timeout, TimeUnit unit) {
+        try {
+            if (message != null) {
+                metric.readNum.incr();
+                boolean result = queue.offer(message, timeout, unit);
+                if (result) {
+                    metric.readSuccessNum.incr();
+                } else {
+                    metric.readFailedNum.incr();
+                }
+                return result;
+            }
+        } catch (InterruptedException ex) {
+            metric.readFailedNum.incr();
+            Thread.currentThread().interrupt();
+        }
+        return false;
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public Message pull(long timeout, TimeUnit unit) {
+        try {
+            Message message = queue.poll(timeout, unit);
+            if (message != null) {
+                metric.sendSuccessNum.incr();
+            }
+            return message;
+        } catch (InterruptedException ex) {
+            metric.sendFailedNum.incr();
+            Thread.currentThread().interrupt();
+            throw new IllegalStateException(ex);
+        }
+    }
+
+    @Override
+    public void init(JobProfile jobConf) {
+        queue = new LinkedBlockingQueue<>(
+                jobConf.getInt(AgentConstants.CHANNEL_MEMORY_CAPACITY,
+                    AgentConstants.DEFAULT_CHANNEL_MEMORY_CAPACITY));
+    }
+
+    @Override
+    public void destroy() {
+        if (queue != null) {
+            queue.clear();
+        }
+        LOGGER.info("destroy channel, memory channel metric, readNum: {}, readSuccessNum: {}, "
+            + "readFailedNum: {}, sendSuccessNum: {}, sendFailedNum: {}",
+            metric.readNum.snapshot(), metric.readSuccessNum.snapshot(), metric.readFailedNum.snapshot(),
+            metric.sendSuccessNum.snapshot(), metric.sendFailedNum.snapshot());
+    }
+}
diff --git a/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/except/FileException.java b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/except/FileException.java
new file mode 100644
index 0000000..e86c223
--- /dev/null
+++ b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/except/FileException.java
@@ -0,0 +1,25 @@
+/*
+ * 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.inlong.agent.plugin.except;
+
+public class FileException extends RuntimeException {
+
+    public FileException(String message, Throwable ex) {
+        super(message, ex);
+    }
+}
diff --git a/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/fetcher/ManagerFetcher.java b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/fetcher/ManagerFetcher.java
new file mode 100755
index 0000000..1012b9c
--- /dev/null
+++ b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/fetcher/ManagerFetcher.java
@@ -0,0 +1,409 @@
+/*
+ * 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.inlong.agent.plugin.fetcher;
+
+
+import static java.util.Objects.requireNonNull;
+import static org.apache.inlong.agent.constants.AgentConstants.AGENT_HOME;
+import static org.apache.inlong.agent.constants.AgentConstants.AGENT_LOCAL_CACHE;
+import static org.apache.inlong.agent.constants.AgentConstants.AGENT_LOCAL_CACHE_TIMEOUT;
+import static org.apache.inlong.agent.constants.AgentConstants.AGENT_LOCAL_IP;
+import static org.apache.inlong.agent.constants.AgentConstants.AGENT_UNIQ_ID;
+import static org.apache.inlong.agent.constants.AgentConstants.DEFAULT_AGENT_HOME;
+import static org.apache.inlong.agent.constants.AgentConstants.DEFAULT_AGENT_LOCAL_CACHE;
+import static org.apache.inlong.agent.constants.AgentConstants.DEFAULT_AGENT_LOCAL_CACHE_TIMEOUT;
+import static org.apache.inlong.agent.constants.AgentConstants.DEFAULT_AGENT_UNIQ_ID;
+import static org.apache.inlong.agent.constants.JobConstants.JOB_OP;
+import static org.apache.inlong.agent.constants.JobConstants.JOB_RETRY_TIME;
+import static org.apache.inlong.agent.plugin.fetcher.ManagerResultFormatter.getResultData;
+import static org.apache.inlong.agent.plugin.fetcher.constants.FetcherConstants.AGENT_FETCHER_INTERVAL;
+import static org.apache.inlong.agent.plugin.fetcher.constants.FetcherConstants.AGENT_MANAGER_VIP_HTTP_HOST;
+import static org.apache.inlong.agent.plugin.fetcher.constants.FetcherConstants.AGENT_MANAGER_VIP_HTTP_PORT;
+import static org.apache.inlong.agent.plugin.fetcher.constants.FetcherConstants.AGENT_MANAGER_IP_CHECK_HTTP_PATH;
+import static org.apache.inlong.agent.plugin.fetcher.constants.FetcherConstants.AGENT_MANAGER_RETURN_PARAM_DATA;
+import static org.apache.inlong.agent.plugin.fetcher.constants.FetcherConstants.AGENT_MANAGER_RETURN_PARAM_IP;
+import static org.apache.inlong.agent.plugin.fetcher.constants.FetcherConstants.AGENT_MANAGER_TASK_HTTP_PATH;
+import static org.apache.inlong.agent.plugin.fetcher.constants.FetcherConstants.AGENT_MANAGER_VIP_HTTP_PATH;
+import static org.apache.inlong.agent.plugin.fetcher.constants.FetcherConstants.AGENT_MANAGER_VIP_HTTP_PREFIX_PATH;
+import static org.apache.inlong.agent.plugin.fetcher.constants.FetcherConstants.DEFAULT_AGENT_FETCHER_INTERVAL;
+import static org.apache.inlong.agent.plugin.fetcher.constants.FetcherConstants.DEFAULT_AGENT_TDM_IP_CHECK_HTTP_PATH;
+import static org.apache.inlong.agent.plugin.fetcher.constants.FetcherConstants.DEFAULT_AGENT_MANAGER_TASK_HTTP_PATH;
+import static org.apache.inlong.agent.plugin.fetcher.constants.FetcherConstants.DEFAULT_AGENT_TDM_VIP_HTTP_PATH;
+import static org.apache.inlong.agent.plugin.fetcher.constants.FetcherConstants.DEFAULT_AGENT_MANAGER_VIP_HTTP_PREFIX_PATH;
+import static org.apache.inlong.agent.plugin.fetcher.constants.FetcherConstants.DEFAULT_LOCAL_IP;
+import static org.apache.inlong.agent.plugin.utils.PluginUtils.copyJobProfile;
+
+import com.google.gson.Gson;
+import com.google.gson.JsonArray;
+import com.google.gson.JsonElement;
+import com.google.gson.JsonObject;
+import java.io.File;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+import org.apache.inlong.agent.cache.LocalFileCache;
+import org.apache.inlong.agent.common.AbstractDaemon;
+import org.apache.inlong.agent.conf.AgentConfiguration;
+import org.apache.inlong.agent.conf.JobProfile;
+import org.apache.inlong.agent.conf.ProfileFetcher;
+import org.apache.inlong.agent.conf.TriggerProfile;
+import org.apache.inlong.agent.core.AgentManager;
+import org.apache.inlong.agent.db.CommandDb;
+import org.apache.inlong.agent.db.CommandEntity;
+import org.apache.inlong.agent.plugin.Trigger;
+import org.apache.inlong.agent.plugin.fetcher.dtos.CmdConfig;
+import org.apache.inlong.agent.plugin.fetcher.dtos.ConfirmAgentIpRequest;
+import org.apache.inlong.agent.plugin.fetcher.dtos.TaskRequsetDto;
+import org.apache.inlong.agent.plugin.fetcher.dtos.TaskResult;
+import org.apache.inlong.agent.plugin.fetcher.enums.ManagerOpEnum;
+import org.apache.inlong.agent.plugin.utils.HttpManager;
+import org.apache.inlong.agent.plugin.utils.PluginUtils;
+import org.apache.inlong.agent.utils.AgentUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * fetch command from manager
+ */
+public class ManagerFetcher extends AbstractDaemon implements ProfileFetcher {
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(ManagerFetcher.class);
+
+    private static final Gson GSON = new Gson();
+    private static final int MAX_RETRY = 2;
+    public static final String AGENT = "agent";
+    private final String managerVipUrl;
+    private final String baseManagerUrl;
+    private final String managerTaskUrl;
+    private final String managerIpsCheckUrl;
+    private final AgentConfiguration conf;
+    private final LocalFileCache localFileCache;
+    private final String uniqId;
+    private List<String> managerList;
+    private final AgentManager agentManager;
+    private final HttpManager httpManager;
+    private String localIp;
+
+
+
+    private CommandDb commandDb;
+
+    private boolean requiredKeys(AgentConfiguration conf) {
+        return conf.hasKey(AGENT_MANAGER_VIP_HTTP_HOST) && conf.hasKey(AGENT_MANAGER_VIP_HTTP_PORT);
+    }
+
+    public ManagerFetcher(AgentManager agentManager) {
+        this.agentManager = agentManager;
+        this.conf = AgentConfiguration.getAgentConf();
+        if (requiredKeys(conf)) {
+            httpManager = new HttpManager(conf);
+            baseManagerUrl = "http://" + conf.get(AGENT_MANAGER_VIP_HTTP_HOST)
+                + ":" + conf.get(AGENT_MANAGER_VIP_HTTP_PORT) + conf.get(
+                AGENT_MANAGER_VIP_HTTP_PREFIX_PATH,
+                DEFAULT_AGENT_MANAGER_VIP_HTTP_PREFIX_PATH);
+            managerVipUrl = baseManagerUrl + conf.get(AGENT_MANAGER_VIP_HTTP_PATH,
+                DEFAULT_AGENT_TDM_VIP_HTTP_PATH);
+            managerTaskUrl = baseManagerUrl + conf.get(AGENT_MANAGER_TASK_HTTP_PATH,
+                DEFAULT_AGENT_MANAGER_TASK_HTTP_PATH);
+            managerIpsCheckUrl = baseManagerUrl + conf.get(AGENT_MANAGER_IP_CHECK_HTTP_PATH,
+                DEFAULT_AGENT_TDM_IP_CHECK_HTTP_PATH);
+            uniqId = conf.get(AGENT_UNIQ_ID, DEFAULT_AGENT_UNIQ_ID);
+            Path localStorage = Paths.get(conf.get(AGENT_HOME, DEFAULT_AGENT_HOME),
+                conf.get(AGENT_LOCAL_CACHE, DEFAULT_AGENT_LOCAL_CACHE), "managerList.txt");
+            long timeout = TimeUnit.MINUTES.toMillis(conf.getInt(AGENT_LOCAL_CACHE_TIMEOUT,
+                DEFAULT_AGENT_LOCAL_CACHE_TIMEOUT));
+            localFileCache = new LocalFileCache(localStorage.toFile(), timeout);
+            this.commandDb = agentManager.getCommandDb();
+        } else {
+            throw new RuntimeException("init manager error, cannot find required key");
+        }
+    }
+
+
+
+    /**
+     * for manager to get job profiles
+     * @return -  job profile list
+     */
+    @Override
+    public List<JobProfile> getJobProfiles() {
+        getTriggerProfiles();
+        return null;
+    }
+
+    public void requestTdmList() {
+        JsonObject result = getResultData(httpManager.doSendGet(managerVipUrl));
+        JsonArray data = result.get(AGENT_MANAGER_RETURN_PARAM_DATA).getAsJsonArray();
+        List<String> managerIpList = new ArrayList<>();
+        for (JsonElement datum : data) {
+            JsonObject asJsonObject = datum.getAsJsonObject();
+            managerIpList.add(asJsonObject.get(AGENT_MANAGER_RETURN_PARAM_IP).getAsString());
+        }
+        if (managerIpList.isEmpty()) {
+            return;
+        }
+        localFileCache.writeToCache(String.join(",", managerIpList));
+    }
+
+
+    /**
+     * request manager to get commands, make sure it not throwing exceptions
+     */
+    public void fetchCommand() {
+        List<CommandEntity> unackedCommands = commandDb.getUnackedCommands();
+        JsonObject resultData = getResultData(httpManager.doSentPost(managerTaskUrl, getFetchRequest(unackedCommands)));
+        dealWithFetchResult(GSON.fromJson(resultData.get(AGENT_MANAGER_RETURN_PARAM_DATA).getAsJsonObject(),
+            TaskResult.class));
+        ackCommands(unackedCommands);
+    }
+
+    private void ackCommands(List<CommandEntity> unackedCommands) {
+        for (CommandEntity command : unackedCommands) {
+            command.setAcked(true);
+            commandDb.storeCommand(command);
+        }
+    }
+
+    /**
+     * the fetch command can be normal or special
+     * @param taskResult
+     */
+    private void dealWithFetchResult(TaskResult taskResult) {
+        LOGGER.info("deal with fetch result {}", taskResult);
+        for (TriggerProfile profile : taskResult.getTriggerProfiles()) {
+            dealWithTdmTriggerProfile(profile);
+        }
+        for (CmdConfig cmdConfig : taskResult.getCmdConfigs()) {
+            dealWithTdmCmd(cmdConfig);
+        }
+    }
+
+    /**
+     * form command fetch request
+     * @return
+     * @param unackedCommands
+     */
+    public TaskRequsetDto getFetchRequest(
+        List<CommandEntity> unackedCommands) {
+        TaskRequsetDto requset = new TaskRequsetDto();
+        requset.setAgentIp(localIp);
+        requset.setCommandInfo(unackedCommands);
+        return requset;
+    }
+
+    /**
+     * get command db
+     * @return
+     */
+    public CommandDb getCommandDb() {
+        return commandDb;
+    }
+
+    /**
+     * deal with special command retry\backtrack
+     * @param cmdConfig
+     */
+    public void dealWithTdmCmd(CmdConfig cmdConfig) {
+        Trigger trigger = agentManager.getTriggerManager().getTrigger(
+            cmdConfig.getTaskId().toString());
+        if (trigger == null) {
+            LOGGER.error("trigger {} doesn't exist, cmd is {}",
+                cmdConfig.getTaskId(), cmdConfig);
+            commandDb.saveSpecialCmds(cmdConfig.getId(),
+                cmdConfig.getTaskId(), false);
+            return;
+        }
+        TriggerProfile copiedProfile =
+            TriggerProfile.parseJsonStr(trigger.getTriggerProfile().toJsonStr());
+        String dataTime = cmdConfig.getDataTime();
+        // set job retry time
+        copiedProfile.set(JOB_RETRY_TIME, dataTime);
+        boolean cmdResult = executeCmd(copiedProfile,
+            ManagerOpEnum.getOpType(cmdConfig.getOp()), dataTime);
+        commandDb.saveSpecialCmds(cmdConfig.getId(),
+            cmdConfig.getTaskId(), cmdResult);
+    }
+
+    /**
+     * execute commands
+     * @param triggerProfile
+     * @param opType
+     * @param dataTime
+     * @return
+     */
+    private boolean executeCmd(TriggerProfile triggerProfile,
+        ManagerOpEnum opType, String dataTime) {
+        switch (opType) {
+            case RETRY:
+            case BACKTRACK:
+                return agentManager.getJobManager().submitJobProfile(triggerProfile);
+            case MAKEUP:
+                return makeUpFiles(triggerProfile, dataTime);
+            case CHECK:
+                return !PluginUtils.findSuitFiles(triggerProfile).isEmpty();
+            default:
+        }
+        LOGGER.error("do not support such opType {}", opType);
+        return false;
+    }
+
+    /**
+     * when execute make up command, files scanned before should not be executed.
+     * @param triggerProfile
+     * @param dataTime
+     * @return
+     */
+    private boolean makeUpFiles(TriggerProfile triggerProfile, String dataTime) {
+        LOGGER.info("start to make up files with trigger {}, dataTime {}",
+            triggerProfile, dataTime);
+        Collection<File> suitFiles = PluginUtils.findSuitFiles(triggerProfile);
+        // filter files exited before
+        List<File> pendingFiles = suitFiles.stream().filter(file ->
+            !agentManager.getJobManager().checkJobExsit(file.getAbsolutePath()))
+            .collect(Collectors.toList());
+        for (File pendingFile : pendingFiles) {
+            JobProfile copiedProfile = copyJobProfile(triggerProfile, dataTime,
+                pendingFile);
+            LOGGER.info("ready to make up file with job {}", copiedProfile.toJsonStr());
+            agentManager.getJobManager().submitJobProfile(copiedProfile);
+        }
+        return true;
+    }
+
+    /**
+     * the trigger profile returned from manager should be parsed
+     * @param triggerProfile
+     */
+    public void dealWithTdmTriggerProfile(TriggerProfile triggerProfile) {
+        ManagerOpEnum opType = ManagerOpEnum.getOpType(triggerProfile.getInt(JOB_OP));
+        boolean success = false;
+        switch (requireNonNull(opType)) {
+            case ACTIVE:
+            case ADD:
+                success = agentManager.getTriggerManager().submitTrigger(triggerProfile);
+                break;
+            case DEL:
+            case FROZEN:
+                success = agentManager.getTriggerManager().deleteTrigger(triggerProfile.getTriggerId());
+                break;
+            default:
+        }
+        commandDb.saveNormalCmds(triggerProfile, success);
+    }
+
+
+    /**
+     * check agent ip from manager
+     */
+    private void fetchLocalIp() {
+        localIp = AgentConfiguration.getAgentConf().get(AGENT_LOCAL_IP, DEFAULT_LOCAL_IP);
+    }
+
+
+    /**
+     * confirm local ips from manager
+     * @param localIps
+     * @return
+     */
+    private String confirmLocalIps(List<String> localIps) {
+        ConfirmAgentIpRequest request = new ConfirmAgentIpRequest(AGENT, localIps);
+        JsonObject resultData = getResultData(httpManager.doSentPost(managerIpsCheckUrl, request))
+            .get(AGENT_MANAGER_RETURN_PARAM_DATA).getAsJsonObject();
+        if (!resultData.has(AGENT_MANAGER_RETURN_PARAM_IP)) {
+            throw new IllegalArgumentException("cannot get ip from data " + resultData.getAsString());
+        }
+        return resultData.get(AGENT_MANAGER_RETURN_PARAM_IP).getAsString();
+    }
+
+
+    /**
+     * fetch manager list, make sure it not throwing exceptions
+     * @param isInitial - is initial
+     * @param retryTime - retry time
+     */
+    private void fetchTdmList(boolean isInitial, int retryTime) {
+        if (retryTime > MAX_RETRY) {
+            return;
+        }
+        try {
+            // check local cache time, make sure cache not timeout
+            if (!isInitial && !localFileCache.cacheIsExpired()) {
+                String result = localFileCache.getCacheInfo();
+                managerList = Arrays.stream(result.split(","))
+                    .map(String::trim)
+                    .collect(Collectors.toList());
+            } else {
+                requestTdmList();
+            }
+        } catch (Exception ex) {
+            fetchTdmList(false, retryTime + 1);
+        }
+    }
+
+
+    /**
+     * thread for profile fetcher.
+     *
+     * @return runnable profile.
+     */
+    private Runnable profileFetchThread() {
+        return () -> {
+            while (isRunnable()) {
+                try {
+                    int configSleepTime = conf.getInt(AGENT_FETCHER_INTERVAL,
+                        DEFAULT_AGENT_FETCHER_INTERVAL);
+                    TimeUnit.SECONDS.sleep(AgentUtils.getRandomBySeed(configSleepTime));
+                    // fetch commands from manager
+                    fetchCommand();
+                    // fetch manager list from vip
+                    fetchTdmList(false, 0);
+                } catch (Exception ex) {
+                    LOGGER.warn("exception caught", ex);
+                }
+            }
+        };
+    }
+
+    /**
+     * request manager to get trigger profiles.
+     * @return - trigger profile list
+     */
+    @Override
+    public List<TriggerProfile> getTriggerProfiles() {
+        return null;
+    }
+
+    @Override
+    public void start() throws Exception {
+        // when agent start, check local ip and fetch manager ip list;
+        fetchLocalIp();
+        fetchTdmList(true, 0);
+        submitWorker(profileFetchThread());
+    }
+
+    @Override
+    public void stop() {
+        waitForTerminate();
+    }
+}
diff --git a/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/fetcher/ManagerResultFormatter.java b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/fetcher/ManagerResultFormatter.java
new file mode 100755
index 0000000..b3c3907
--- /dev/null
+++ b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/fetcher/ManagerResultFormatter.java
@@ -0,0 +1,136 @@
+/*
+ * 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.inlong.agent.plugin.fetcher;
+
+import static org.apache.inlong.agent.plugin.fetcher.constants.FetcherConstants.AGENT_MANAGER_VIP_HTTP_HOST;
+import static org.apache.inlong.agent.plugin.fetcher.constants.FetcherConstants.AGENT_MANAGER_VIP_HTTP_PORT;
+
+import com.google.gson.Gson;
+import com.google.gson.JsonObject;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import org.apache.inlong.agent.conf.AgentConfiguration;
+import org.apache.inlong.agent.conf.TriggerProfile;
+import org.apache.inlong.agent.plugin.fetcher.dtos.DataConfig;
+import org.apache.inlong.agent.plugin.fetcher.dtos.JobProfileDto;
+import org.apache.inlong.agent.plugin.fetcher.dtos.JobProfileDto.Proxy;
+import org.apache.inlong.agent.plugin.fetcher.dtos.JobProfileDto.Dir;
+import org.apache.inlong.agent.plugin.fetcher.dtos.JobProfileDto.Job;
+import org.apache.inlong.agent.plugin.fetcher.dtos.JobProfileDto.Manager;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * check manager interface result with json formatter.
+ */
+public class ManagerResultFormatter {
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(ManagerResultFormatter.class);
+
+    private static final String RESULT_CODE = "success";
+    private static final String RESULT_DATA = "data";
+    public static final String SUCCESS_CODE = "true";
+
+    private static final Gson GSON = new Gson();
+    public static final String DEFAULT_TRIGGER = "org.apache.inlong.agent.plugin.trigger.DirectoryTrigger";
+    public static final String DEFAULT_CHANNEL = "org.apache.inlong.agent.plugin.channel.MemoryChannel";
+    public static final String TDM_JOB = "TDM_JOB";
+    public static final String DEFAULT_BUS_SINK = "org.apache.inlong.agent.plugin.sinks.TdBusSink";
+    public static final String DEFAULT_SOURCE = "org.apache.inlong.agent.plugin.sources.TextFileSource";
+
+    /**
+     * get json result
+     * @return json object
+     */
+    public static JsonObject getResultData(String jsonStr) {
+        JsonObject object = GSON.fromJson(jsonStr, JsonObject.class);
+        if (object == null || !object.has(RESULT_CODE)
+                || !object.has(RESULT_DATA)
+                || !SUCCESS_CODE.equals(object.get(RESULT_CODE).getAsString())) {
+            throw new IllegalArgumentException("cannot get result data,"
+                + " please check manager status, return str is" + jsonStr);
+
+        }
+        return object;
+    }
+
+    /**
+     * get random list of base list.
+     * @param baseList - base list
+     * @param num - max Num
+     * @return random list
+     */
+    public static <T> List<T> getRandomList(List<T> baseList, int num) {
+        if (baseList == null) {
+            return new ArrayList<>();
+        }
+        // make sure num cannot exceed size of base list
+        List<T> newHostList = new ArrayList<>(baseList);
+        Collections.shuffle(newHostList);
+        num = Math.min(num, baseList.size());
+        return newHostList.subList(0, num);
+    }
+
+
+
+    public static TriggerProfile convertToTriggerProfile(DataConfig dataConfigs) {
+        if (!dataConfigs.isValid()) {
+            throw new IllegalArgumentException("input dataConfig" + dataConfigs + "is invalid please check");
+        }
+        JobProfileDto profileDto = new JobProfileDto();
+        Proxy proxy = getProxy(dataConfigs);
+        Job job = getJob(dataConfigs);
+        profileDto.setProxy(proxy);
+        profileDto.setJob(job);
+        return TriggerProfile.parseJsonStr(GSON.toJson(profileDto));
+    }
+
+    private static Job getJob(DataConfig dataConfigs) {
+        Job job = new Job();
+        Dir dir = new Dir();
+        dir.setPattern(dataConfigs.getDataName());
+        job.setDir(dir);
+        job.setTrigger(DEFAULT_TRIGGER);
+        job.setChannel(DEFAULT_CHANNEL);
+        job.setName(TDM_JOB);
+        job.setSource(DEFAULT_SOURCE);
+        job.setSink(DEFAULT_BUS_SINK);
+        job.setId(dataConfigs.getTaskId());
+        job.setOp(dataConfigs.getOp());
+        job.setDeliveryTime(dataConfigs.getDeliveryTime());
+        if (!dataConfigs.getAddtictiveAttr().isEmpty()) {
+            job.setAddictiveString(dataConfigs.getAddtictiveAttr());
+        }
+        return job;
+    }
+
+    private static Proxy getProxy(DataConfig dataConfigs) {
+        Proxy proxy = new Proxy();
+        Manager manager = new Manager();
+        AgentConfiguration agentConf = AgentConfiguration.getAgentConf();
+        manager.setHost(agentConf.get(AGENT_MANAGER_VIP_HTTP_HOST));
+        manager.setPort(agentConf.get(AGENT_MANAGER_VIP_HTTP_PORT));
+        proxy.setBid(dataConfigs.getBusinessIdentifier());
+        proxy.setManager(manager);
+        return proxy;
+    }
+
+
+}
diff --git a/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/fetcher/constants/CommandConstants.java b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/fetcher/constants/CommandConstants.java
new file mode 100644
index 0000000..c21de95
--- /dev/null
+++ b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/fetcher/constants/CommandConstants.java
@@ -0,0 +1,57 @@
+/*
+ * 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.inlong.agent.plugin.fetcher.constants;
+
+/**
+ * old version of command result
+ */
+public class CommandConstants {
+    public static final String ID = "id";
+
+    public static final String IDC = "idc";
+    public static final String IP = "ip";
+    public static final String OP = "op";
+    public static final String OUTPUT_TYPE = "output_type";
+    public static final String DATA_NAME = "data_name";
+    public static final String CHECK_NAME = "check_name";
+    public static final String HAVING_CHECK = "havingcheck";
+    public static final String USING_VERIFY = "usingverify";
+    public static final String ADDICTIVE_ATTR = "addictive_attr";
+    public static final String TIME_OFFSET = "time_offset";
+    public static final String SCHEDULE_TIME = "schedule_time";
+    public static final String SPECIFIED_DATA_TIME = "specified_data_time";
+    public static final String TASK_TYPE = "task_type";
+    public static final String MAX_FILE_NUM = "max_file_num";
+    public static final String TOPIC = "topic";
+    public static final String SEQUENCIAL = "sequencial";
+    public static final String DELAY_MILL_SEC = "delay_mill_sec";
+    public static final String STOP_TIME_MILL_SEC = "stop_time_mill_sec";
+    public static final String SCAN_INTERVAL_SEC = "scan_interval_sec";
+    public static final String SCAN_DURA_SEC = "scan_dura_sec";
+    public static final String CYCLE_UNIT = "cycle_unit";
+    public static final String DRIVER_CLASS_NAME = "driver_class_name";
+    public static final String THREAD_CLASS_NAME = "thread_class_name";
+    public static final String FILTER_CLASS_NAME = "filter_class_name";
+    public static final String CLUSTER_ID = "clusterId";
+    public static final String BUSINESS_ID = "business_id";
+    public static final String INTERFACE_ID = "interface_id";
+    public static final String FIELD_SPLITTER = "field_splitter";
+    public static final String MD5 = "md5";
+
+
+}
diff --git a/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/fetcher/constants/FetcherConstants.java b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/fetcher/constants/FetcherConstants.java
new file mode 100644
index 0000000..d257faf
--- /dev/null
+++ b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/fetcher/constants/FetcherConstants.java
@@ -0,0 +1,54 @@
+/*
+ * 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.inlong.agent.plugin.fetcher.constants;
+
+public class FetcherConstants {
+    public static final String AGENT_FETCHER_INTERVAL = "agent.fetcher.interval";
+    public static final int DEFAULT_AGENT_FETCHER_INTERVAL = 60;
+
+    public static final String AGENT_MANAGER_REQUEST_TIMEOUT = "agent.manager.request.timeout";
+    // default is 30s
+    public static final int DEFAULT_AGENT_MANAGER_REQUEST_TIMEOUT = 30;
+
+    // required config
+    public static final String AGENT_MANAGER_VIP_HTTP_HOST = "agent.manager.vip.http.host";
+    public static final String AGENT_MANAGER_VIP_HTTP_PORT = "agent.manager.vip.http.port";
+
+    public static final String AGENT_MANAGER_VIP_HTTP_PATH = "agent.manager.vip.http.managerIp.path";
+    public static final String DEFAULT_AGENT_TDM_VIP_HTTP_PATH = "/getilmvirtualip";
+
+    public static final String AGENT_MANAGER_VIP_HTTP_PREFIX_PATH = "agent.manager.vip.http.prefix.path";
+    public static final String DEFAULT_AGENT_MANAGER_VIP_HTTP_PREFIX_PATH = "/openapi/inlong/manager/agent";
+
+    public static final String AGENT_MANAGER_TASK_HTTP_PATH = "agent.manager.vip.http.task.path";
+    public static final String DEFAULT_AGENT_MANAGER_TASK_HTTP_PATH = "/file_agent/taskconf";
+
+    public static final String AGENT_MANAGER_IP_CHECK_HTTP_PATH = "agent.manager.vip.http.checkIP.path";
+    public static final String DEFAULT_AGENT_TDM_IP_CHECK_HTTP_PATH = "/file_agent/confirmAgentIp";
+
+    public static final String AGENT_HTTP_APPLICATION_JSON = "application/json";
+
+    public static final int AGENT_HTTP_SUCCESS_CODE = 200;
+
+    public static final String DEFAULT_LOCAL_IP = "127.0.0.1";
+
+    public static final String AGENT_MANAGER_RETURN_PARAM_IP = "ip";
+    public static final String AGENT_MANAGER_RETURN_PARAM_DATA = "data";
+
+}
diff --git a/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/fetcher/dtos/CmdConfig.java b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/fetcher/dtos/CmdConfig.java
new file mode 100644
index 0000000..5b17474
--- /dev/null
+++ b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/fetcher/dtos/CmdConfig.java
@@ -0,0 +1,29 @@
+/*
+ * 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.inlong.agent.plugin.fetcher.dtos;
+
+import lombok.Data;
+
+@Data
+public class CmdConfig {
+    private String dataTime;
+    private Integer id;
+    private Integer op;
+    private Integer taskId;
+}
diff --git a/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/fetcher/dtos/CommandInfoDto.java b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/fetcher/dtos/CommandInfoDto.java
new file mode 100644
index 0000000..a1a9d9d
--- /dev/null
+++ b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/fetcher/dtos/CommandInfoDto.java
@@ -0,0 +1,31 @@
... 4933 lines suppressed ...