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 ...