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

[incubator-inlong] branch INLONG-613 updated: [INLONG-643] add dataproxy-sdk directory (#499)

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

gosonzhang 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 8ea3f59  [INLONG-643] add dataproxy-sdk directory (#499)
8ea3f59 is described below

commit 8ea3f59dd87f0122495c802e8440ecbf001e8430
Author: ziruipeng <zp...@connect.ust.hk>
AuthorDate: Mon Jul 5 20:20:41 2021 +0800

    [INLONG-643] add dataproxy-sdk directory (#499)
    
    Co-authored-by: stingpeng <st...@tencent.com>
---
 inlong-dataproxy-sdk/pom.xml                       | 116 +++
 .../apache/inlong/dataproxy/ConfigConstants.java   |  69 ++
 .../inlong/dataproxy/DefaultMessageSender.java     | 603 +++++++++++++
 .../org/apache/inlong/dataproxy/FileCallback.java  |  42 +
 .../org/apache/inlong/dataproxy/MessageSender.java | 146 ++++
 .../inlong/dataproxy/MessageSenderFactory.java     |  22 +
 .../apache/inlong/dataproxy/ProxyClientConfig.java | 434 ++++++++++
 .../inlong/dataproxy/SendMessageCallback.java      |  26 +
 .../org/apache/inlong/dataproxy/SendResult.java    |  31 +
 .../inlong/dataproxy/codec/EncodeObject.java       | 387 +++++++++
 .../apache/inlong/dataproxy/codec/ErrorCode.java   |  52 ++
 .../inlong/dataproxy/codec/ProtocolDecoder.java    | 117 +++
 .../inlong/dataproxy/codec/ProtocolEncoder.java    | 420 ++++++++++
 .../dataproxy/config/EncryptConfigEntry.java       | 145 ++++
 .../inlong/dataproxy/config/EncryptInfo.java       |  55 ++
 .../apache/inlong/dataproxy/config/HostInfo.java   |  67 ++
 .../inlong/dataproxy/config/ProxyConfigEntry.java  | 106 +++
 .../dataproxy/config/ProxyConfigManager.java       | 931 +++++++++++++++++++++
 .../org/apache/inlong/dataproxy/demo/Event.java    |  96 +++
 .../inlong/dataproxy/demo/MyFileCallBack.java      |  59 ++
 .../inlong/dataproxy/demo/MyMessageCallBack.java   |  60 ++
 .../inlong/dataproxy/demo/SendMsgThread.java       |  74 ++
 .../inlong/dataproxy/http/InternalHttpSender.java  | 278 ++++++
 .../inlong/dataproxy/metric/MessageRecord.java     |  77 ++
 .../dataproxy/metric/MetricTimeNumSummary.java     | 107 +++
 .../inlong/dataproxy/network/ClientHandler.java    |  93 ++
 .../apache/inlong/dataproxy/network/ClientMgr.java | 907 ++++++++++++++++++++
 .../dataproxy/network/ClientPipelineFactory.java   |  48 ++
 .../inlong/dataproxy/network/HttpMessage.java      |  75 ++
 .../inlong/dataproxy/network/HttpProxySender.java  | 230 +++++
 .../inlong/dataproxy/network/NettyClient.java      | 215 +++++
 .../dataproxy/network/ProxysdkException.java       |  39 +
 .../inlong/dataproxy/network/QueueObject.java      |  65 ++
 .../apache/inlong/dataproxy/network/Sender.java    | 694 +++++++++++++++
 .../inlong/dataproxy/network/SequentialID.java     |  47 ++
 .../dataproxy/network/SyncMessageCallable.java     |  71 ++
 .../inlong/dataproxy/network/TimeScanObject.java   |  59 ++
 .../org/apache/inlong/dataproxy/network/Utils.java | 188 +++++
 .../dataproxy/threads/IndexCollectThread.java      |  71 ++
 .../dataproxy/threads/ManagerFetcherThread.java    |  66 ++
 .../dataproxy/threads/MetricWorkerThread.java      | 320 +++++++
 .../dataproxy/threads/TimeoutScanThread.java       | 186 ++++
 .../inlong/dataproxy/utils/ConcurrentHashSet.java  |  45 +
 .../apache/inlong/dataproxy/utils/EncryptUtil.java | 478 +++++++++++
 .../inlong/dataproxy/utils/MapBackedSet.java       |  74 ++
 .../apache/inlong/dataproxy/utils/ProxyUtils.java  | 151 ++++
 .../dataproxy/utils/ServiceDiscoveryUtils.java     | 292 +++++++
 .../java/org/apache/inlong/dataproxy/AppTest.java  |  51 ++
 .../inlong/dataproxy/TestMetricWorkerThread.java   |  69 ++
 pom.xml                                            |   1 +
 50 files changed, 9055 insertions(+)

diff --git a/inlong-dataproxy-sdk/pom.xml b/inlong-dataproxy-sdk/pom.xml
new file mode 100644
index 0000000..72ae7c7
--- /dev/null
+++ b/inlong-dataproxy-sdk/pom.xml
@@ -0,0 +1,116 @@
+<!--
+  ~ 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
+  ~ <p>
+  ~ http://www.apache.org/licenses/LICENSE-2.0
+  ~ <p>
+  ~ 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="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+    <parent>
+        <groupId>org.apache.inlong</groupId>
+        <artifactId>inlong</artifactId>
+        <version>0.9.0-incubating-SNAPSHOT</version>
+    </parent>
+    <artifactId>inlong-dataproxy-sdk</artifactId>
+    <name>Apache InLong - dataproxy SDK</name>
+    <packaging>jar</packaging>
+    <build>
+        <plugins>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-compiler-plugin</artifactId>
+                <configuration>
+                    <source>1.8</source>
+                    <target>1.8</target>
+                </configuration>
+            </plugin>
+            <plugin>
+                <artifactId>maven-release-plugin</artifactId>
+                <version>2.5.3</version>
+                <configuration>
+                    <autoVersionSubmodules>true</autoVersionSubmodules>
+                    <tagNameFormat>v@{project.version}</tagNameFormat>
+                    <releaseProfiles>release</releaseProfiles>
+                    <arguments>-Dmaven.javadoc.skip=true -Dmaven.test.skipTests=true -Dmaven.test.skip=true
+                        -Dmaven.deploy.skip=true
+                    </arguments>
+                </configuration>
+            </plugin>
+        </plugins>
+    </build>
+
+    <properties>
+        <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
+        <plugin.compile.version>3.8.1</plugin.compile.version>
+    </properties>
+
+    <dependencies>
+        <dependency>
+            <groupId>junit</groupId>
+            <artifactId>junit</artifactId>
+            <version>4.10</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>commons-codec</groupId>
+            <artifactId>commons-codec</artifactId>
+            <version>1.10</version>
+        </dependency>
+        <dependency>
+            <groupId>io.netty</groupId>
+            <artifactId>netty</artifactId>
+            <version>3.8.0.Final</version>
+        </dependency>
+        <!--<dependency>
+            <groupId>log4j</groupId>
+            <artifactId>log4j</artifactId>
+            <version>1.2.16</version>
+            <type>jar</type>
+        </dependency>-->
+        <dependency>
+            <groupId>com.google.code.gson</groupId>
+            <artifactId>gson</artifactId>
+            <version>2.8.5</version>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.httpcomponents</groupId>
+            <artifactId>httpclient</artifactId>
+            <version>4.5.1</version>
+        </dependency>
+        <dependency>
+            <groupId>commons-io</groupId>
+            <artifactId>commons-io</artifactId>
+            <version>2.1</version>
+        </dependency>
+        <dependency>
+            <groupId>commons-lang</groupId>
+            <artifactId>commons-lang</artifactId>
+            <version>2.6</version>
+        </dependency>
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-log4j12</artifactId>
+            <version>1.6.1</version>
+        </dependency>
+        <dependency>
+            <groupId>org.xerial.snappy</groupId>
+            <artifactId>snappy-java</artifactId>
+            <version>1.1.0</version>
+        </dependency>
+
+    </dependencies>
+
+</project>
diff --git a/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/ConfigConstants.java b/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/ConfigConstants.java
new file mode 100644
index 0000000..55ff383
--- /dev/null
+++ b/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/ConfigConstants.java
@@ -0,0 +1,69 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.dataproxy;
+
+import java.util.concurrent.TimeUnit;
+
+public class ConfigConstants {
+    public static final String PROXY_SDK_VERSION = "1.2.11";
+
+    public static final int ALIVE_CONNECTIONS = 3;
+    public static final int MAX_TIMEOUT_CNT = 3;
+    public static final int LOAD_THRESHOLD = 0;
+    public static final int CYCLE = 30;
+
+    public static final int MSG_TYPE = 7;
+    public static final int COMPRESS_SIZE = 120;
+
+    /* Configure the thread pool size for sync message sending. */
+    public static final int SYNC_THREAD_POOL_SIZE = 5;
+    public static final int MAX_SYNC_THREAD_POOL_SIZE = 10;
+
+    /* Configure the in-memory callback size for asynchronously message sending. */
+    public static final int ASYNC_CALLBACK_SIZE = 50000;
+    public static final int MAX_ASYNC_CALLBACK_SIZE = 2000000;
+
+    /* Configure the proxy IP list refresh parameters. */
+    public static final int PROXY_UPDATE_INTERVAL_MINUTES = 5;
+
+    /* one hour interval */
+    public static final int PROXY_HTTP_UPDATE_INTERVAL_MINUTES = 60;
+
+    public static final int PROXY_UPDATE_MAX_RETRY = 10;
+
+    public static final int MAX_LINE_CNT = 30;
+
+    /* Default connection,connect timeout in milliseconds.*/
+    public static final long DEFAULT_CONNECT_TIMEOUT_MILLIS =
+            TimeUnit.MILLISECONDS.convert(40, TimeUnit.SECONDS);
+
+    public static final long DEFAULT_REQUEST_TIMEOUT_MILLIS =
+            TimeUnit.MILLISECONDS.convert(40, TimeUnit.SECONDS);
+
+    public static final long DEFAULT_SEND_BUFFER_SIZE = 16777216;
+    public static final long DEFAULT_RECEIVE_BUFFER_SIZE = 16777216;
+
+    public static final String RECEIVE_BUFFER_SIZE = "receiveBufferSize";
+    public static final String SEND_BUFFER_SIZE = "sendBufferSize";
+
+    public static final String REQUEST_HEADER_AUTHORIZATION = "Authorization";
+    public static final int FLAG_ALLOW_AUTH = 1 << 7;
+    public static final int FLAG_ALLOW_ENCRYPT = 1 << 6;
+    public static final int FLAG_ALLOW_COMPRESS = 1 << 5;
+
+}
diff --git a/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/DefaultMessageSender.java b/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/DefaultMessageSender.java
new file mode 100644
index 0000000..5859945
--- /dev/null
+++ b/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/DefaultMessageSender.java
@@ -0,0 +1,603 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.dataproxy;
+
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.inlong.dataproxy.codec.EncodeObject;
+import org.apache.inlong.dataproxy.config.ProxyConfigEntry;
+import org.apache.inlong.dataproxy.config.ProxyConfigManager;
+import org.apache.inlong.dataproxy.network.ProxysdkException;
+import org.apache.inlong.dataproxy.network.Sender;
+import org.apache.inlong.dataproxy.network.SequentialID;
+import org.apache.inlong.dataproxy.network.Utils;
+import org.apache.inlong.dataproxy.threads.IndexCollectThread;
+import org.apache.inlong.dataproxy.threads.ManagerFetcherThread;
+import org.apache.inlong.dataproxy.utils.ProxyUtils;
+import org.jboss.netty.channel.ChannelFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class DefaultMessageSender implements MessageSender {
+    private static final Logger logger = LoggerFactory.getLogger(DefaultMessageSender.class);
+    private final Sender sender;
+    private final SequentialID idGenerator;
+    private String bid;
+    private int msgtype = ConfigConstants.MSG_TYPE;
+    private boolean isCompress = true;
+    private boolean isBidTransfer = false;
+    private boolean isReport = false;
+    private boolean isSupportLF = false;
+    private int cpsSize = ConfigConstants.COMPRESS_SIZE;
+
+
+    private static final ConcurrentHashMap<String, DefaultMessageSender> cacheSender =
+            new ConcurrentHashMap<>();
+
+    private final IndexCollectThread indexCol;
+    /* Store index <bid_tid,cnt>*/
+    private final Map<String, Long> storeIndex = new ConcurrentHashMap<String, Long>();
+
+    private static final AtomicBoolean ManagerFetcherThreadStarted = new AtomicBoolean(false);
+
+    private static ManagerFetcherThread managerFetcherThread;
+
+    public boolean isSupportLF() {
+        return isSupportLF;
+    }
+
+    public void setSupportLF(boolean supportLF) {
+        isSupportLF = supportLF;
+    }
+
+    public boolean isBidTransfer() {
+        return isBidTransfer;
+    }
+
+    public void setBidTransfer(boolean isBidTransfer) {
+        this.isBidTransfer = isBidTransfer;
+    }
+
+    public boolean isReport() {
+        return isReport;
+    }
+
+    public void setReport(boolean isReport) {
+        this.isReport = isReport;
+    }
+
+    public int getCpsSize() {
+        return cpsSize;
+    }
+
+    public void setCpsSize(int cpsSize) {
+        this.cpsSize = cpsSize;
+    }
+
+    public int getMsgtype() {
+        return msgtype;
+    }
+
+    public void setMsgtype(int msgtype) {
+        this.msgtype = msgtype;
+    }
+
+    public boolean isCompress() {
+        return isCompress;
+    }
+
+    public void setCompress(boolean isCompress) {
+        this.isCompress = isCompress;
+    }
+
+    public String getBid() {
+        return bid;
+    }
+
+    public void setBid(String bid) {
+        this.bid = bid;
+    }
+
+    public DefaultMessageSender(ProxyClientConfig configure) throws Exception {
+        this(configure, null);
+    }
+
+    public DefaultMessageSender(ProxyClientConfig configure, ChannelFactory selfDefineFactory) throws Exception {
+        ProxyUtils.validClientConfig(configure);
+        sender = new Sender(configure, selfDefineFactory);
+        idGenerator = new SequentialID(Utils.getLocalIp());
+        bid = configure.getBid();
+        indexCol = new IndexCollectThread(storeIndex);
+        indexCol.start();
+
+        if (configure.isEnableSaveManagerVIps()
+                && configure.isLocalVisit()
+
+                && ManagerFetcherThreadStarted.compareAndSet(false, true)) {
+            managerFetcherThread = new ManagerFetcherThread(configure);
+            managerFetcherThread.start();
+        }
+    }
+
+    /**
+     * generate by cluster id
+     *
+     * @param configure - sender
+     * @return - sender
+     */
+    public static DefaultMessageSender generateSenderByClusterId(
+            ProxyClientConfig configure) throws Exception {
+
+        return generateSenderByClusterId(configure, null);
+    }
+
+    /**
+     *  generate by cluster id
+     *
+     * @param configure         - sender
+     * @param selfDefineFactory - sender factory
+     * @return - sender
+     */
+    public static DefaultMessageSender generateSenderByClusterId(ProxyClientConfig configure,
+                                                                 ChannelFactory selfDefineFactory) throws Exception {
+        ProxyConfigManager proxyConfigManager = new ProxyConfigManager(configure,
+                Utils.getLocalIp(), null);
+        proxyConfigManager.setBusinessId(configure.getBid());
+        ProxyConfigEntry entry = proxyConfigManager.getBidConfigure();
+        DefaultMessageSender sender = cacheSender.get(entry.getClusterId());
+        if (sender != null) {
+            return sender;
+        } else {
+            DefaultMessageSender tmpMessageSender =
+                    new DefaultMessageSender(configure, selfDefineFactory);
+            cacheSender.put(entry.getClusterId(), tmpMessageSender);
+            return tmpMessageSender;
+        }
+    }
+
+    /**
+     * finally clean up
+     */
+    public static void finallyCleanup() {
+        for (DefaultMessageSender sender : cacheSender.values()) {
+            sender.close();
+        }
+        cacheSender.clear();
+    }
+
+    public String getSDKVersion() {
+        return ConfigConstants.PROXY_SDK_VERSION;
+    }
+
+
+    @Deprecated
+    public SendResult sendMessage(byte[] body, String attributes, String msgUUID,
+                                  long timeout, TimeUnit timeUnit) {
+        return sender.syncSendMessage(new EncodeObject(body, attributes,
+                idGenerator.getNextId()), msgUUID, timeout, timeUnit);
+    }
+
+
+    public SendResult sendMessage(byte[] body, String bid, String tid, long dt, String msgUUID,
+                                  long timeout, TimeUnit timeUnit) {
+        dt = ProxyUtils.covertZeroDt(dt);
+        if (!ProxyUtils.isBodyValid(body) || !ProxyUtils.isDtValid(dt)) {
+            return SendResult.INVALID_ATTRIBUTES;
+        }
+        addIndexCnt(bid, tid, 1);
+
+        boolean isCompressEnd = (isCompress && (body.length > cpsSize));
+
+        if (msgtype == 7 || msgtype == 8) {
+            EncodeObject encodeObject = new EncodeObject(body, msgtype, isCompressEnd, isReport,
+                    isBidTransfer, dt / 1000, idGenerator.getNextInt(), bid, tid, "");
+            encodeObject.setSupportLF(isSupportLF);
+            return sender.syncSendMessage(encodeObject, msgUUID, timeout, timeUnit);
+        } else if (msgtype == 3 || msgtype == 5) {
+            if (isCompressEnd) {
+                return sender.syncSendMessage(new EncodeObject(body, "bid=" + bid
+                        + "&tid=" + tid + "&dt=" + dt + "&cp=snappy",
+                        idGenerator.getNextId(), this.getMsgtype(), true, bid), msgUUID, timeout, timeUnit);
+            } else {
+                return sender.syncSendMessage(new EncodeObject(body, "bid=" + bid + "&tid=" + tid + "&dt=" + dt,
+                        idGenerator.getNextId(), this.getMsgtype(), false, bid), msgUUID, timeout, timeUnit);
+            }
+        }
+
+        return null;
+    }
+
+
+
+    public SendResult sendMessage(byte[] body, String bid, String tid, long dt, String msgUUID,
+                                  long timeout, TimeUnit timeUnit, Map<String, String> extraAttrMap) {
+
+        dt = ProxyUtils.covertZeroDt(dt);
+        if (!ProxyUtils.isBodyValid(body) || !ProxyUtils.isDtValid(dt) || !ProxyUtils.isAttrKeysValid(extraAttrMap)) {
+            return SendResult.INVALID_ATTRIBUTES;
+        }
+        addIndexCnt(bid, tid, 1);
+
+        StringBuilder attrs = ProxyUtils.convertAttrToStr(extraAttrMap);
+
+        boolean isCompressEnd = (isCompress && (body.length > cpsSize));
+
+        if (msgtype == 7 || msgtype == 8) {
+            EncodeObject encodeObject = new EncodeObject(body, msgtype, isCompressEnd, isReport,
+                    isBidTransfer, dt / 1000,
+                    idGenerator.getNextInt(), bid, tid, attrs.toString());
+            encodeObject.setSupportLF(isSupportLF);
+            return sender.syncSendMessage(encodeObject, msgUUID, timeout, timeUnit);
+        } else if (msgtype == 3 || msgtype == 5) {
+            attrs.append("&bid=").append(bid).append("&tid=").append(tid).append("&dt=").append(dt);
+            if (isCompressEnd) {
+                attrs.append("&cp=snappy");
+                return sender.syncSendMessage(new EncodeObject(body, attrs.toString(),
+                        idGenerator.getNextId(), this.getMsgtype(), true, bid), msgUUID, timeout, timeUnit);
+            } else {
+                return sender.syncSendMessage(new EncodeObject(body, attrs.toString(),
+                        idGenerator.getNextId(), this.getMsgtype(), false, bid), msgUUID, timeout, timeUnit);
+            }
+        }
+        return null;
+
+
+    }
+
+
+    public SendResult sendMessage(List<byte[]> bodyList, String bid, String tid, long dt, String msgUUID,
+                                  long timeout, TimeUnit timeUnit) {
+        dt = ProxyUtils.covertZeroDt(dt);
+        if (!ProxyUtils.isBodyValid(bodyList) || !ProxyUtils.isDtValid(dt)) {
+            return SendResult.INVALID_ATTRIBUTES;
+        }
+        addIndexCnt(bid, tid, bodyList.size());
+
+        if (msgtype == 7 || msgtype == 8) {
+            EncodeObject encodeObject = new EncodeObject(bodyList, msgtype, isCompress, isReport,
+                    isBidTransfer, dt / 1000,
+                    idGenerator.getNextInt(), bid, tid, "");
+            encodeObject.setSupportLF(isSupportLF);
+            return sender.syncSendMessage(encodeObject, msgUUID, timeout, timeUnit);
+        } else if (msgtype == 3 || msgtype == 5) {
+            if (isCompress) {
+                return sender.syncSendMessage(new EncodeObject(bodyList, "bid=" + bid + "&tid=" + tid
+                        + "&dt=" + dt + "&cp=snappy" + "&cnt=" + bodyList.size(),
+                        idGenerator.getNextId(), this.getMsgtype(), true, bid), msgUUID, timeout, timeUnit);
+            } else {
+                return sender.syncSendMessage(new EncodeObject(bodyList, "bid=" + bid + "&tid=" + tid
+                        + "&dt=" + dt + "&cnt=" + bodyList.size(),
+                        idGenerator.getNextId(), this.getMsgtype(), false, bid), msgUUID, timeout, timeUnit);
+            }
+        }
+        return null;
+    }
+
+
+    public SendResult sendMessage(List<byte[]> bodyList, String bid, String tid, long dt, String msgUUID,
+                                  long timeout, TimeUnit timeUnit, Map<String, String> extraAttrMap) {
+        dt = ProxyUtils.covertZeroDt(dt);
+        if (!ProxyUtils.isBodyValid(bodyList) || !ProxyUtils.isDtValid(dt)
+            || !ProxyUtils.isAttrKeysValid(extraAttrMap)) {
+            return SendResult.INVALID_ATTRIBUTES;
+        }
+        addIndexCnt(bid, tid, bodyList.size());
+        StringBuilder attrs = ProxyUtils.convertAttrToStr(extraAttrMap);
+
+        if (msgtype == 7 || msgtype == 8) {
+            EncodeObject encodeObject = new EncodeObject(bodyList, msgtype, isCompress, isReport,
+                    isBidTransfer, dt / 1000,
+                    idGenerator.getNextInt(), bid, tid, attrs.toString());
+            encodeObject.setSupportLF(isSupportLF);
+            return sender.syncSendMessage(encodeObject, msgUUID, timeout, timeUnit);
+        } else if (msgtype == 3 || msgtype == 5) {
+            attrs.append("&bid=").append(bid).append("&tid=").append(tid)
+                    .append("&dt=").append(dt).append("&cnt=").append(bodyList.size());
+            if (isCompress) {
+                attrs.append("&cp=snappy");
+                return sender.syncSendMessage(new EncodeObject(bodyList, attrs.toString(),
+                        idGenerator.getNextId(), this.getMsgtype(), true, bid), msgUUID, timeout, timeUnit);
+            } else {
+                return sender.syncSendMessage(new EncodeObject(bodyList, attrs.toString(),
+                        idGenerator.getNextId(), this.getMsgtype(), false, bid), msgUUID, timeout, timeUnit);
+            }
+        }
+        return null;
+    }
+
+
+
+    @Deprecated
+    public void asyncSendMessage(SendMessageCallback callback, byte[] body, String attributes, String msgUUID,
+                                 long timeout, TimeUnit timeUnit) throws ProxysdkException {
+        sender.asyncSendMessage(new EncodeObject(body, attributes, idGenerator.getNextId()),
+                callback, msgUUID, timeout, timeUnit);
+    }
+
+
+    public void asyncSendMessage(SendMessageCallback callback, byte[] body,
+                                 String bid, String tid, long dt, String msgUUID,
+                                 long timeout, TimeUnit timeUnit) throws ProxysdkException {
+        dt = ProxyUtils.covertZeroDt(dt);
+        if (!ProxyUtils.isBodyValid(body) || !ProxyUtils.isDtValid(dt)) {
+            throw new ProxysdkException(SendResult.INVALID_ATTRIBUTES.toString());
+        }
+        addIndexCnt(bid, tid, 1);
+
+        boolean isCompressEnd = (isCompress && (body.length > cpsSize));
+        if (msgtype == 7 || msgtype == 8) {
+            EncodeObject encodeObject = new EncodeObject(body, this.getMsgtype(), isCompressEnd, isReport,
+                    isBidTransfer, dt / 1000, idGenerator.getNextInt(),
+                    bid, tid, "");
+            encodeObject.setSupportLF(isSupportLF);
+            sender.asyncSendMessage(encodeObject, callback, msgUUID, timeout, timeUnit);
+        } else if (msgtype == 3 || msgtype == 5) {
+            if (isCompressEnd) {
+                sender.asyncSendMessage(new EncodeObject(body, "bid="
+                        + bid + "&tid=" + tid + "&dt=" + dt + "&cp=snappy",
+                        idGenerator.getNextId(), this.getMsgtype(), true, bid), callback, msgUUID, timeout, timeUnit);
+            } else {
+                sender.asyncSendMessage(
+                        new EncodeObject(body, "bid=" + bid + "&tid=" + tid + "&dt=" + dt, idGenerator.getNextId(),
+                                this.getMsgtype(), false, bid), callback, msgUUID, timeout, timeUnit);
+            }
+        }
+
+    }
+
+
+    public void asyncSendMessage(SendMessageCallback callback,
+                                 byte[] body, String bid, String tid, long dt, String msgUUID,
+                                 long timeout, TimeUnit timeUnit,
+                                 Map<String, String> extraAttrMap) throws ProxysdkException {
+        dt = ProxyUtils.covertZeroDt(dt);
+        if (!ProxyUtils.isBodyValid(body) || !ProxyUtils.isDtValid(dt) || !ProxyUtils.isAttrKeysValid(extraAttrMap)) {
+            throw new ProxysdkException(SendResult.INVALID_ATTRIBUTES.toString());
+        }
+        addIndexCnt(bid, tid, 1);
+        StringBuilder attrs = ProxyUtils.convertAttrToStr(extraAttrMap);
+
+        boolean isCompressEnd = (isCompress && (body.length > cpsSize));
+        if (msgtype == 7 || msgtype == 8) {
+            EncodeObject encodeObject = new EncodeObject(body, this.getMsgtype(), isCompressEnd,
+                    isReport, isBidTransfer, dt / 1000, idGenerator.getNextInt(),
+                    bid, tid, attrs.toString());
+            encodeObject.setSupportLF(isSupportLF);
+            sender.asyncSendMessage(encodeObject, callback, msgUUID, timeout, timeUnit);
+        } else if (msgtype == 3 || msgtype == 5) {
+            attrs.append("&bid=").append(bid).append("&tid=").append(tid).append("&dt=").append(dt);
+            if (isCompressEnd) {
+                attrs.append("&cp=snappy");
+                sender.asyncSendMessage(new EncodeObject(body, attrs.toString(),
+                                idGenerator.getNextId(), this.getMsgtype(), true, bid),
+                        callback, msgUUID, timeout, timeUnit);
+            } else {
+                sender.asyncSendMessage(new EncodeObject(body, attrs.toString(), idGenerator.getNextId(),
+                                this.getMsgtype(), false, bid),
+                        callback, msgUUID, timeout, timeUnit);
+            }
+        }
+    }
+
+
+    public void asyncSendMessage(SendMessageCallback callback, List<byte[]> bodyList,
+                                 String bid, String tid, long dt, String msgUUID,
+                                 long timeout, TimeUnit timeUnit) throws ProxysdkException {
+        dt = ProxyUtils.covertZeroDt(dt);
+        if (!ProxyUtils.isBodyValid(bodyList) || !ProxyUtils.isDtValid(dt)) {
+            throw new ProxysdkException(SendResult.INVALID_ATTRIBUTES.toString());
+        }
+        addIndexCnt(bid, tid, bodyList.size());
+        if (msgtype == 7 || msgtype == 8) {
+            EncodeObject encodeObject = new EncodeObject(bodyList, this.getMsgtype(), isCompress,
+                    isReport, isBidTransfer, dt / 1000, idGenerator.getNextInt(),
+                    bid, tid, "");
+            encodeObject.setSupportLF(isSupportLF);
+            sender.asyncSendMessage(encodeObject, callback, msgUUID, timeout, timeUnit);
+        } else if (msgtype == 3 || msgtype == 5) {
+            if (isCompress) {
+                sender.asyncSendMessage(
+                        new EncodeObject(bodyList, "bid=" + bid + "&tid=" + tid
+                                + "&dt=" + dt + "&cp=snappy" + "&cnt=" + bodyList.size(),
+                                idGenerator.getNextId(), this.getMsgtype(),
+                                true, bid), callback, msgUUID, timeout, timeUnit);
+            } else {
+                sender.asyncSendMessage(
+                        new EncodeObject(bodyList, "bid=" + bid + "&tid="
+                                + tid + "&dt=" + dt + "&cnt=" + bodyList.size(),
+                                idGenerator.getNextId(), this.getMsgtype(),
+                                false, bid), callback, msgUUID, timeout, timeUnit);
+            }
+        }
+    }
+
+
+    public void asyncSendMessage(SendMessageCallback callback,
+                                 List<byte[]> bodyList, String bid, String tid, long dt, String msgUUID,
+                                 long timeout, TimeUnit timeUnit,
+                                 Map<String, String> extraAttrMap) throws ProxysdkException {
+        dt = ProxyUtils.covertZeroDt(dt);
+        if (!ProxyUtils.isBodyValid(bodyList) || !ProxyUtils.isDtValid(dt)
+            || !ProxyUtils.isAttrKeysValid(extraAttrMap)) {
+            throw new ProxysdkException(SendResult.INVALID_ATTRIBUTES.toString());
+        }
+        addIndexCnt(bid, tid, bodyList.size());
+        StringBuilder attrs = ProxyUtils.convertAttrToStr(extraAttrMap);
+
+        if (msgtype == 7 || msgtype == 8) {
+//            if (!isBidTransfer)
+            EncodeObject encodeObject = new EncodeObject(bodyList, this.getMsgtype(),
+                    isCompress, isReport, isBidTransfer, dt / 1000, idGenerator.getNextInt(),
+                    bid, tid, attrs.toString());
+            encodeObject.setSupportLF(isSupportLF);
+            sender.asyncSendMessage(encodeObject, callback, msgUUID, timeout, timeUnit);
+        } else if (msgtype == 3 || msgtype == 5) {
+            attrs.append("&bid=").append(bid).append("&tid=").append(tid)
+                    .append("&dt=").append(dt).append("&cnt=").append(bodyList.size());
+            if (isCompress) {
+                attrs.append("&cp=snappy");
+                sender.asyncSendMessage(new EncodeObject(bodyList, attrs.toString(), idGenerator.getNextId(),
+                        this.getMsgtype(), true, bid), callback, msgUUID, timeout, timeUnit);
+            } else {
+                sender.asyncSendMessage(new EncodeObject(bodyList, attrs.toString(), idGenerator.getNextId(),
+                        this.getMsgtype(), false, bid), callback, msgUUID, timeout, timeUnit);
+            }
+        }
+
+    }
+
+    private void addIndexCnt(String bid, String tid, long cnt) {
+        try {
+            String key = bid + "|" + tid;
+            if (storeIndex.containsKey(key)) {
+                long sum = storeIndex.get(key);
+                storeIndex.put(key, sum + cnt);
+            } else {
+                storeIndex.put(key, cnt);
+            }
+        } catch (Exception e) {
+            logger.error(e.getMessage());
+        }
+    }
+
+
+
+    public void asyncsendMessageData(FileCallback callback, List<byte[]> bodyList, String bid,
+                                     String tid, long dt, int sid, boolean isSupportLF, String msgUUID,
+                                     long timeout, TimeUnit timeUnit,
+                                     Map<String, String> extraAttrMap) throws ProxysdkException {
+        dt = ProxyUtils.covertZeroDt(dt);
+        if (!ProxyUtils.isBodyValid(bodyList) || !ProxyUtils.isDtValid(dt)
+            || !ProxyUtils.isAttrKeysValid(extraAttrMap)) {
+            throw new ProxysdkException(SendResult.INVALID_ATTRIBUTES.toString());
+        }
+        addIndexCnt(bid, tid, bodyList.size());
+
+        StringBuilder attrs = ProxyUtils.convertAttrToStr(extraAttrMap);
+
+        if (msgtype == 7 || msgtype == 8) {
+            EncodeObject encodeObject = new EncodeObject(bodyList, msgtype,
+                    isCompress, isReport, isBidTransfer,
+                    dt / 1000, sid, bid, tid, attrs.toString(), "data", "");
+            encodeObject.setSupportLF(isSupportLF);
+            sender.asyncSendMessageIndex(encodeObject, callback, msgUUID, timeout, timeUnit);
+        }
+    }
+
+    private void asyncSendMetric(FileCallback callback, byte[] body, String bid,
+                                 String tid, long dt, int sid, String ip, String msgUUID,
+                                 long timeout, TimeUnit timeUnit, String messageKey) throws ProxysdkException {
+        dt = ProxyUtils.covertZeroDt(dt);
+        if (!ProxyUtils.isBodyValid(body) || !ProxyUtils.isDtValid(dt)) {
+            throw new ProxysdkException(SendResult.INVALID_ATTRIBUTES.toString());
+        }
+        boolean isCompressEnd = false;
+        if (msgtype == 7 || msgtype == 8) {
+            sender.asyncSendMessageIndex(new EncodeObject(body, msgtype, isCompressEnd,
+                    isReport, isBidTransfer, dt / 1000,
+                    sid, bid, tid, "", messageKey, ip), callback, msgUUID, timeout, timeUnit);
+        }
+    }
+
+
+
+
+    public void asyncsendMessageProxy(FileCallback callback, byte[] body, String bid, String tid,
+                                    long dt, int sid, String ip, String msgUUID,
+                                    long timeout, TimeUnit timeUnit) throws ProxysdkException {
+        asyncSendMetric(callback, body, bid, tid, dt, sid, ip, msgUUID, timeout, timeUnit, "minute");
+    }
+
+
+    public void asyncsendMessageFile(FileCallback callback, byte[] body, String bid,
+                                     String tid, long dt, int sid, String msgUUID,
+                                     long timeout, TimeUnit timeUnit) throws ProxysdkException {
+        asyncSendMetric(callback, body, bid, tid, dt, sid, "", msgUUID, timeout, timeUnit, "file");
+    }
+
+
+
+    public String sendMessageData(List<byte[]> bodyList, String bid,
+                                  String tid, long dt, int sid, boolean isSupportLF, String msgUUID,
+                                  long timeout, TimeUnit timeUnit, Map<String, String> extraAttrMap) {
+        dt = ProxyUtils.covertZeroDt(dt);
+        if (!ProxyUtils.isBodyValid(bodyList) || !ProxyUtils.isDtValid(dt)
+            || !ProxyUtils.isAttrKeysValid(extraAttrMap)) {
+            return SendResult.INVALID_ATTRIBUTES.toString();
+        }
+        addIndexCnt(bid, tid, bodyList.size());
+
+        StringBuilder attrs = ProxyUtils.convertAttrToStr(extraAttrMap);
+
+        if (msgtype == 7 || msgtype == 8) {
+            EncodeObject encodeObject = new EncodeObject(bodyList, msgtype, isCompress,
+                    isReport, isBidTransfer, dt / 1000,
+                    sid, bid, tid, attrs.toString(), "data", "");
+            encodeObject.setSupportLF(isSupportLF);
+            return sender.syncSendMessageIndex(encodeObject, msgUUID, timeout, timeUnit);
+        }
+        return null;
+    }
+
+    private String sendMetric(byte[] body, String bid, String tid, long dt, int sid, String ip, String msgUUID,
+                              long timeout, TimeUnit timeUnit, String messageKey) {
+        dt = ProxyUtils.covertZeroDt(dt);
+        if (!ProxyUtils.isBodyValid(body) || !ProxyUtils.isDtValid(dt)) {
+            return SendResult.INVALID_ATTRIBUTES.toString();
+        }
+        if (msgtype == 7 || msgtype == 8) {
+            EncodeObject encodeObject = new EncodeObject(body, msgtype, false, isReport,
+                    isBidTransfer, dt / 1000, sid, bid, tid, "", messageKey, ip);
+            return sender.syncSendMessageIndex(encodeObject, msgUUID, timeout, timeUnit);
+        }
+        return null;
+    }
+
+
+
+
+    public String sendMessageProxy(byte[] body, String bid, String tid, long dt, int sid, String ip, String msgUUID,
+                                 long timeout, TimeUnit timeUnit) {
+        return sendMetric(body, bid, tid, dt, sid, ip, msgUUID, timeout, timeUnit, "minute");
+    }
+
+
+    public String sendMessageFile(byte[] body, String bid, String tid, long dt, int sid, String msgUUID,
+                                  long timeout, TimeUnit timeUnit) {
+        return sendMetric(body, bid, tid, dt, sid, "", msgUUID, timeout, timeUnit, "file");
+    }
+
+    private void shutdownInternalThreads() {
+        indexCol.shutDown();
+        managerFetcherThread.shutdown();
+        ManagerFetcherThreadStarted.set(false);
+    }
+
+    public void close() {
+        logger.info("ready to close resources, may need five minutes !");
+        if (sender.getClusterId() != null) {
+            cacheSender.remove(sender.getClusterId());
+        }
+        sender.close();
+        shutdownInternalThreads();
+    }
+}
diff --git a/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/FileCallback.java b/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/FileCallback.java
new file mode 100644
index 0000000..d999dc4
--- /dev/null
+++ b/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/FileCallback.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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.dataproxy;
+
+/**
+ * Created by jesseyzhou on 2018/6/5.
+ */
+
+
+public abstract class FileCallback implements SendMessageCallback {
+    /* Invoked when a message is confirmed by TDBus. */
+    public void onMessageAck(String result) {
+    }
+
+    ;
+
+    public void onMessageAck(SendResult result) {
+    }
+
+    ;
+
+    /* Invoked when a message transportation interrupted by an exception. */
+    public void onException(Throwable e) {
+    }
+
+    ;
+}
diff --git a/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/MessageSender.java b/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/MessageSender.java
new file mode 100644
index 0000000..1d44718
--- /dev/null
+++ b/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/MessageSender.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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.dataproxy;
+
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.inlong.dataproxy.network.ProxysdkException;
+
+public interface MessageSender {
+
+    /**
+     * This method provides a synchronized function which you want to send data
+     * with extra attributes except  bid,tid,dt,etc
+     * This method is deprecated,we suggest you don't use it.
+     *
+     * @param body       The data will be sent
+     * @param attributes The attributes you want to add
+     */
+    @Deprecated
+    public SendResult sendMessage(byte[] body, String attributes, String msgUUID,
+                                  long timeout, TimeUnit timeUnit);
+
+    /**
+     * This method provides a synchronized  function which you want to send data  without packing
+     *
+     * @param body The data will be sent
+     *             
+     */
+    public SendResult sendMessage(byte[] body, String bid, String tid, long dt, String msgUUID,
+                                  long timeout, TimeUnit timeUnit);
+
+    /**
+     * This method provides a synchronized  function which you want to send data without packing
+     * with extra attributes except  bid,tid,dt,etc
+     *
+     * @param body         The data will be sent
+     *                     
+     * @param extraAttrMap The attributes you want to add,
+     *                     and each element of extraAttrMap contains a pair like <attrKey,attrValue>
+     */
+    public SendResult sendMessage(byte[] body, String bid, String tid, long dt, String msgUUID,
+                                  long timeout, TimeUnit timeUnit, Map<String, String> extraAttrMap);
+
+    /**
+     * This method provides a synchronized  function which you want to send data  with packing
+     * 
+     *
+     * @param bodyList The data will be sent,which is a collection consisting of byte arrays
+     */
+    public SendResult sendMessage(List<byte[]> bodyList, String bid, String tid, long dt, String msgUUID,
+                                  long timeout, TimeUnit timeUnit);
+
+    /**
+     * This method provides a synchronized  function which you want to send data with packing
+     * with extra attributes except  bid,tid,dt,etc
+     * 
+     *
+     * @param bodyList     The data will be sent,which is a collection consisting of byte arrays
+     * @param extraAttrMap The attributes you want to add,
+     *                     and each element of extraAttrMap contains a pair like <attrKey,attrValue>
+     */
+    public SendResult sendMessage(List<byte[]> bodyList, String bid, String tid, long dt, String msgUUID,
+                                  long timeout, TimeUnit timeUnit, Map<String, String> extraAttrMap);
+
+    /**
+     * This method provides an asynchronized  function which you want to send data
+     * with extra attributes except  bid,tid,dt,etc
+     * This method is deprecated,we suggest you don't use it.
+     * 
+     *
+     * @param body       The data will be sent
+     * @param attributes The attributes you want to add
+     */
+    @Deprecated
+    public void asyncSendMessage(SendMessageCallback callback,
+                                 byte[] body, String attributes, String msgUUID,
+                                 long timeout, TimeUnit timeUnit) throws ProxysdkException;
+
+    /**
+     * This method provides a synchronized  function which you want to send data without packing
+     * with extra attributes except  bid,tid,dt,etc
+     * 
+     *
+     * @param body         The data will be sent
+     * @param extraAttrMap The attributes you want to add,
+     *                     and each element of extraAttrMap contains a pair like <attrKey,attrValue>
+     */
+    public void asyncSendMessage(SendMessageCallback callback,
+                                 byte[] body, String bid, String tid, long dt, String msgUUID,
+                                 long timeout, TimeUnit timeUnit,
+                                 Map<String, String> extraAttrMap) throws ProxysdkException;
+
+    /**
+     * This method provides an asynchronized  function which you want to send data  without packing
+     * 
+     *
+     * @param callback The implementation of callback function
+     * @param body     The data will be sent
+     */
+    public void asyncSendMessage(SendMessageCallback callback,
+                                 byte[] body, String bid, String tid, long dt, String msgUUID,
+                                 long timeout, TimeUnit timeUnit) throws ProxysdkException;
+
+    /**
+     * This method provides an asynchronized  function which you want to send data  with packing
+     * 
+     *
+     * @param bodyList The data will be sent,which is a collection consisting of byte arrays
+     */
+    public void asyncSendMessage(SendMessageCallback callback,
+                                 List<byte[]> bodyList, String bid, String tid, long dt, String msgUUID,
+                                 long timeout, TimeUnit timeUnit) throws ProxysdkException;
+
+    /**
+     * This method provides an asynchronized  function which you want to send data with packing
+     * with extra attributes except  bid,tid,dt,etc
+     * 
+     *
+     * @param bodyList     The data will be sent,which is a collection consisting of byte arrays
+     * @param extraAttrMap The attributes you want to add, and each
+     *                     element of extraAttrMap contains a pair like <attrKey,attrValue>
+     */
+    public void asyncSendMessage(SendMessageCallback callback,
+                                 List<byte[]> bodyList, String bid, String tid, long dt, String msgUUID,
+                                 long timeout, TimeUnit timeUnit,
+                                 Map<String, String> extraAttrMap) throws ProxysdkException;
+
+    public void close();
+}
diff --git a/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/MessageSenderFactory.java b/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/MessageSenderFactory.java
new file mode 100644
index 0000000..29352ab
--- /dev/null
+++ b/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/MessageSenderFactory.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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.dataproxy;
+
+public class MessageSenderFactory {
+
+}
diff --git a/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/ProxyClientConfig.java b/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/ProxyClientConfig.java
new file mode 100644
index 0000000..392665e
--- /dev/null
+++ b/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/ProxyClientConfig.java
@@ -0,0 +1,434 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.dataproxy;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.inlong.dataproxy.network.ProxysdkException;
+import org.apache.inlong.dataproxy.network.Utils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ProxyClientConfig {
+    private static final Logger logger = LoggerFactory.getLogger(ProxyClientConfig.class);
+    private int aliveConnections;
+    private int syncThreadPoolSize;
+    private int asyncCallbackSize;
+    private int managerPort = 8099;
+    private String managerIP = "";
+
+    private String managerIpLocalPath = System.getProperty("user.dir") + "/.inlong/.managerIps";
+    private String proxyIPServiceURL = "";
+    private int proxyUpdateIntervalMinutes;
+    private int proxyUpdateMaxRetry;
+    private String netTag;
+    private String bid;
+    private boolean isFile = false;
+    private boolean isLocalVisit = true;
+    private boolean isNeedDataEncry = false;
+    private boolean needAuthentication = false;
+    private String userName = "";
+    private String secretKey = "";
+    private String rsaPubKeyUrl = "";
+    private String confStoreBasePath = ".inlong/";
+    private boolean needVerServer = false;
+    private String tlsServerCertFilePathAndName;
+    private String tlsServerKey;
+    private int maxTimeoutCnt = ConfigConstants.MAX_TIMEOUT_CNT;
+
+    private boolean enableSaveManagerVIps = true;
+
+    private boolean enableSlaMetric = false;
+
+    private int managerConnectionTimeout = 10000;
+    private boolean readProxyIPFromLocal = true;
+    /**
+     * Default connection, handshake, and initial request timeout in
+     * milliseconds.
+     */
+    private long connectTimeoutMillis;
+    private long requestTimeoutMillis;
+
+    private int managerSocketTimeout = 30 * 1000;
+
+    // configuration for http client
+    // whether discard old metric when cache is full.
+    private boolean discardOldMessage = false;
+    private int proxyHttpUpdateIntervalMinutes;
+    // thread number for async sending data.
+    private int asyncWorkerNumber = 3;
+    // interval for async worker in microseconds.
+    private int asyncWorkerInterval = 500;
+    private boolean cleanHttpCacheWhenClosing = false;
+
+    // config for metric collector
+    // whether use bid as key for metric, default is true
+    private boolean useBidAsKey = true;
+    // whether use tid as key for metric, default is true
+    private boolean useTidAsKey = true;
+    // whether use localIp as key for metric, default is true
+    private boolean useLocalIpAsKey = true;
+    // metric collection interval, default is 1 mins in milliseconds.
+    private int metricIntervalInMs = 60 * 1000;
+    // max cache time for proxy config.
+    private long maxProxyCacheTimeInMs = 30 * 60 * 1000;
+
+    // metric bid
+    private String metricBid = "inlong_sla_metric";
+
+    /*pay attention to the last url parameter ip*/
+    public ProxyClientConfig(String localHost, boolean isLocalVisit, String managerIp,
+                           int managerPort, String bid, String netTag) throws ProxysdkException {
+        if (Utils.isBlank(localHost)) {
+            throw new ProxysdkException("localHost is blank!");
+        }
+        if (Utils.isBlank(managerIp)) {
+            throw new IllegalArgumentException("managerIp is Blank!");
+        }
+        this.bid = bid;
+        this.netTag = netTag;
+        this.isLocalVisit = isLocalVisit;
+        this.managerPort = managerPort;
+        this.managerIP = managerIp;
+        Utils.validLocalIp(localHost);
+        this.aliveConnections = ConfigConstants.ALIVE_CONNECTIONS;
+        this.syncThreadPoolSize = ConfigConstants.SYNC_THREAD_POOL_SIZE;
+        this.asyncCallbackSize = ConfigConstants.ASYNC_CALLBACK_SIZE;
+        this.proxyUpdateIntervalMinutes = ConfigConstants.PROXY_UPDATE_INTERVAL_MINUTES;
+        this.proxyHttpUpdateIntervalMinutes = ConfigConstants.PROXY_HTTP_UPDATE_INTERVAL_MINUTES;
+        this.proxyUpdateMaxRetry = ConfigConstants.PROXY_UPDATE_MAX_RETRY;
+        this.connectTimeoutMillis = ConfigConstants.DEFAULT_CONNECT_TIMEOUT_MILLIS;
+        this.setRequestTimeoutMillis(ConfigConstants.DEFAULT_SEND_BUFFER_SIZE);
+    }
+
+    public String getTlsServerCertFilePathAndName() {
+        return tlsServerCertFilePathAndName;
+    }
+
+    public String getTlsServerKey() {
+        return tlsServerKey;
+    }
+
+    public boolean isLocalVisit() {
+        return isLocalVisit;
+    }
+
+    public boolean isFile() {
+        return isFile;
+    }
+
+    public void setFile(boolean file) {
+        isFile = file;
+    }
+
+    public String getBid() {
+        return bid;
+    }
+
+    public void setBid(String bid) {
+        this.bid = bid;
+    }
+
+    public int getManagerPort() {
+        return managerPort;
+    }
+
+    public String getManagerIP() {
+        return managerIP;
+    }
+
+    public void setManagerIpLocalPath(String managerIpLocalPath) throws ProxysdkException {
+        if (StringUtils.isEmpty(managerIpLocalPath)) {
+            throw new ProxysdkException("managerIpLocalPath is empty.");
+        }
+        if (managerIpLocalPath.charAt(managerIpLocalPath.length() - 1) == '/') {
+            managerIpLocalPath = managerIpLocalPath.substring(0, managerIpLocalPath.length() - 1);
+        }
+        this.managerIpLocalPath = managerIpLocalPath + "/.managerIps";
+    }
+
+    public String getManagerIpLocalPath() {
+        return managerIpLocalPath;
+    }
+
+    public void setEnableSaveManagerVIps(boolean enable) {
+        this.enableSaveManagerVIps = enable;
+    }
+
+    public boolean isEnableSaveManagerVIps() {
+        return enableSaveManagerVIps;
+    }
+
+    public String getConfStoreBasePath() {
+        return confStoreBasePath;
+    }
+
+    public void setConfStoreBasePath(String confStoreBasePath) {
+        this.confStoreBasePath = confStoreBasePath;
+    }
+
+    public void setAliveConnections(int aliveConnections) {
+        this.aliveConnections = aliveConnections;
+    }
+
+    public int getAliveConnections() {
+        return this.aliveConnections;
+    }
+
+    public void setSyncThreadPoolSize(int syncThreadPoolSize) {
+        if (syncThreadPoolSize > ConfigConstants.MAX_SYNC_THREAD_POOL_SIZE) {
+            throw new IllegalArgumentException("");
+        }
+        this.syncThreadPoolSize = syncThreadPoolSize;
+    }
+
+    public int getSyncThreadPoolSize() {
+        return syncThreadPoolSize;
+    }
+
+    public void setTotalAsyncCallbackSize(int asyncCallbackSize) {
+        this.asyncCallbackSize = asyncCallbackSize;
+    }
+
+    public int getTotalAsyncCallbackSize() {
+        return asyncCallbackSize;
+    }
+
+    public String getProxyIPServiceURL() {
+        return proxyIPServiceURL;
+    }
+
+    public void setProxyUpdateIntervalMinutes(int proxyUpdateIntervalMinutes) {
+        this.proxyUpdateIntervalMinutes = proxyUpdateIntervalMinutes;
+    }
+
+    public int getMaxTimeoutCnt() {
+        return maxTimeoutCnt;
+    }
+
+    public void setMaxTimeoutCnt(int maxTimeoutCnt) {
+        if (maxTimeoutCnt < 0) {
+            throw new IllegalArgumentException("maxTimeoutCnt must bigger than 0");
+        }
+        this.maxTimeoutCnt = maxTimeoutCnt;
+    }
+
+    public int getProxyUpdateIntervalMinutes() {
+        return proxyUpdateIntervalMinutes;
+    }
+
+    public void setProxyUpdateMaxRetry(int proxyUpdateMaxRetry) {
+        this.proxyUpdateMaxRetry = proxyUpdateMaxRetry;
+    }
+
+    public int getProxyUpdateMaxRetry() {
+        return proxyUpdateMaxRetry;
+    }
+
+    public long getConnectTimeoutMillis() {
+        return connectTimeoutMillis;
+    }
+
+    public void setConnectTimeoutMillis(long connectTimeoutMillis) {
+        this.connectTimeoutMillis = connectTimeoutMillis;
+    }
+
+    public boolean isNeedVerServer() {
+        return needVerServer;
+    }
+
+    public long getRequestTimeoutMillis() {
+        return requestTimeoutMillis;
+    }
+
+    public void setRequestTimeoutMillis(long requestTimeoutMillis) {
+        this.requestTimeoutMillis = requestTimeoutMillis;
+    }
+
+    public String getNetTag() {
+        return netTag;
+    }
+
+    public String getRsaPubKeyUrl() {
+        return rsaPubKeyUrl;
+    }
+
+    public boolean isNeedDataEncry() {
+        return isNeedDataEncry;
+    }
+
+    public boolean isNeedAuthentication() {
+        return this.needAuthentication;
+    }
+
+    public void setAuthenticationInfo(boolean needAuthentication, boolean needDataEncry,
+                                      final String userName, final String secretKey) {
+        this.needAuthentication = needAuthentication;
+        this.isNeedDataEncry = needDataEncry;
+        if (this.needAuthentication || this.isNeedDataEncry) {
+            if (Utils.isBlank(userName)) {
+                throw new IllegalArgumentException("userName is Blank!");
+            }
+            if (Utils.isBlank(secretKey)) {
+                throw new IllegalArgumentException("secretKey is Blank!");
+            }
+        }
+        this.userName = userName.trim();
+        this.secretKey = secretKey.trim();
+    }
+
+    public void setHttpsInfo(String tlsServerCertFilePathAndName, String tlsServerKey) {
+        if (Utils.isBlank(tlsServerCertFilePathAndName)) {
+            throw new IllegalArgumentException("tlsServerCertFilePathAndName is Blank!");
+        }
+        if (Utils.isBlank(tlsServerKey)) {
+            throw new IllegalArgumentException("tlsServerKey is Blank!");
+        }
+        this.needVerServer = true;
+        this.tlsServerKey = tlsServerKey;
+        this.tlsServerCertFilePathAndName = tlsServerCertFilePathAndName;
+    }
+
+    public String getUserName() {
+        return userName;
+    }
+
+    public String getSecretKey() {
+        return secretKey;
+    }
+
+    public boolean isReadProxyIPFromLocal() {
+        return readProxyIPFromLocal;
+    }
+
+    public void setReadProxyIPFromLocal(boolean readProxyIPFromLocal) {
+        this.readProxyIPFromLocal = readProxyIPFromLocal;
+    }
+
+    public int getProxyHttpUpdateIntervalMinutes() {
+        return proxyHttpUpdateIntervalMinutes;
+    }
+
+    public void setProxyHttpUpdateIntervalMinutes(int proxyHttpUpdateIntervalMinutes) {
+        this.proxyHttpUpdateIntervalMinutes = proxyHttpUpdateIntervalMinutes;
+    }
+
+    public boolean isDiscardOldMessage() {
+        return discardOldMessage;
+    }
+
+    public void setDiscardOldMessage(boolean discardOldMessage) {
+        this.discardOldMessage = discardOldMessage;
+    }
+
+    public int getAsyncWorkerNumber() {
+        return asyncWorkerNumber;
+    }
+
+    public void setAsyncWorkerNumber(int asyncWorkerNumber) {
+        this.asyncWorkerNumber = asyncWorkerNumber;
+    }
+
+    public int getAsyncWorkerInterval() {
+        return asyncWorkerInterval;
+    }
+
+    public void setAsyncWorkerInterval(int asyncWorkerInterval) {
+        this.asyncWorkerInterval = asyncWorkerInterval;
+    }
+
+    public int getManagerSocketTimeout() {
+        return managerSocketTimeout;
+    }
+
+    public void setManagerSocketTimeout(int managerSocketTimeout) {
+        this.managerSocketTimeout = managerSocketTimeout;
+    }
+
+    public boolean isCleanHttpCacheWhenClosing() {
+        return cleanHttpCacheWhenClosing;
+    }
+
+    public void setCleanHttpCacheWhenClosing(boolean cleanHttpCacheWhenClosing) {
+        this.cleanHttpCacheWhenClosing = cleanHttpCacheWhenClosing;
+    }
+
+    public boolean isUseBidAsKey() {
+        return useBidAsKey;
+    }
+
+    public void setUseBidAsKey(boolean useBidAsKey) {
+        this.useBidAsKey = useBidAsKey;
+    }
+
+    public boolean isUseTidAsKey() {
+        return useTidAsKey;
+    }
+
+    public void setUseTidAsKey(boolean useTidAsKey) {
+        this.useTidAsKey = useTidAsKey;
+    }
+
+    public boolean isUseLocalIpAsKey() {
+        return useLocalIpAsKey;
+    }
+
+    public void setUseLocalIpAsKey(boolean useLocalIpAsKey) {
+        this.useLocalIpAsKey = useLocalIpAsKey;
+    }
+
+    public int getMetricIntervalInMs() {
+        return metricIntervalInMs;
+    }
+
+    public void setMetricIntervalInMs(int metricIntervalInMs) {
+        this.metricIntervalInMs = metricIntervalInMs;
+    }
+
+
+    public String getMetricBid() {
+        return metricBid;
+    }
+
+    public void setMetricBid(String metricBid) {
+        this.metricBid = metricBid;
+    }
+
+    public long getMaxProxyCacheTimeInMs() {
+        return maxProxyCacheTimeInMs;
+    }
+
+    public void setMaxProxyCacheTimeInMs(long maxProxyCacheTimeInMs) {
+        this.maxProxyCacheTimeInMs = maxProxyCacheTimeInMs;
+    }
+
+    public int getManagerConnectionTimeout() {
+        return managerConnectionTimeout;
+    }
+
+    public void setManagerConnectionTimeout(int managerConnectionTimeout) {
+        this.managerConnectionTimeout = managerConnectionTimeout;
+    }
+
+    public boolean isEnableSlaMetric() {
+        return enableSlaMetric;
+    }
+
+    public void setEnableSlaMetric(boolean enableSlaMetric) {
+        this.enableSlaMetric = enableSlaMetric;
+    }
+}
diff --git a/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/SendMessageCallback.java b/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/SendMessageCallback.java
new file mode 100644
index 0000000..9489980
--- /dev/null
+++ b/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/SendMessageCallback.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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.dataproxy;
+
+public interface SendMessageCallback {
+    /* Invoked when a message is confirmed by TDBus. */
+    public void onMessageAck(SendResult result);
+
+    /* Invoked when a message transportation interrupted by an exception. */
+    public void onException(Throwable e);
+}
diff --git a/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/SendResult.java b/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/SendResult.java
new file mode 100644
index 0000000..883e4ff
--- /dev/null
+++ b/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/SendResult.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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.dataproxy;
+
+public enum SendResult {
+    INVALID_ATTRIBUTES,
+    OK,
+    TIMEOUT,
+    CONNECTION_BREAK,
+    THREAD_INTERRUPT,
+    ASYNC_CALLBACK_BUFFER_FULL,
+    NO_CONNECTION,
+    INVALID_DATA,
+    UNKOWN_ERROR
+
+}
diff --git a/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/codec/EncodeObject.java b/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/codec/EncodeObject.java
new file mode 100644
index 0000000..d8d72d9
--- /dev/null
+++ b/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/codec/EncodeObject.java
@@ -0,0 +1,387 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.dataproxy.codec;
+
+import java.util.List;
+
+import org.apache.inlong.dataproxy.config.EncryptConfigEntry;
+
+public class EncodeObject {
+    private static final String MESSAGE_ID_PREFIX = "messageId=";
+
+    private byte[] bodyBytes;
+    private String attributes;
+    private String messageId;
+    private int msgtype;
+    private List<byte[]> bodylist;
+    private String commonattr = "";
+    private String messageKey = "data";
+    private String proxyIp = "";
+    // private long seqId
+    private long dt;
+    // package time
+    private long packageTime = System.currentTimeMillis();
+    private int cnt = -1;
+    private boolean isReport = false;
+    private boolean isBidTransfer = false;
+    private boolean isSupportLF = false;
+    private boolean isAuth = false;
+    private boolean isEncrypt = false;
+    private boolean isCompress = true;
+    private int bidNum;
+    private int tidNum;
+    private String bid;
+    private String tid;
+    private short load;
+    private String userName = "";
+    private String secretKey = "";
+    private String msgUUID = null;
+    private EncryptConfigEntry encryptEntry = null;
+
+
+    private boolean isException = false;
+    private ErrorCode exceptionError = null;
+
+    /* Used by de_serialization. msgtype=7/8*/
+    public EncodeObject() {
+    }
+
+    /* Used by de_serialization. */
+    public EncodeObject(byte[] bodyBytes, String attributes) {
+        this.bodyBytes = bodyBytes;
+        this.attributes = attributes;
+        this.messageId = "";
+        String[] tokens = attributes.split("&");
+        for (int i = 0; i < tokens.length; i++) {
+            if (tokens[i].startsWith("messageId=")) {
+                this.messageId = tokens[i].substring(MESSAGE_ID_PREFIX.length(), tokens[i].length());
+                break;
+            }
+        }
+    }
+
+    /* Used by serialization.But never used */
+// old version:we need add message id by attr
+    public EncodeObject(byte[] bodyBytes, String attributes, String messageId) {
+        this.bodyBytes = bodyBytes;
+        this.messageId = messageId;
+        this.attributes = attributes + "&messageId=" + messageId;
+    }
+
+    // used for bytes initializtion,msgtype=3/5
+    public EncodeObject(byte[] bodyBytes, String attributes, String messageId,
+                        int msgtype, boolean isCompress, final String bid) {
+        this.bodyBytes = bodyBytes;
+        this.messageId = messageId;
+        this.attributes = attributes + "&messageId=" + messageId;
+        this.msgtype = msgtype;
+        this.bid = bid;
+        this.isCompress = isCompress;
+    }
+
+    // used for bodylist initializtion,msgtype=3/5
+    public EncodeObject(List<byte[]> bodyList, String attributes, String messageId,
+                        int msgtype, boolean isCompress, final String bid) {
+        this.bodylist = bodyList;
+        this.messageId = messageId;
+        this.attributes = attributes + "&messageId=" + messageId;
+        this.msgtype = msgtype;
+        this.bid = bid;
+        this.isCompress = isCompress;
+    }
+
+    // used for bytes initializtion,msgtype=7/8
+    public EncodeObject(byte[] bodyBytes, int msgtype, boolean isCompress, boolean isReport,
+                        boolean isBidTransfer, long dt, long seqId, String bid, String tid, String commonattr) {
+        this.bodyBytes = bodyBytes;
+        this.msgtype = msgtype;
+        this.isCompress = isCompress;
+        this.isReport = isReport;
+        this.dt = dt;
+        this.isBidTransfer = isBidTransfer;
+        this.commonattr = commonattr;
+        this.messageId = String.valueOf(seqId);
+        this.bid = bid;
+        this.tid = tid;
+    }
+
+    // used for bodylist initializtion,msgtype=7/8
+    public EncodeObject(List<byte[]> bodyList, int msgtype, boolean isCompress,
+                        boolean isReport, boolean isBidTransfer, long dt,
+                        long seqId, String bid, String tid, String commonattr) {
+        this.bodylist = bodyList;
+        this.msgtype = msgtype;
+        this.isCompress = isCompress;
+        this.isReport = isReport;
+        this.dt = dt;
+        this.isBidTransfer = isBidTransfer;
+        this.commonattr = commonattr;
+        this.messageId = String.valueOf(seqId);
+        this.bid = bid;
+        this.tid = tid;
+    }
+
+    // file agent, used for bytes initializtion,msgtype=7/8
+    public EncodeObject(byte[] bodyBytes, int msgtype, boolean isCompress,
+                        boolean isReport, boolean isBidTransfer, long dt,
+                        long seqId, String bid, String tid, String commonattr, String messageKey, String proxyIp) {
+        this.bodyBytes = bodyBytes;
+        this.msgtype = msgtype;
+        this.isCompress = isCompress;
+        this.isReport = isReport;
+        this.dt = dt;
+        this.isBidTransfer = isBidTransfer;
+        this.commonattr = commonattr;
+        this.messageId = String.valueOf(seqId);
+        this.bid = bid;
+        this.tid = tid;
+        this.messageKey = messageKey;
+        this.proxyIp = proxyIp;
+    }
+
+    // file agent, used for bodylist initializtion,msgtype=7/8
+    public EncodeObject(List<byte[]> bodyList, int msgtype, boolean isCompress,
+                        boolean isReport, boolean isBidTransfer, long dt,
+                        long seqId, String bid, String tid, String commonattr, String messageKey, String proxyIp) {
+        this.bodylist = bodyList;
+        this.msgtype = msgtype;
+        this.isCompress = isCompress;
+        this.isReport = isReport;
+        this.dt = dt;
+        this.isBidTransfer = isBidTransfer;
+        this.commonattr = commonattr;
+        this.messageId = String.valueOf(seqId);
+        this.bid = bid;
+        this.tid = tid;
+        this.messageKey = messageKey;
+        this.proxyIp = proxyIp;
+    }
+
+    public String getMsgUUID() {
+        return msgUUID;
+    }
+
+    public void setMsgUUID(String msgUUID) {
+        this.msgUUID = msgUUID;
+    }
+
+    public boolean isBidTransfer() {
+        return isBidTransfer;
+    }
+
+    public void setBidTransfer(boolean isBidTransfer) {
+        this.isBidTransfer = isBidTransfer;
+    }
+
+    public short getLoad() {
+        return load;
+    }
+
+    public void setLoad(short load) {
+        this.load = load;
+    }
+
+    public String getBid() {
+        return bid;
+    }
+
+    public void setBid(String bid) {
+        this.bid = bid;
+    }
+
+    public String getTid() {
+        return tid;
+    }
+
+    public void setTid(String tid) {
+        this.tid = tid;
+    }
+
+    public void setMsgtype(int msgtype) {
+        this.msgtype = msgtype;
+    }
+
+    public void setBodyBytes(byte[] bodyBytes) {
+        this.bodyBytes = bodyBytes;
+    }
+
+    public boolean isReport() {
+        return isReport;
+    }
+
+    public void setReport(boolean isReport) {
+        this.isReport = isReport;
+    }
+
+    public boolean isAuth() {
+        return isAuth;
+    }
+
+    public void setAuth(boolean auth, final String userName, final String secretKey) {
+        this.isAuth = auth;
+        this.userName = userName;
+        this.secretKey = secretKey;
+    }
+
+    public String getUserName() {
+        return userName;
+    }
+
+    public String getSecretKey() {
+        return secretKey;
+    }
+
+    public boolean isEncrypt() {
+        return isEncrypt;
+    }
+
+    public EncryptConfigEntry getEncryptEntry() {
+        return encryptEntry;
+    }
+
+    public void setEncryptEntry(boolean isEncrypt, String userName, EncryptConfigEntry encryptEntry) {
+        this.isEncrypt = isEncrypt;
+        if (userName != null) {
+            this.userName = userName;
+        }
+        this.encryptEntry = encryptEntry;
+    }
+
+    public int getBidNum() {
+        return bidNum;
+    }
+
+    public void setBidNum(int bidNum) {
+        this.bidNum = bidNum;
+    }
+
+    public int getTidNum() {
+        return tidNum;
+    }
+
+    public void setTidNum(int tidNum) {
+        this.tidNum = tidNum;
+    }
+
+    public long getDt() {
+        return dt;
+    }
+
+    public void setDt(long dt) {
+        this.dt = dt;
+    }
+
+    public long getPackageTime() {
+        return packageTime;
+    }
+
+    public void setPackageTime(long packageTime) {
+        this.packageTime = packageTime;
+    }
+
+    public String getCommonattr() {
+        return commonattr;
+    }
+
+    public void setCommonattr(String commonattr) {
+        this.commonattr = commonattr;
+    }
+
+    public boolean isCompress() {
+        return isCompress;
+    }
+
+    public List<byte[]> getBodylist() {
+        return bodylist;
+    }
+
+    public int getMsgtype() {
+        return msgtype;
+    }
+
+    public byte[] getBodyBytes() {
+        return bodyBytes;
+    }
+
+    public String getAttributes() {
+        return attributes;
+    }
+
+    public String getMessageId() {
+        return messageId;
+    }
+
+    public void setMessageId(String messageId) {
+        this.messageId = messageId;
+    }
+
+    public String getMessageKey() {
+        return messageKey;
+    }
+
+    public void setMessageKey(String messageKey) {
+        this.messageKey = messageKey;
+    }
+
+    public String getProxyIp() {
+        return proxyIp;
+    }
+
+    public void setProxyIp(String proxyIp) {
+        this.proxyIp = proxyIp;
+    }
+
+    public boolean isSupportLF() {
+        return isSupportLF;
+    }
+
+    public void setSupportLF(boolean supportLF) {
+        isSupportLF = supportLF;
+    }
+
+    public int getCnt() {
+        return cnt;
+    }
+
+    public int getRealCnt() {
+        if (bodylist != null) {
+            return bodylist.size();
+        }
+        return 1;
+    }
+
+    public void setCnt(int cnt) {
+        this.cnt = cnt;
+    }
+
+    public boolean isException() {
+        return isException;
+    }
+
+    public void setException(boolean exception) {
+        isException = exception;
+    }
+
+    public ErrorCode getExceptionError() {
+        return exceptionError;
+    }
+
+    public void setExceptionError(ErrorCode exceptionError) {
+        this.exceptionError = exceptionError;
+    }
+}
diff --git a/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/codec/ErrorCode.java b/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/codec/ErrorCode.java
new file mode 100644
index 0000000..7a687ca
--- /dev/null
+++ b/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/codec/ErrorCode.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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.dataproxy.codec;
+
+import java.util.HashMap;
+import java.util.Map;
+
+public enum ErrorCode {
+
+
+    ATTR_ERROR(1),
+
+    DT_ERROR(2),
+
+    COMPRESS_ERROR(3),
+
+    OTHER_ERROR(4),
+
+    LONG_LENGTH_ERROR(5);
+    private final int value;
+    private static final Map<Integer, ErrorCode> map = new HashMap<>();
+
+    static {
+        for (ErrorCode errorCode : ErrorCode.values()) {
+            map.put(errorCode.value, errorCode);
+        }
+    }
+
+    ErrorCode(int value) {
+        this.value = value;
+    }
+
+    public static ErrorCode valueOf(int value) {
+        return map.get(value);
+    }
+
+}
diff --git a/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/codec/ProtocolDecoder.java b/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/codec/ProtocolDecoder.java
new file mode 100644
index 0000000..c4907a8
--- /dev/null
+++ b/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/codec/ProtocolDecoder.java
@@ -0,0 +1,117 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.dataproxy.codec;
+
+import java.nio.charset.StandardCharsets;
+
+import org.jboss.netty.buffer.ChannelBuffer;
+import org.jboss.netty.channel.Channel;
+import org.jboss.netty.channel.ChannelHandlerContext;
+import org.jboss.netty.channel.DefaultExceptionEvent;
+import org.jboss.netty.handler.codec.frame.FrameDecoder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ProtocolDecoder extends FrameDecoder {
+
+    private static final Logger logger = LoggerFactory.getLogger(FrameDecoder.class);
+
+    @Override
+    protected Object decode(ChannelHandlerContext ctx, Channel channel,
+                            ChannelBuffer buffer) throws Exception {
+//        if(!channel.isConnected()||channel.isReadable()){return null;}
+
+
+        buffer.markReaderIndex();
+        // totallen
+        int totalLen = buffer.readInt();
+        //logger.info("totalLen : {}",totalLen);
+        if (totalLen != buffer.readableBytes()) {
+            logger.error("totalLen is not equal readableBytes.total:" + totalLen
+                    + ";readableBytes:" + buffer.readableBytes());
+            buffer.resetReaderIndex();
+            return new DefaultExceptionEvent(channel, new Exception("totalLen is not equal readableBytes.total"));
+        }
+        // msgtype
+        int msgType = buffer.readByte() & 0x1f;
+
+        if (msgType == 4) {
+            logger.info("debug decode");
+        }
+        if (msgType == 3 | msgType == 5) {
+            // bodylen
+            int bodyLength = buffer.readInt();
+            if (bodyLength >= totalLen) {
+                logger.error("bodyLen is greater than totalLen.totalLen:" + totalLen
+                        + ";bodyLen:" + bodyLength);
+                buffer.resetReaderIndex();
+                return new DefaultExceptionEvent(channel, new Exception("bodyLen is greater than totalLen.totalLen"));
+            }
+            byte[] bodyBytes = null;
+            if (bodyLength > 0) {
+                bodyBytes = new byte[bodyLength];
+                buffer.readBytes(bodyBytes);
+            }
+
+            // attrlen
+            int attrLength = buffer.readInt();
+            byte[] attrBytes = null;
+            if (attrLength > 0) {
+                attrBytes = new byte[attrLength];
+                buffer.readBytes(attrBytes);
+            }
+            EncodeObject object = new EncodeObject(bodyBytes, new String(attrBytes,
+                    StandardCharsets.UTF_8));
+            object.setMsgtype(5);
+            return object;
+        } else if (msgType == 7) {
+
+            int attrLen = buffer.readShort();
+
+            EncodeObject object = new EncodeObject();
+            if (attrLen == 4) {
+                int errorValue = buffer.readInt();
+                ErrorCode errorCode = ErrorCode.valueOf(errorValue);
+                if (errorCode != null) {
+                    object.setException(true);
+                    object.setExceptionError(errorCode);
+                }
+            } else {
+                byte[] attrContent = new byte[attrLen];
+                buffer.readBytes(attrContent);
+            }
+
+
+            buffer.readShort();
+
+            int seqId = buffer.readInt();
+            object.setMsgtype(msgType);
+            object.setMessageId(String.valueOf(seqId));
+            return object;
+
+        } else if (msgType == 8) {
+            int attrlen = buffer.getShort(4 + 1 + 4 + 1 + 4 + 2);
+            buffer.skipBytes(13 + attrlen + 2);
+            EncodeObject object = new EncodeObject();
+            object.setMsgtype(8);
+            object.setLoad(buffer.getShort(4 + 1 + 4 + 1 + 4));
+            return object;
+        }
+        return null;
+    }
+}
diff --git a/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/codec/ProtocolEncoder.java b/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/codec/ProtocolEncoder.java
new file mode 100644
index 0000000..b3f033a
--- /dev/null
+++ b/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/codec/ProtocolEncoder.java
@@ -0,0 +1,420 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.dataproxy.codec;
+
+import static org.apache.inlong.dataproxy.ConfigConstants.FLAG_ALLOW_AUTH;
+import static org.apache.inlong.dataproxy.ConfigConstants.FLAG_ALLOW_COMPRESS;
+import static org.apache.inlong.dataproxy.ConfigConstants.FLAG_ALLOW_ENCRYPT;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.UnsupportedEncodingException;
+import java.nio.ByteBuffer;
+import java.security.SecureRandom;
+import java.util.Iterator;
+
+import org.apache.inlong.dataproxy.config.EncryptConfigEntry;
+import org.apache.inlong.dataproxy.config.EncryptInfo;
+import org.apache.inlong.dataproxy.network.Utils;
+import org.apache.inlong.dataproxy.utils.EncryptUtil;
+import org.jboss.netty.buffer.ChannelBuffer;
+import org.jboss.netty.buffer.ChannelBuffers;
+import org.jboss.netty.channel.Channel;
+import org.jboss.netty.channel.ChannelHandlerContext;
+import org.jboss.netty.handler.codec.oneone.OneToOneEncoder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.xerial.snappy.Snappy;
+
+public class ProtocolEncoder extends OneToOneEncoder {
+    private static final Logger logger = LoggerFactory
+            .getLogger(ProtocolEncoder.class);
+
+    @Override
+    protected Object encode(ChannelHandlerContext ctx, Channel channel,
+                            Object message) {
+        ChannelBuffer buf = null;
+        try {
+            EncodeObject object = (EncodeObject) message;
+
+            buf = ChannelBuffers.dynamicBuffer();
+            if (object.getMsgtype() == 3) {
+                buf = writeToBuf3(object);
+            }
+            if (object.getMsgtype() == 5) {
+                buf = writeToBuf5(object);
+            }
+
+            if (object.getMsgtype() == 7) {
+                buf = writeToBuf7(object);
+            }
+            if (object.getMsgtype() == 8) {
+                buf = writeToBuf8(object);
+            }
+        } catch (Exception e) {
+            logger.error("{}", e.getMessage());
+            e.printStackTrace();
+        }
+        return buf;
+    }
+
+    private ChannelBuffer writeToBuf8(EncodeObject object) {
+        ChannelBuffer buf = ChannelBuffers.dynamicBuffer();
+        try {
+            String endAttr = object.getCommonattr();
+            if (object.isAuth()) {
+                if (Utils.isNotBlank(endAttr)) {
+                    endAttr = endAttr + "&";
+                }
+                long timestamp = System.currentTimeMillis();
+                int nonce = new SecureRandom(String.valueOf(timestamp).getBytes()).nextInt(Integer.MAX_VALUE);
+                endAttr = endAttr + "_userName=" + object.getUserName() + "&_clientIP=" + Utils.getLocalIp()
+                        + "&_signature=" + Utils.generateSignature(object.getUserName(),
+                        timestamp, nonce, object.getSecretKey())
+                        + "&_timeStamp=" + timestamp + "&_nonce=" + nonce;
+            }
+            if (Utils.isNotBlank(object.getMsgUUID())) {
+                if (Utils.isNotBlank(endAttr)) {
+                    endAttr = endAttr + "&";
+                }
+                endAttr = endAttr + "msgUUID=" + object.getMsgUUID();
+            }
+            int msgType = 8;
+            if (object.isAuth()) {
+                msgType |= FLAG_ALLOW_AUTH;
+            }
+            int totalLength = 1 + 4 + 1 + 4 + 2 + endAttr.getBytes("utf8").length + 2;
+            buf.writeInt(totalLength);
+            buf.writeByte(msgType);
+            buf.writeInt((int) object.getDt());
+            buf.writeByte(1);
+            buf.writeInt(0);
+            buf.writeShort(endAttr.getBytes("utf8").length);
+            if (endAttr.getBytes("utf8").length > 0) {
+                buf.writeBytes(endAttr.getBytes("utf8"));
+            }
+            buf.writeShort(0xee01);
+        } catch (Exception e) {
+            logger.error(e.getMessage());
+        }
+        return buf;
+    }
+
+    private void constructBody(byte[] body, EncodeObject object,
+        int totalLength, ChannelBuffer buf, int cnt) throws UnsupportedEncodingException {
+        if (body != null) {
+            if (object.isCompress()) {
+                body = processCompress(body);
+            }
+            String endAttr = object.getCommonattr();
+            if (object.isEncrypt()) {
+                EncryptConfigEntry encryptEntry = object.getEncryptEntry();
+                if (encryptEntry != null) {
+                    if (Utils.isNotBlank(endAttr)) {
+                        endAttr = endAttr + "&";
+                    }
+                    EncryptInfo encryptInfo = encryptEntry.getRsaEncryptInfo();
+                    endAttr = endAttr + "_userName=" + object.getUserName()
+                        + "&_encyVersion=" + encryptInfo.getVersion()
+                        + "&_encyDesKey=" + encryptInfo.getRsaEncryptedKey();
+                    body = EncryptUtil.desEncrypt(body, encryptInfo.getDesKey());
+                }
+            }
+            if (!object.isBidTransfer()) {
+                if (Utils.isNotBlank(endAttr)) {
+                    endAttr = endAttr + "&";
+                }
+                endAttr = (endAttr + "bid=" + object.getBid() + "&tid=" + object.getTid());
+            }
+            if (Utils.isNotBlank(object.getMsgUUID())) {
+                if (Utils.isNotBlank(endAttr)) {
+                    endAttr = endAttr + "&";
+                }
+                endAttr = endAttr + "msgUUID=" + object.getMsgUUID();
+            }
+
+            int msgType = 7;
+            if (object.isEncrypt()) {
+                msgType |= FLAG_ALLOW_ENCRYPT;
+            }
+            if (object.isCompress()) {
+                msgType |= FLAG_ALLOW_COMPRESS;
+            }
+            totalLength = totalLength + body.length + endAttr.getBytes("utf8").length;
+            buf.writeInt(totalLength);
+            buf.writeByte(msgType);
+            buf.writeShort(object.getBidNum());
+            buf.writeShort(object.getTidNum());
+            String bitStr = object.isSupportLF() ? "1" : "0";
+            bitStr += (object.getMessageKey().equals("minute")) ? "1" : "0";
+            bitStr += (object.getMessageKey().equals("file")) ? "1" : "0";
+            bitStr += !object.isBidTransfer() ? "1" : "0";
+            bitStr += object.isReport() ? "1" : "0";
+            bitStr += "0";
+            buf.writeShort(Integer.parseInt(bitStr, 2));
+            buf.writeInt((int) object.getDt());
+
+            buf.writeShort(cnt);
+            buf.writeInt(Integer.valueOf(object.getMessageId()));
+
+            buf.writeInt(body.length);
+            buf.writeBytes(body);
+
+            buf.writeShort(endAttr.getBytes("utf8").length);
+            buf.writeBytes(endAttr.getBytes("utf8"));
+            buf.writeShort(0xee01);
+        }
+    }
+
+    private ChannelBuffer writeToBuf7(EncodeObject object) {
+        ChannelBuffer buf = ChannelBuffers.dynamicBuffer();
+        try {
+            int totalLength = 1 + 2 + 2 + 2 + 4 + 2 + 4 + 4 + 2 + 2;
+            byte[] body = null;
+            int cnt = 1;
+
+            if (object.getBodylist() != null && object.getBodylist().size() != 0) {
+                if (object.getCnt() > 0) {
+                    cnt = object.getCnt();
+                } else {
+                    cnt = object.getBodylist().size();
+                }
+
+                ByteArrayOutputStream out = new ByteArrayOutputStream();
+                Iterator<byte[]> iter = object.getBodylist().iterator();
+
+                if (object.isSupportLF()) {
+                    ByteArrayOutputStream data = new ByteArrayOutputStream();
+                    int len = object.getBodylist().size();
+                    for (int i = 0; i < len - 1; i++) {
+                        data.write(object.getBodylist().get(i));
+                        data.write("\n".getBytes("utf8"));
+                    }
+                    data.write(object.getBodylist().get(len - 1));
+                    ByteBuffer databuffer = ByteBuffer.allocate(4);
+                    databuffer.putInt(data.toByteArray().length);
+                    out.write(databuffer.array());
+                    out.write(data.toByteArray());
+                } else {
+                    while (iter.hasNext()) {
+                        byte[] entry = iter.next();
+                        ByteBuffer databuffer = ByteBuffer.allocate(4);
+                        databuffer.putInt(entry.length);
+                        out.write(databuffer.array());
+                        out.write(entry);
+                    }
+                }
+                body = out.toByteArray();
+            }
+            //send single message one time
+            if (object.getBodyBytes() != null && object.getBodyBytes().length != 0) {
+                ByteArrayOutputStream out = new ByteArrayOutputStream();
+
+                ByteBuffer databuffer = ByteBuffer.allocate(4);
+                databuffer.putInt(object.getBodyBytes().length);
+                out.write(databuffer.array());
+                out.write(object.getBodyBytes());
+                body = out.toByteArray();
+            }
+
+            constructBody(body, object, totalLength, buf, cnt);
+        } catch (Exception e) {
+            logger.error("{}", e.getMessage());
+            e.printStackTrace();
+        }
+        return buf;
+    }
+
+    private ChannelBuffer writeToBuf5(EncodeObject object) {
+        ChannelBuffer buf = ChannelBuffers.dynamicBuffer();
+        try {
+            int totalLength = 1 + 4 + 4;
+            byte[] body = null;
+
+            //send multiple  messages one time
+            if (object.getBodylist() != null && object.getBodylist().size() != 0) {
+                ByteArrayOutputStream out = new ByteArrayOutputStream();
+                Iterator<byte[]> iter = object.getBodylist().iterator();
+                while (iter.hasNext()) {
+                    byte[] entry = iter.next();
+                    ByteBuffer byteBuffer = ByteBuffer.allocate(4);
+                    byteBuffer.putInt(entry.length);
+                    out.write(byteBuffer.array());
+                    out.write(entry);
+                }
+                body = out.toByteArray();
+            }
+            //send single message one time
+            if (object.getBodyBytes() != null && object.getBodyBytes().length != 0) {
+                ByteArrayOutputStream out = new ByteArrayOutputStream();
+                ByteBuffer byteBuffer = ByteBuffer.allocate(4);
+                byteBuffer.putInt(object.getBodyBytes().length);
+                out.write(byteBuffer.array());
+                out.write(object.getBodyBytes());
+                body = out.toByteArray();
+            }
+            if (body != null) {
+                String msgAttrs = object.getAttributes();
+                if (object.isCompress()) {
+                    body = processCompress(body);
+                }
+                if (object.isEncrypt()) {
+                    EncryptConfigEntry encryptEntry = object.getEncryptEntry();
+                    if (encryptEntry != null) {
+                        if (Utils.isNotBlank(msgAttrs)) {
+                            msgAttrs = msgAttrs + "&";
+                        }
+                        EncryptInfo encryptInfo = encryptEntry.getRsaEncryptInfo();
+                        msgAttrs = msgAttrs + "_userName=" + object.getUserName()
+                                + "&_encyVersion=" + encryptInfo.getVersion()
+                                + "&_encyDesKey=" + encryptInfo.getRsaEncryptedKey();
+                        body = EncryptUtil.desEncrypt(body, encryptInfo.getDesKey());
+                    }
+                }
+                if (Utils.isNotBlank(object.getMsgUUID())) {
+                    if (Utils.isNotBlank(msgAttrs)) {
+                        msgAttrs = msgAttrs + "&";
+                    }
+                    msgAttrs = msgAttrs + "msgUUID=" + object.getMsgUUID();
+                }
+
+                int msgType = 5;
+                if (object.isEncrypt()) {
+                    msgType |= FLAG_ALLOW_ENCRYPT;
+                }
+                totalLength = totalLength + body.length + msgAttrs.getBytes("utf8").length;
+                buf.writeInt(totalLength);
+                buf.writeByte(msgType);
+                buf.writeInt(body.length);
+                buf.writeBytes(body);
+                buf.writeInt(msgAttrs.getBytes("utf8").length);
+                buf.writeBytes(msgAttrs.getBytes("utf8"));
+            }
+        } catch (Exception e) {
+            logger.error("{}", e.getMessage());
+            e.printStackTrace();
+        }
+        return buf;
+    }
+
+    /*private ChannelBuffer writeToBuf4(EncodeObject object) {
+        ChannelBuffer buf = ChannelBuffers.dynamicBuffer();
+        try {
+            int totalLength = 1 + 4 + 4;
+            byte[] body = null;
+
+            //send single message one time
+            if (object.getBodyBytes() != null && object.getBodyBytes().length != 0) {
+                body = object.getBodyBytes();
+            }
+            totalLength = totalLength + body.length + object.getAttributes().getBytes("utf8").length;
+            buf.writeInt(totalLength);
+            buf.writeByte(4);
+            buf.writeInt(body.length);
+            buf.writeBytes(body);
+            buf.writeInt(object.getAttributes().getBytes().length);
+            buf.writeBytes(object.getAttributes().getBytes());
+        } catch (Exception e) {
+            logger.error(e.getMessage());
+        }
+        return buf;
+    }
+*/
+    private ChannelBuffer writeToBuf3(EncodeObject object) {
+        ChannelBuffer buf = ChannelBuffers.dynamicBuffer();
+        try {
+            int totalLength = 1 + 4 + 4;
+            byte[] body = null;
+
+            //send multiple  messages one time
+            if (object.getBodylist() != null && object.getBodylist().size() != 0) {
+                ByteArrayOutputStream out = new ByteArrayOutputStream();
+                Iterator<byte[]> iter = object.getBodylist().iterator();
+                while (iter.hasNext()) {
+                    byte[] entry = iter.next();
+                    out.write(entry);
+                    out.write("\n".getBytes("utf8"));
+                }
+                body = out.toByteArray();
+            }
+            //send single message one time
+            if (object.getBodyBytes() != null && object.getBodyBytes().length != 0) {
+                body = object.getBodyBytes();
+            }
+            if (body != null) {
+                String msgAttrs = object.getAttributes();
+                if (object.isCompress()) {
+                    body = processCompress(body);
+                }
+                if (object.isEncrypt()) {
+                    EncryptConfigEntry encryptEntry = object.getEncryptEntry();
+                    if (encryptEntry != null) {
+                        if (Utils.isNotBlank(msgAttrs)) {
+                            msgAttrs = msgAttrs + "&";
+                        }
+                        EncryptInfo encryptInfo = encryptEntry.getRsaEncryptInfo();
+                        msgAttrs = msgAttrs + "_userName=" + object.getUserName()
+                                + "&_encyVersion=" + encryptInfo.getVersion()
+                                + "&_encyDesKey=" + encryptInfo.getRsaEncryptedKey();
+                        body = EncryptUtil.desEncrypt(body, encryptInfo.getDesKey());
+                    }
+                }
+                if (Utils.isNotBlank(object.getMsgUUID())) {
+                    if (Utils.isNotBlank(msgAttrs)) {
+                        msgAttrs = msgAttrs + "&";
+                    }
+                    msgAttrs = msgAttrs + "msgUUID=" + object.getMsgUUID();
+                }
+
+                int msgType = 3;
+                if (object.isEncrypt()) {
+                    msgType |= FLAG_ALLOW_ENCRYPT;
+                }
+                totalLength = totalLength + body.length + msgAttrs.getBytes("utf8").length;
+                buf.writeInt(totalLength);
+                buf.writeByte(msgType);
+                buf.writeInt(body.length);
+                buf.writeBytes(body);
+                buf.writeInt(msgAttrs.getBytes("utf8").length);
+                buf.writeBytes(msgAttrs.getBytes("utf8"));
+            }
+        } catch (Exception e) {
+            logger.error("{}", e.getMessage());
+            e.printStackTrace();
+        }
+        return buf;
+    }
+
+
+    private byte[] processCompress(byte[] body) {
+        try {
+            ByteArrayOutputStream out = new ByteArrayOutputStream();
+            out.write(body);
+            int guessLen = Snappy.maxCompressedLength(out.size());
+            byte[] tmpData = new byte[guessLen];
+            int len = Snappy.compress(out.toByteArray(), 0, out.size(),
+                    tmpData, 0);
+            body = new byte[len];
+            System.arraycopy(tmpData, 0, body, 0, len);
+        } catch (IOException e) {
+            logger.error("{}", e.getMessage());
+            e.printStackTrace();
+        }
+        return body;
+    }
+}
diff --git a/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/config/EncryptConfigEntry.java b/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/config/EncryptConfigEntry.java
new file mode 100644
index 0000000..a045ba7
--- /dev/null
+++ b/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/config/EncryptConfigEntry.java
@@ -0,0 +1,145 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.dataproxy.config;
+
+import java.net.URLEncoder;
+import java.security.interfaces.RSAPublicKey;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.commons.codec.binary.Base64;
+import org.apache.inlong.dataproxy.utils.EncryptUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Created by lamberliu on 2016/5/13.
+ */
+public class EncryptConfigEntry implements java.io.Serializable {
+    private static final Logger logger = LoggerFactory.getLogger(EncryptConfigEntry.class);
+    private String userName = "";
+    private String version;
+    private String pubKey;
+    private byte[] desKey;
+    private String rsaEncryptedKey;
+    private AtomicLong lastUpdateTime = new AtomicLong(0);
+
+
+    public EncryptConfigEntry(final String userName, final String version, final String pubKey) {
+        this.userName = userName;
+        this.version = version;
+        this.pubKey = pubKey;
+        this.desKey = null;
+        this.rsaEncryptedKey = null;
+//        this.rsaKey = EncryptUtil.loadPublicKeyByText(pubKey);
+    }
+
+    public String getVersion() {
+        return version;
+    }
+
+    public String getPubKey() {
+        return pubKey;
+    }
+
+    public String getUserName() {
+        return userName;
+    }
+
+    public synchronized byte[] getDesKey() {
+        if (desKey == null) {
+            desKey = EncryptUtil.generateDesKey();
+        }
+
+        return desKey;
+    }
+
+    public String getRsaEncryptedKey() {
+        if (rsaEncryptedKey == null) {
+            RSAPublicKey rsaKey = EncryptUtil.loadPublicKeyByText(pubKey);
+            try {
+                byte[] encryptedKey = EncryptUtil.rsaEncrypt(rsaKey, getDesKey());
+                String tmpKey = Base64.encodeBase64String(encryptedKey);
+                rsaEncryptedKey = URLEncoder.encode(tmpKey, "utf8");
+                this.lastUpdateTime.set(System.currentTimeMillis());
+                return rsaEncryptedKey;
+            } catch (Exception e) {
+                logger.error("RSA Encrypt error {}", e);
+                return null;
+            }
+
+        }
+        return rsaEncryptedKey;
+    }
+
+
+    public EncryptInfo getRsaEncryptInfo() {
+        EncryptInfo encryptInfo = null;
+        long visitTime = this.lastUpdateTime.get();
+        if (rsaEncryptedKey != null && (System.currentTimeMillis() - visitTime) <= 3 * 60 * 1000) {
+            encryptInfo = new EncryptInfo(this.version, this.rsaEncryptedKey, this.desKey);
+            if (visitTime == this.lastUpdateTime.get()) {
+                return encryptInfo;
+            }
+            encryptInfo = null;
+        }
+        synchronized (this.lastUpdateTime) {
+            if (visitTime == this.lastUpdateTime.get()) {
+                RSAPublicKey rsaKey = EncryptUtil.loadPublicKeyByText(pubKey);
+                this.desKey = EncryptUtil.generateDesKey();
+                try {
+                    byte[] encryptedKey = EncryptUtil.rsaEncrypt(rsaKey, this.desKey);
+                    String tmpKey = Base64.encodeBase64String(encryptedKey);
+                    rsaEncryptedKey = URLEncoder.encode(tmpKey, "utf8");
+                    this.lastUpdateTime.set(System.currentTimeMillis());
+                    return new EncryptInfo(this.version, this.rsaEncryptedKey, this.desKey);
+                } catch (Throwable e) {
+                    logger.error("getRsaEncryptInfo failure, RSA Encrypt error {}", e);
+                    return null;
+                }
+            }
+        }
+        return new EncryptInfo(this.version, this.rsaEncryptedKey, this.desKey);
+    }
+
+    @Override
+    public boolean equals(Object other) {
+        if (other == null || !(other instanceof EncryptConfigEntry)) {
+            return false;
+        }
+        if (other == this) {
+            return true;
+        }
+        EncryptConfigEntry info = (EncryptConfigEntry) other;
+        return (this.userName.equals(info.getUserName()))
+                && (this.version.equals(info.getVersion()))
+                && (this.pubKey == info.getPubKey());
+    }
+
+    public void setVersion(String version) {
+        this.version = version;
+    }
+
+    public void setPubKey(String pubKey) {
+        this.pubKey = pubKey;
+    }
+
+    public String toString() {
+        return "{\"version\":\"" + version + "\",\"public_key\":\"" + pubKey + "\",\"bid\":\"" + userName + "\"}";
+    }
+
+}
diff --git a/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/config/EncryptInfo.java b/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/config/EncryptInfo.java
new file mode 100644
index 0000000..217ea44
--- /dev/null
+++ b/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/config/EncryptInfo.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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.dataproxy.config;
+
+
+public class EncryptInfo {
+    private String version;
+    private byte[] desKey;
+    private String rsaEncryptedKey;
+
+    public EncryptInfo(String version, String rsaEncryptedKey, byte[] desKey) {
+        this.version = version;
+        this.rsaEncryptedKey = rsaEncryptedKey;
+        this.desKey = desKey;
+    }
+
+    public String getVersion() {
+        return version;
+    }
+
+    public void setVersion(String version) {
+        this.version = version;
+    }
+
+    public byte[] getDesKey() {
+        return desKey;
+    }
+
+    public void setDesKey(byte[] desKey) {
+        this.desKey = desKey;
+    }
+
+    public String getRsaEncryptedKey() {
+        return rsaEncryptedKey;
+    }
+
+    public void setRsaEncryptedKey(String rsaEncryptedKey) {
+        this.rsaEncryptedKey = rsaEncryptedKey;
+    }
+}
diff --git a/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/config/HostInfo.java b/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/config/HostInfo.java
new file mode 100644
index 0000000..b41fa79
--- /dev/null
+++ b/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/config/HostInfo.java
@@ -0,0 +1,67 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.dataproxy.config;
+
+
+public class HostInfo implements Comparable<HostInfo>, java.io.Serializable {
+    private final String referenceName;
+    private final String hostName;
+    private final int portNumber;
+
+    public HostInfo(String referenceName, String hostName, int portNumber) {
+        this.referenceName = referenceName;
+        this.hostName = hostName;
+        this.portNumber = portNumber;
+    }
+
+    public String getReferenceName() {
+        return referenceName;
+    }
+
+    public String getHostName() {
+        return hostName;
+    }
+
+    public int getPortNumber() {
+        return portNumber;
+    }
+
+    @Override
+    public String toString() {
+        return referenceName + "{" + hostName + ":" + portNumber + "}";
+    }
+
+    public int compareTo(HostInfo other) {
+        return referenceName.compareTo(other.getReferenceName());
+    }
+
+    @Override
+    public boolean equals(Object other) {
+        if (other == null || !(other instanceof HostInfo)) {
+            return false;
+        }
+        if (other == this) {
+            return true;
+        }
+        HostInfo info = (HostInfo) other;
+        return (this.referenceName.equals(info.getReferenceName()))
+                && (this.hostName.equals(info.getHostName()))
+                 && (this.portNumber == info.getPortNumber());
+    }
+
+}
diff --git a/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/config/ProxyConfigEntry.java b/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/config/ProxyConfigEntry.java
new file mode 100644
index 0000000..17c399b
--- /dev/null
+++ b/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/config/ProxyConfigEntry.java
@@ -0,0 +1,106 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.dataproxy.config;
+
+import java.util.Map;
+
+public class ProxyConfigEntry implements java.io.Serializable {
+    private String clusterId;
+    private String bid;
+    private int size;
+    private Map<String, HostInfo> hostMap;
+    private int bidNum;
+    private Map<String, Integer> tidNumMap;
+    private int load;
+    private int switchStat;
+    private boolean isInterVisit;
+
+    public int getLoad() {
+        return load;
+    }
+
+    public void setLoad(int load) {
+        this.load = load;
+    }
+
+    public int getBidNum() {
+        return bidNum;
+    }
+
+    public Map<String, Integer> getTidNumMap() {
+        return tidNumMap;
+    }
+
+    public void setBidNumAndTidNumMap(int bidNum, Map<String, Integer> tidNumMap) {
+        this.bidNum = bidNum;
+        this.tidNumMap = tidNumMap;
+    }
+
+    public int getSwitchStat() {
+        return switchStat;
+    }
+
+    public void setSwitchStat(int switchStat) {
+        this.switchStat = switchStat;
+    }
+
+    public Map<String, HostInfo> getHostMap() {
+        return hostMap;
+    }
+
+    public void setHostMap(Map<String, HostInfo> hostMap) {
+        this.size = hostMap.size();
+        this.hostMap = hostMap;
+    }
+
+    public int getSize() {
+        return size;
+    }
+
+    public String getBid() {
+        return bid;
+    }
+
+    public void setBid(String bid) {
+        this.bid = bid;
+    }
+
+    public boolean isInterVisit() {
+        return isInterVisit;
+    }
+
+    public void setInterVisit(boolean interVisit) {
+        isInterVisit = interVisit;
+    }
+
+    @Override
+    public String toString() {
+        return "ProxyConfigEntry [hostMap=" + hostMap + ", load=" + load + ", bsn=" + bidNum + ", tsnMap=" + tidNumMap
+                + ", size=" + size + ", isInterVisit=" + isInterVisit + ", bid=" + bid
+                + ", switch=" + switchStat + "]";
+    }
+
+
+    public String getClusterId() {
+        return clusterId;
+    }
+
+    public void setClusterId(String clusterId) {
+        this.clusterId = clusterId;
+    }
+}
diff --git a/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/config/ProxyConfigManager.java b/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/config/ProxyConfigManager.java
new file mode 100644
index 0000000..a6491b5
--- /dev/null
+++ b/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/config/ProxyConfigManager.java
@@ -0,0 +1,931 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.dataproxy.config;
+
+import static org.apache.inlong.dataproxy.ConfigConstants.REQUEST_HEADER_AUTHORIZATION;
+
+import com.google.gson.Gson;
+import com.google.gson.JsonArray;
+import com.google.gson.JsonObject;
+import com.google.gson.JsonParser;
+import com.google.gson.stream.JsonReader;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.FileReader;
+import java.io.FileWriter;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.io.UnsupportedEncodingException;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.security.KeyManagementException;
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+import java.security.SecureRandom;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import javax.net.ssl.SSLContext;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.http.Header;
+import org.apache.http.HttpResponse;
+import org.apache.http.client.config.RequestConfig;
+import org.apache.http.client.entity.UrlEncodedFormEntity;
+import org.apache.http.client.methods.HttpPost;
+import org.apache.http.conn.ssl.SSLConnectionSocketFactory;
+import org.apache.http.impl.client.CloseableHttpClient;
+import org.apache.http.impl.client.DefaultHttpClient;
+import org.apache.http.impl.client.HttpClients;
+import org.apache.http.message.BasicHeader;
+import org.apache.http.message.BasicNameValuePair;
+import org.apache.http.params.BasicHttpParams;
+import org.apache.http.params.HttpConnectionParams;
+import org.apache.http.params.HttpParams;
+import org.apache.http.ssl.SSLContexts;
+import org.apache.http.util.EntityUtils;
+import org.apache.inlong.dataproxy.ProxyClientConfig;
+import org.apache.inlong.dataproxy.ConfigConstants;
+import org.apache.inlong.dataproxy.network.ClientMgr;
+import org.apache.inlong.dataproxy.network.Utils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class ProxyConfigManager extends Thread {
+    private static final Logger logger = LoggerFactory.getLogger(ProxyConfigManager.class);
+
+    private List<HostInfo> proxyInfoList = new ArrayList<HostInfo>();
+    /*the status of the cluster.if this value is changed,we need rechoose  three proxy*/
+    private int oldStat = 0;
+    private String businessId;
+    private final ProxyClientConfig clientConfig;
+    private final String localIP;
+    private String localMd5;
+    private final ClientMgr clientManager;
+    private boolean bShutDown = false;
+    private long doworkTime = 0;
+    private EncryptConfigEntry userEncryConfigEntry;
+    private final ReentrantReadWriteLock rw = new ReentrantReadWriteLock();
+
+    private final JsonParser jsonParser = new JsonParser();
+    private final Gson gson = new Gson();
+
+    public ProxyConfigManager(final ProxyClientConfig configure, final String localIP, final ClientMgr clientManager) {
+        this.clientConfig = configure;
+        this.localIP = localIP;
+        this.clientManager = clientManager;
+    }
+
+    public String getBusinessId() {
+        return businessId;
+    }
+
+    public void setBusinessId(String businessId) {
+        this.businessId = businessId;
+    }
+
+    public void shutDown() {
+        logger.info("Begin to shut down ProxyConfigManager!");
+        bShutDown = true;
+    }
+
+    @Override
+    public void run() {
+        while (!bShutDown) {
+            try {
+                doProxyEntryQueryWork();
+                updateEncryptConfigEntry();
+                logger.info("ProxyConf update!");
+            } catch (Throwable e) {
+                logger.error("Refresh proxy ip list runs into exception {}, {}", e.toString(), e.getStackTrace());
+                e.printStackTrace();
+            }
+
+            /* Sleep some time.240-360s */
+            try {
+                Random random = new Random();
+                int proxyUpdateIntervalSec = this.clientConfig.getProxyUpdateIntervalMinutes() * 60;
+
+                int sleepTimeSec = proxyUpdateIntervalSec;
+                if (proxyUpdateIntervalSec > 5) {
+                    sleepTimeSec = proxyUpdateIntervalSec + random.nextInt() % (proxyUpdateIntervalSec / 5);
+                }
+                logger.info("sleep time {}", sleepTimeSec);
+                Thread.sleep(sleepTimeSec * 1000);
+            } catch (Throwable e2) {
+                //
+            }
+        }
+        logger.info("ProxyConfigManager worker existed!");
+    }
+
+    /**
+     * try to read cache of proxy entry
+     *
+     * @return
+     */
+    private ProxyConfigEntry tryToReadCacheProxyEntry(String configCachePath) {
+        rw.readLock().lock();
+        try {
+            File file = new File(configCachePath);
+            long diffTime = System.currentTimeMillis() - file.lastModified();
+
+            if (diffTime < clientConfig.getMaxProxyCacheTimeInMs()) {
+                JsonReader reader = new JsonReader(new FileReader(configCachePath));
+                ProxyConfigEntry proxyConfigEntry = gson.fromJson(reader, ProxyConfigEntry.class);
+                logger.info("{} has a backup! {}", businessId, proxyConfigEntry);
+                return proxyConfigEntry;
+            }
+        } catch (Exception ex) {
+            logger.warn("try to read local cache, caught {}", ex.getMessage());
+        } finally {
+            rw.readLock().unlock();
+        }
+        return null;
+    }
+
+    private void tryToWriteCacheProxyEntry(ProxyConfigEntry entry, String configCachePath) {
+        rw.writeLock().lock();
+        try {
+            File file = new File(configCachePath);
+            if (!file.getParentFile().exists()) {
+                // try to create parent
+                file.getParentFile().mkdirs();
+            }
+            logger.info("try to write {}} to local cache {}", entry, configCachePath);
+            FileWriter fileWriter = new FileWriter(configCachePath);
+            gson.toJson(entry, fileWriter);
+            fileWriter.flush();
+            fileWriter.close();
+        } catch (Exception ex) {
+            logger.warn("try to write local cache, caught {}", ex.getMessage());
+        } finally {
+            rw.writeLock().unlock();
+        }
+    }
+
+    private ProxyConfigEntry requestProxyEntryQuietly() {
+        try {
+            return requestProxyList(this.clientConfig.getProxyIPServiceURL());
+        } catch (Exception e) {
+            logger.warn("try to request proxy list by http, caught {}", e.getMessage());
+        }
+        return null;
+    }
+
+    /**
+     *  get bid config
+     *
+     * @return proxyConfigEntry
+     * @throws Exception
+     */
+    public ProxyConfigEntry getBidConfigure() throws Exception {
+        ProxyConfigEntry proxyEntry;
+        String configAddr = clientConfig.getConfStoreBasePath() + businessId;
+        if (this.clientConfig.isReadProxyIPFromLocal()) {
+            configAddr = configAddr + ".local";
+            proxyEntry = getLocalProxyListFromFile(configAddr);
+        } else {
+            configAddr = configAddr + ".proxyip";
+
+            proxyEntry = tryToReadCacheProxyEntry(configAddr);
+            if (proxyEntry == null) {
+                proxyEntry = requestProxyEntryQuietly();
+                int requestCount = 0;
+
+                while (requestCount < 3 && proxyEntry == null) {
+                    proxyEntry = requestProxyEntryQuietly();
+                    requestCount += 1;
+                    if (proxyEntry == null) {
+                        // sleep then retry
+                        TimeUnit.MILLISECONDS.sleep(500);
+                    }
+                }
+            }
+            if (proxyEntry == null) {
+                throw new Exception("Visit manager error, please check log!");
+            } else {
+                tryToWriteCacheProxyEntry(proxyEntry, configAddr);
+            }
+        }
+        return proxyEntry;
+    }
+
+    public void doProxyEntryQueryWork() throws Exception {
+        /* Request the configuration from manager. */
+        if (localMd5 == null) {
+            localMd5 = calcHostInfoMd5(proxyInfoList);
+        }
+        ProxyConfigEntry proxyEntry = null;
+        String configAddr = clientConfig.getConfStoreBasePath() + businessId;
+        if (clientConfig.isReadProxyIPFromLocal()) {
+            configAddr = configAddr + ".local";
+            proxyEntry = getLocalProxyListFromFile(configAddr);
+        } else {
+            /* Do a compare and see if it needs to re-choose the channel. */
+            configAddr = configAddr + ".managerip";
+            int retryCount = 1;
+            while (proxyEntry == null && retryCount < this.clientConfig.getProxyUpdateMaxRetry()) {
+                proxyEntry = requestProxyEntryQuietly();
+                retryCount++;
+                if (proxyEntry == null) {
+                    // sleep then retry.
+                    TimeUnit.SECONDS.sleep(1);
+                }
+            }
+            if (proxyEntry != null) {
+                tryToWriteCacheProxyEntry(proxyEntry, configAddr);
+            }
+            /* We should exit if no local IP list and can't request it from manager.*/
+            if (localMd5 == null && proxyEntry == null) {
+                logger.error("Can't connect manager at the start of proxy API {}",
+                        this.clientConfig.getProxyIPServiceURL());
+                proxyEntry = tryToReadCacheProxyEntry(configAddr);
+            }
+            if (localMd5 != null && proxyEntry == null && proxyInfoList != null) {
+                StringBuffer s = new StringBuffer();
+                for (HostInfo tmp : proxyInfoList) {
+                    s.append(tmp.getHostName()).append(";").append(tmp.getPortNumber())
+                            .append(",");
+                }
+                logger.warn("Backup proxyEntry [{}]", s);
+            }
+        }
+        if (localMd5 == null && proxyEntry == null && proxyInfoList == null) {
+            if (clientConfig.isReadProxyIPFromLocal()) {
+                throw new Exception("Local proxy address configure "
+                        + "read failure, please check first!");
+            } else {
+                throw new Exception("Connect Manager failure, please check first!");
+            }
+        }
+        compareProxyList(proxyEntry);
+    }
+
+    /**
+     * compare proxy list
+     *
+     * @param proxyEntry
+     */
+    private void compareProxyList(ProxyConfigEntry proxyEntry) {
+        if (proxyEntry != null) {
+            logger.info("{}", proxyEntry.toString());
+            if (proxyEntry.getSize() != 0) {
+                /* Initialize the current proxy information list first. */
+                clientManager.setLoadThreshold(proxyEntry.getLoad());
+                clientManager.setBidNum(proxyEntry.getBidNum());
+                clientManager.setTidMap(proxyEntry.getTidNumMap());
+
+                List<HostInfo> newProxyInfoList = new ArrayList<HostInfo>();
+                for (Map.Entry<String, HostInfo> entry : proxyEntry.getHostMap().entrySet()) {
+                    newProxyInfoList.add(entry.getValue());
+                }
+
+                String newMd5 = calcHostInfoMd5(newProxyInfoList);
+                String oldMd5 = calcHostInfoMd5(proxyInfoList);
+                if (newMd5 != null && !newMd5.equals(oldMd5)) {
+                    /* Choose random alive connections to send messages. */
+                    logger.info("old md5 {} new md5 {}", oldMd5, newMd5);
+                    proxyInfoList.clear();
+                    proxyInfoList = newProxyInfoList;
+                    clientManager.setProxyInfoList(proxyInfoList);
+                    doworkTime = System.currentTimeMillis();
+                } else if (proxyEntry.getSwitchStat() != oldStat) {
+                    /*judge  cluster's switch state*/
+                    oldStat = proxyEntry.getSwitchStat();
+                    if ((System.currentTimeMillis() - doworkTime) > 3 * 60 * 1000) {
+                        logger.info("switch the cluster!");
+                        proxyInfoList.clear();
+                        proxyInfoList = newProxyInfoList;
+                        clientManager.setProxyInfoList(proxyInfoList);
+                    } else {
+                        logger.info("only change oldStat ");
+                    }
+                } else {
+                    newProxyInfoList.clear();
+                    logger.info("proxy IP list doesn't change, load {}", proxyEntry.getLoad());
+                }
+            } else {
+                logger.error("proxyEntry's size is zero");
+            }
+        }
+    }
+
+    public EncryptConfigEntry getEncryptConfigEntry(final String userName) {
+        if (Utils.isBlank(userName)) {
+            return null;
+        }
+        EncryptConfigEntry encryptEntry = this.userEncryConfigEntry;
+        if (encryptEntry == null) {
+            int retryCount = 0;
+            encryptEntry = requestPubKey(this.clientConfig.getRsaPubKeyUrl(), userName, false);
+            while (encryptEntry == null && retryCount < this.clientConfig.getProxyUpdateMaxRetry()) {
+                encryptEntry = requestPubKey(this.clientConfig.getRsaPubKeyUrl(), userName, false);
+                retryCount++;
+            }
+            if (encryptEntry == null) {
+                encryptEntry = getStoredPubKeyEntry(userName);
+                if (encryptEntry != null) {
+                    encryptEntry.getRsaEncryptedKey();
+                    synchronized (this) {
+                        if (this.userEncryConfigEntry == null) {
+                            this.userEncryConfigEntry = encryptEntry;
+                        } else {
+                            encryptEntry = this.userEncryConfigEntry;
+                        }
+                    }
+                }
+            } else {
+                synchronized (this) {
+                    if (this.userEncryConfigEntry == null || this.userEncryConfigEntry != encryptEntry) {
+                        storePubKeyEntry(encryptEntry);
+                        encryptEntry.getRsaEncryptedKey();
+                        this.userEncryConfigEntry = encryptEntry;
+                    } else {
+                        encryptEntry = this.userEncryConfigEntry;
+                    }
+                }
+            }
+        }
+        return encryptEntry;
+    }
+
+
+    private void updateEncryptConfigEntry() {
+        if (Utils.isBlank(this.clientConfig.getUserName())) {
+            return;
+        }
+        int retryCount = 0;
+        EncryptConfigEntry encryptConfigEntry = requestPubKey(this.clientConfig.getRsaPubKeyUrl(),
+                this.clientConfig.getUserName(), false);
+        while (encryptConfigEntry == null && retryCount < this.clientConfig.getProxyUpdateMaxRetry()) {
+            encryptConfigEntry = requestPubKey(this.clientConfig.getRsaPubKeyUrl(),
+                    this.clientConfig.getUserName(), false);
+            retryCount++;
+        }
+        if (encryptConfigEntry == null) {
+            return;
+        }
+        synchronized (this) {
+            if (this.userEncryConfigEntry == null || this.userEncryConfigEntry != encryptConfigEntry) {
+                storePubKeyEntry(encryptConfigEntry);
+                encryptConfigEntry.getRsaEncryptedKey();
+                this.userEncryConfigEntry = encryptConfigEntry;
+            }
+        }
+        return;
+    }
+
+    private EncryptConfigEntry getStoredPubKeyEntry(String userName) {
+        if (Utils.isBlank(userName)) {
+            logger.warn(" userName(" + userName + ") is not available");
+            return null;
+        }
+        EncryptConfigEntry entry;
+        FileInputStream fis = null;
+        ObjectInputStream is = null;
+        rw.readLock().lock();
+        try {
+            File file = new File(clientConfig.getConfStoreBasePath() + userName + ".pubKey");
+            if (file.exists()) {
+                fis = new FileInputStream(file);
+                is = new ObjectInputStream(fis);
+                entry = (EncryptConfigEntry) is.readObject();
+                //is.close();
+                fis.close();
+                return entry;
+            } else {
+                return null;
+            }
+        } catch (Throwable e1) {
+            logger.error("Read " + userName + " stored PubKeyEntry error ", e1);
+            return null;
+        } finally {
+            if (fis != null) {
+                try {
+                    fis.close();
+                } catch (Throwable e2) {
+                    //
+                }
+            }
+            rw.readLock().unlock();
+        }
+    }
+
+    private void storePubKeyEntry(EncryptConfigEntry entry) {
+        FileOutputStream fos = null;
+        ObjectOutputStream p = null;
+        rw.writeLock().lock();
+        try {
+            File file = new File(clientConfig.getConfStoreBasePath() + entry.getUserName() + ".pubKey");
+            if (!file.getParentFile().exists()) {
+                file.getParentFile().mkdir();
+            }
+            if (!file.exists()) {
+                file.createNewFile();
+            }
+            fos = new FileOutputStream(file);
+            p = new ObjectOutputStream(fos);
+            p.writeObject(entry);
+            p.flush();
+            //p.close();
+        } catch (Throwable e) {
+            logger.error("store EncryptConfigEntry " + entry.toString() + " exception ", e);
+            e.printStackTrace();
+        } finally {
+            if (fos != null) {
+                try {
+                    fos.close();
+                } catch (Throwable e2) {
+                    //
+                }
+            }
+            rw.writeLock().unlock();
+        }
+    }
+
+    private String calcHostInfoMd5(List<HostInfo> hostInfoList) {
+        if (hostInfoList == null || hostInfoList.isEmpty()) {
+            return null;
+        }
+        Collections.sort(hostInfoList);
+        StringBuffer hostInfoMd5 = new StringBuffer();
+        for (HostInfo hostInfo : hostInfoList) {
+            if (hostInfo == null) {
+                continue;
+            }
+            hostInfoMd5.append(hostInfo.getHostName());
+            hostInfoMd5.append(";");
+            hostInfoMd5.append(hostInfo.getPortNumber());
+            hostInfoMd5.append(";");
+        }
+
+        MessageDigest md = null;
+        try {
+            md = MessageDigest.getInstance("MD5");
+        } catch (NoSuchAlgorithmException e) {
+            logger.error("{}, {}", e.toString(), e.getStackTrace());
+        }
+        try {
+            md.update(hostInfoMd5.toString().getBytes("utf8"), 0, hostInfoMd5.toString().length());
+        } catch (UnsupportedEncodingException e) {
+            e.printStackTrace();
+        }
+        byte[] mdbytes = md.digest();
+        // convert the byte to hex format method 1
+        StringBuffer sb = new StringBuffer();
+        for (int i = 0; i < mdbytes.length; i++) {
+            sb.append(Integer.toString((mdbytes[i] & 0xff) + 0x100, 16).substring(1));
+        }
+        return sb.toString();
+    }
+
+    private EncryptConfigEntry requestPubKey(String pubKeyUrl, String userName, boolean needGet) {
+        if (Utils.isBlank(userName)) {
+            logger.error("Queried userName is null!");
+            return null;
+        }
+        List<BasicNameValuePair> params = new ArrayList<BasicNameValuePair>();
+        params.add(new BasicNameValuePair("operation", "query"));
+        params.add(new BasicNameValuePair("username", userName));
+        JsonObject pubKeyConf = requestConfiguration(pubKeyUrl, params);
+        if (pubKeyConf == null) {
+            logger.info("No public key information returned from manager");
+            return null;
+        }
+        if (!pubKeyConf.has("resultCode")) {
+            logger.info("Parse pubKeyConf failure: No resultCode key information returned from manager");
+            return null;
+        }
+        int resultCode = pubKeyConf.get("resultCode").getAsInt();
+        if (resultCode != 0) {
+            logger.info("query pubKeyConf failure, error code is " + resultCode + ", errInfo is "
+                    + pubKeyConf.get("message").getAsString());
+            return null;
+        }
+        if (!pubKeyConf.has("resultData")) {
+            logger.info("Parse pubKeyConf failure: No resultData key information returned from manager");
+            return null;
+        }
+        JsonObject resultData = pubKeyConf.get("resultData").getAsJsonObject();
+        if (resultData != null) {
+            String publicKey = resultData.get("publicKey").getAsString();
+            if (Utils.isBlank(publicKey)) {
+                return null;
+            }
+            String username = resultData.get("username").getAsString();
+            if (Utils.isBlank(username)) {
+                return null;
+            }
+            String versionStr = resultData.get("version").getAsString();
+            if (Utils.isBlank(versionStr)) {
+                return null;
+            }
+            return new EncryptConfigEntry(username, versionStr, publicKey);
+        }
+        return null;
+    }
+
+    private ProxyConfigEntry getLocalProxyListFromFile(String filePath) throws Exception {
+        JsonObject localProxyAddrJson;
+        try {
+            byte[] fileBytes = Files.readAllBytes(Paths.get(filePath));
+            localProxyAddrJson = jsonParser.parse(new String(fileBytes)).getAsJsonObject();
+        } catch (Throwable e) {
+            throw new Exception("Read local proxyList File failure by " + filePath + ", reason is " + e.getCause());
+        }
+
+
+
+        int bidNum = 0;
+        if (localProxyAddrJson.has("bsn")) {
+            bidNum = localProxyAddrJson.get("bsn").getAsInt();
+        }
+
+        if (!localProxyAddrJson.has("switch")) {
+            throw new Exception("Parse local proxyList failure: switch field is not exist !");
+        }
+        int load = ConfigConstants.LOAD_THRESHOLD;
+        if (localProxyAddrJson.has("load")) {
+            int inLoad = localProxyAddrJson.get("load").getAsInt();
+            load = inLoad > 200 ? 200 : (Math.max(inLoad, 0));
+        }
+        int switchStat = Integer.parseInt(localProxyAddrJson.get("switch").getAsString());
+        ProxyConfigEntry proxyEntry = new ProxyConfigEntry();
+        proxyEntry.setBid(clientConfig.getBid());
+        boolean isInterVisit = checkValidProxy(filePath, localProxyAddrJson);
+        proxyEntry.setInterVisit(isInterVisit);
+        Map<String, HostInfo> hostMap = getHostInfoMap(
+            localProxyAddrJson);
+        proxyEntry.setHostMap(hostMap);
+        proxyEntry.setSwitchStat(switchStat);
+        Map<String, Integer> tidMap = getTidMap(localProxyAddrJson);
+        proxyEntry.setBidNumAndTidNumMap(bidNum, tidMap);
+        proxyEntry.setLoad(load);
+        if (localProxyAddrJson.has("cluster_id")) {
+            proxyEntry.setClusterId(localProxyAddrJson.get("cluster_id").getAsString());
+        }
+        return proxyEntry;
+    }
+
+
+    private Map<String, HostInfo> getHostInfoMap(JsonObject localProxyAddrJson)
+        throws Exception {
+        Map<String, HostInfo> hostMap = new HashMap<String, HostInfo>();
+        JsonArray jsonHostList = localProxyAddrJson.get("address").getAsJsonArray();
+        if (jsonHostList == null) {
+            throw new Exception("Parse local proxyList failure: address field is not exist!");
+        }
+        for (int i = 0; i < jsonHostList.size(); i++) {
+            JsonObject jsonItem = jsonHostList.get(i).getAsJsonObject();
+            if (jsonItem != null) {
+                if (!jsonItem.has("port")) {
+                    throw new Exception("Parse local proxyList failure: "
+                        + "port field is not exist in address(" + i + ")!");
+                }
+                int port = jsonItem.get("port").getAsInt();
+                if (port <= 0) {
+                    throw new Exception("Parse local proxyList failure: "
+                        + "port value <= 0 in address(" + i + ")!");
+                }
+                if (!jsonItem.has("host")) {
+                    throw new Exception("Parse local proxyList failure: "
+                        + "host field is not exist in address(" + i + ")!");
+                }
+                String hostItem = jsonItem.get("host").getAsString();
+                if (Utils.isBlank(hostItem)) {
+                    throw new Exception("Parse local proxyList failure: "
+                        + "host value is blank in address(" + i + ")!");
+                }
+                String refId = hostItem + ":" + String.valueOf(port);
+                hostMap.put(refId, new HostInfo(refId, hostItem, port));
+            }
+        }
+        if (hostMap.isEmpty()) {
+            throw new Exception("Parse local proxyList failure: address is empty !");
+        }
+        return hostMap;
+    }
+
+    private Map<String, Integer> getTidMap(JsonObject localProxyAddrJson) {
+        Map<String, Integer> tidMap = new HashMap<String, Integer>();
+        if (localProxyAddrJson.has("tsn")) {
+            JsonArray jsontid = localProxyAddrJson.getAsJsonArray("tsn");
+            for (int i = 0; i < jsontid.size(); i++) {
+                JsonObject jsonItem = jsontid.get(i).getAsJsonObject();
+                if (jsonItem != null && jsonItem.has("tid") && jsonItem.has("sn")) {
+                    tidMap.put(jsonItem.get("tid").getAsString(), jsonItem.get("sn").getAsInt());
+                }
+            }
+        }
+        return tidMap;
+    }
+
+    private boolean checkValidProxy(String filePath, JsonObject localProxyAddrJson) throws Exception {
+        if (localProxyAddrJson == null) {
+            throw new Exception("Read local proxyList File failure by " + filePath + ", reason is content is null!");
+        }
+        if (!localProxyAddrJson.has("size")) {
+            throw new Exception("Parse local proxyList failure: size field is not exist!");
+        }
+        int size = localProxyAddrJson.get("size").getAsInt();
+        if (size == 0) {
+            throw new Exception("Parse local proxyList failure: proxy list size = 0!");
+        }
+        boolean isInterVisit = false;
+        if (localProxyAddrJson.has("isInterVisit")) {
+            isInterVisit = localProxyAddrJson.get("isInterVisit").getAsInt() != 0;
+        }
+        return isInterVisit;
+    }
+
+
+    public ProxyConfigEntry requestProxyList(String url) {
+        ArrayList<BasicNameValuePair> params = new ArrayList<BasicNameValuePair>();
+        params.add(new BasicNameValuePair("bid", clientConfig.getBid()));
+        params.add(new BasicNameValuePair("net_tag", clientConfig.getNetTag()));
+        params.add(new BasicNameValuePair("ip", this.localIP));
+        params.add(new BasicNameValuePair("sn_query", String.valueOf(true)));
+        logger.info("Begin to get configure from manager {}, param is {}", url, params);
+
+        JsonObject jsonRes = requestConfiguration(url, params);
+        if (jsonRes == null) {
+            return null;
+        }
+        if (!jsonRes.has("size")) {
+            logger.error("Parse proxyList failure: size field is not exist for response from manager!");
+            return null;
+        }
+        int size = jsonRes.get("size").getAsInt();
+        if (size == 0) {
+            logger.info("Parse proxyList failure: proxy list size = 0!");
+            return null;
+        }
+        boolean isInterVisit = false;
+        if (jsonRes.has("isInterVisit")) {
+            isInterVisit = jsonRes.get("isInterVisit").getAsInt() != 0;
+        }
+        Map<String, HostInfo> hostMap = formatHostInfoMap(
+            jsonRes);
+
+        if (hostMap == null) {
+            return null;
+        }
+        size = hostMap.size();
+        int bidNum = 0;
+        if (jsonRes.has("bsn")) {
+            bidNum = jsonRes.get("bsn").getAsInt();
+        }
+        if (!jsonRes.has("switch")) {
+            logger.error("Parse proxyList failure: switch field is not exist for response from manager!");
+            return null;
+        }
+        int load = ConfigConstants.LOAD_THRESHOLD;
+        if (jsonRes.has("load")) {
+            int inLoad = jsonRes.get("load").getAsInt();
+            load = inLoad > 200 ? 200 : (Math.max(inLoad, 0));
+        }
+        int switchStat = Integer.parseInt(jsonRes.get("switch").getAsString());
+        ProxyConfigEntry proxyEntry = new ProxyConfigEntry();
+        proxyEntry.setBid(clientConfig.getBid());
+        proxyEntry.setInterVisit(isInterVisit);
+        proxyEntry.setHostMap(hostMap);
+        proxyEntry.setSwitchStat(switchStat);
+        Map<String, Integer> tidMap = getTidMap(jsonRes);
+        proxyEntry.setBidNumAndTidNumMap(bidNum, tidMap);
+        proxyEntry.setLoad(load);
+        if (jsonRes.has("cluster_id")) {
+            proxyEntry.setClusterId(jsonRes.get("cluster_id").getAsString());
+        }
+
+        return proxyEntry;
+    }
+
+    private Map<String, HostInfo> formatHostInfoMap(JsonObject jsonRes) {
+        Map<String, HostInfo> hostMap = new HashMap<String, HostInfo>();
+        JsonArray jsonHostList = jsonRes.getAsJsonArray("address");
+        if (jsonHostList == null) {
+            logger.info("Parse proxyList failure: address field is not exist for response from manager!");
+            return null;
+        }
+        for (int i = 0; i < jsonHostList.size(); i++) {
+            JsonObject jsonItem = jsonHostList.get(i).getAsJsonObject();
+            if (jsonItem != null) {
+                if (!jsonItem.has("port")) {
+                    logger.error("Parse proxyList failure: port field is not exist in address("
+                            + i + ") for response from manager!");
+                    return null;
+                }
+                int port = jsonItem.get("port").getAsInt();
+                if (port <= 0) {
+                    logger.info("Parse proxyList failure: port value <= 0 in address("
+                            + i + ") for response from manager!");
+                    return null;
+                }
+                if (!jsonItem.has("host")) {
+                    logger.error("Parse proxyList failure: host field is not exist in address("
+                            + i + ") for response from manager!");
+                    return null;
+                }
+                String hostItem = jsonItem.get("host").getAsString();
+                if (Utils.isBlank(hostItem)) {
+                    logger.error("Parse proxyList failure: host value is blank in address("
+                            + i + ") for response from manager!");
+                    return null;
+                }
+                String refId = hostItem + ":" + String.valueOf(port);
+                hostMap.put(refId, new HostInfo(refId, hostItem, port));
+            }
+        }
+        if (hostMap.isEmpty()) {
+            logger.error("Parse proxyList failure: address is empty for response from manager!");
+            return null;
+        }
+        return hostMap;
+    }
+
+    private String updateUrl(String url, int tryIdx, String localManagerIpList) {
+        if (tryIdx == 0) {
+            return url;
+        }
+
+        int headerIdx = url.indexOf("://");
+        if (headerIdx == -1) {
+            return null;
+        }
+        String header = "";
+        header = url.substring(0, headerIdx + 3);
+        String tmpUrl = url.substring(headerIdx + 3);
+        int tailerIdx = tmpUrl.indexOf("/");
+        if (tailerIdx == -1) {
+            return null;
+        }
+        String tailer = "";
+        tailer = tmpUrl.substring(tailerIdx);
+        String[] managerIps = localManagerIpList.split(",");
+        String currentManagerIp = "";
+        int idx = 1;
+        for (String managerIp : managerIps) {
+            if (idx++ == tryIdx) {
+                currentManagerIp = managerIp;
+                break;
+            }
+        }
+        if (!currentManagerIp.equals("")) {
+            return header + currentManagerIp + ":" + clientConfig.getManagerPort() + tailer;
+        }
+        return null;
+    }
+
+    /* Request new configurations from Manager. */
+    private JsonObject requestConfiguration(String url, List<BasicNameValuePair> params) {
+        if (Utils.isBlank(url)) {
+            logger.error("request url is null");
+            return null;
+        }
+        // get local managerIpList
+        String localManagerIps = "";
+        int tryIdx = 0;
+        while (true) {
+            HttpPost httpPost = null;
+            String returnStr = null;
+            HttpParams myParams = new BasicHttpParams();
+            HttpConnectionParams.setConnectionTimeout(myParams, 10000);
+            HttpConnectionParams.setSoTimeout(myParams, clientConfig.getManagerSocketTimeout());
+            CloseableHttpClient httpClient = null;
+            if (this.clientConfig.isLocalVisit()) {
+                httpClient = new DefaultHttpClient(myParams);
+            } else {
+                try {
+                    httpClient = getCloseableHttpClient(params);
+                } catch (Throwable eHttps) {
+                    logger.error("Create Https cliet failure, error 1 is ", eHttps);
+                    eHttps.printStackTrace();
+                    return null;
+                }
+            }
+
+            if (!clientConfig.isEnableSaveManagerVIps() && tryIdx > 0) {
+                return null;
+            }
+            // change url's manager host port when occur error
+            url = updateUrl(url, tryIdx, localManagerIps);
+            if (url == null) {
+                return null;
+            }
+            tryIdx++;
+
+            logger.info("Request url : " + url + ", localManagerIps : " + localManagerIps);
+            try {
+                httpPost = new HttpPost(url);
+                if (this.clientConfig.isNeedAuthentication()) {
+                    long timestamp = System.currentTimeMillis();
+                    int nonce = new SecureRandom(String.valueOf(timestamp).getBytes()).nextInt(Integer.MAX_VALUE);
+                    httpPost.setHeader(REQUEST_HEADER_AUTHORIZATION,
+                            Utils.getAuthorizenInfo(clientConfig.getUserName(),
+                                    clientConfig.getSecretKey(), timestamp, nonce));
+                }
+                UrlEncodedFormEntity se = new UrlEncodedFormEntity(params);
+                httpPost.setEntity(se);
+                HttpResponse response = httpClient.execute(httpPost);
+                returnStr = EntityUtils.toString(response.getEntity());
+                if (Utils.isNotBlank(returnStr) && response.getStatusLine().getStatusCode() == 200) {
+                    logger.info("Get configure from manager is " + returnStr);
+                    JsonObject jsonRes = jsonParser.parse(returnStr).getAsJsonObject();
+                    return jsonRes;
+                }
+
+                if (!clientConfig.isLocalVisit()) {
+                    return null;
+                }
+            } catch (Throwable e) {
+                logger.error("Connect Manager error, message: {}, url is {}", e.getMessage(), url);
+
+                if (!clientConfig.isLocalVisit()) {
+                    return null;
+                }
+                // get localManagerIps
+                localManagerIps = getLocalManagerIps();
+                if (localManagerIps == null) {
+                    return null;
+                }
+            } finally {
+                if (httpPost != null) {
+                    httpPost.releaseConnection();
+                }
+                if (httpClient != null) {
+                    httpClient.getConnectionManager().shutdown();
+                }
+            }
+        }
+    }
+
+    private CloseableHttpClient getCloseableHttpClient(List<BasicNameValuePair> params)
+        throws NoSuchAlgorithmException, KeyManagementException {
+        CloseableHttpClient httpClient;
+        ArrayList<Header> headers = new ArrayList<Header>();
+        for (BasicNameValuePair paramItem : params) {
+            headers.add(new BasicHeader(paramItem.getName(), paramItem.getValue()));
+        }
+        RequestConfig requestConfig = RequestConfig.custom().setConnectTimeout(10000)
+                .setSocketTimeout(clientConfig.getManagerSocketTimeout()).build();
+        SSLContext sslContext = SSLContexts.custom().build();
+        SSLConnectionSocketFactory sslsf = new SSLConnectionSocketFactory(sslContext,
+                new String[]{"TLSv1"}, null,
+                SSLConnectionSocketFactory.getDefaultHostnameVerifier());
+        httpClient = HttpClients.custom().setDefaultHeaders(headers).setDefaultRequestConfig(requestConfig)
+                .setSSLSocketFactory(sslsf).build();
+        return httpClient;
+    }
+
+    private String getLocalManagerIps() {
+        String localManagerIps;
+        try {
+            File localManagerIpsFile = new File(clientConfig.getManagerIpLocalPath());
+            if (localManagerIpsFile.exists()) {
+                byte[] serialized;
+                serialized = FileUtils.readFileToByteArray(localManagerIpsFile);
+                if (serialized == null) {
+                    logger.error("Local managerIp file is empty, file path : "
+                                + clientConfig.getManagerIpLocalPath());
+                    return null;
+                }
+                localManagerIps = new String(serialized, "UTF-8");
+            } else {
+                if (!localManagerIpsFile.getParentFile().exists()) {
+                    localManagerIpsFile.getParentFile().mkdirs();
+                }
+                localManagerIps = "";
+                logger.error("Get local managerIpList not exist, file path : "
+                            + clientConfig.getManagerIpLocalPath());
+            }
+        } catch (Throwable t) {
+            localManagerIps = "";
+            logger.error("Get local managerIpList occur exception,", t);
+        }
+        return localManagerIps;
+    }
+}
diff --git a/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/demo/Event.java b/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/demo/Event.java
new file mode 100644
index 0000000..8cd6669
--- /dev/null
+++ b/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/demo/Event.java
@@ -0,0 +1,96 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.dataproxy.demo;
+
+import java.util.ArrayList;
+
+public class Event {
+    private byte[] body;
+    private String bid;
+    private String tid;
+    private long dt;
+    private int tryTimes = 0;
+    ArrayList<byte[]> bodylist = new ArrayList<byte[]>();
+
+    public Event(byte[] body, String bid, String tid, long dt) {
+        super();
+        this.body = body;
+        this.bid = bid;
+        this.tid = tid;
+        this.dt = dt;
+        this.setTryTimes(0);
+    }
+
+    public Event(ArrayList<byte[]> bodylist, String bid, String tid, long dt) {
+        super();
+        this.bodylist = bodylist;
+        this.bid = bid;
+        this.tid = tid;
+        this.dt = dt;
+        this.setTryTimes(0);
+    }
+
+    public ArrayList<byte[]> getBodylist() {
+        return bodylist;
+    }
+
+    public void setBodylist(ArrayList<byte[]> bodylist) {
+        this.bodylist = bodylist;
+    }
+
+    public byte[] getBody() {
+        return body;
+    }
+
+    public void setBody(byte[] body) {
+        this.body = body;
+    }
+
+    public String getBid() {
+        return bid;
+    }
+
+    public void setBid(String bid) {
+        this.bid = bid;
+    }
+
+    public String getTid() {
+        return tid;
+    }
+
+    public void setTid(String tid) {
+        this.tid = tid;
+    }
+
+    public long getDt() {
+        return dt;
+    }
+
+    public void setDt(long dt) {
+        this.dt = dt;
+    }
+
+    public int getTryTimes() {
+        return tryTimes;
+    }
+
+    public void setTryTimes(int tryTimes) {
+        this.tryTimes = tryTimes;
+    }
+
+}
diff --git a/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/demo/MyFileCallBack.java b/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/demo/MyFileCallBack.java
new file mode 100644
index 0000000..363ce1b
--- /dev/null
+++ b/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/demo/MyFileCallBack.java
@@ -0,0 +1,59 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.dataproxy.demo;
+
+import org.apache.inlong.dataproxy.DefaultMessageSender;
+import org.apache.inlong.dataproxy.FileCallback;
+import org.apache.inlong.dataproxy.SendResult;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class MyFileCallBack extends FileCallback {
+    private static final Logger logger = LoggerFactory
+            .getLogger(MyFileCallBack.class);
+    private DefaultMessageSender messageSender = null;
+    private Event event = null;
+
+    public MyFileCallBack() {
+
+    }
+
+    public MyFileCallBack(DefaultMessageSender messageSender, Event event) {
+        super();
+        this.messageSender = messageSender;
+        this.event = event;
+    }
+
+    public void onMessageAck(String result) {
+        logger.info("onMessageAck return result = {}", result);
+    }
+
+    public void onMessageAck(SendResult result) {
+        if (result == SendResult.OK) {
+            logger.info("onMessageAck return Ok");
+        } else {
+            logger.info("onMessageAck return failure = {}", result);
+        }
+    }
+
+    public void onException(Throwable e) {
+        logger.error("Send message failure, error {}", e.getMessage());
+        e.printStackTrace();
+    }
+
+}
diff --git a/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/demo/MyMessageCallBack.java b/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/demo/MyMessageCallBack.java
new file mode 100644
index 0000000..21d5b41
--- /dev/null
+++ b/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/demo/MyMessageCallBack.java
@@ -0,0 +1,60 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.dataproxy.demo;
+
+import org.apache.inlong.dataproxy.DefaultMessageSender;
+import org.apache.inlong.dataproxy.FileCallback;
+import org.apache.inlong.dataproxy.SendResult;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class MyMessageCallBack extends FileCallback {
+    private static final Logger logger = LoggerFactory
+            .getLogger(MyMessageCallBack.class);
+    private DefaultMessageSender messageSender = null;
+    private Event event = null;
+
+    public MyMessageCallBack() {
+
+    }
+
+    public MyMessageCallBack(DefaultMessageSender messageSender, Event event) {
+        super();
+        this.messageSender = messageSender;
+        this.event = event;
+    }
+
+    public void onMessageAck(String result) {
+        logger.info("onMessageAck return result = {}", result);
+    }
+
+
+    public void onMessageAck(SendResult result) {
+        if (result == SendResult.OK) {
+            logger.info("onMessageAck return Ok");
+        } else {
+            logger.info("onMessageAck return failure = {}", result);
+        }
+    }
+
+    public void onException(Throwable e) {
+        logger.error("Send message failure, error {}", e.getMessage());
+        e.printStackTrace();
+    }
+
+}
diff --git a/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/demo/SendMsgThread.java b/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/demo/SendMsgThread.java
new file mode 100644
index 0000000..8024fe7
--- /dev/null
+++ b/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/demo/SendMsgThread.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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.dataproxy.demo;
+
+import java.io.BufferedReader;
+import java.io.FileReader;
+import java.io.IOException;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.inlong.dataproxy.DefaultMessageSender;
+import org.apache.inlong.dataproxy.SendResult;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SendMsgThread extends Thread {
+    private static final Logger logger = LoggerFactory.getLogger(SendMsgThread.class);
+    private DefaultMessageSender messageSender = null;
+
+    public SendMsgThread(DefaultMessageSender messageSender) {
+        this.messageSender = messageSender;
+    }
+
+    @Override
+    public void run() {
+        FileReader reader = null;
+        try {
+            reader = new FileReader("/data/work/jessey/d5.txt");
+
+            BufferedReader br = new BufferedReader(reader);
+            String line = null;
+            while ((line = br.readLine()) != null) {
+
+                long startTime = System.currentTimeMillis();
+                SendResult result = messageSender.sendMessage("hhhh".getBytes("utf8"),
+                        "b_test", "n_test1", 0, String.valueOf(System.currentTimeMillis()), 1,
+                        TimeUnit.MILLISECONDS);
+                long endTime = System.currentTimeMillis();
+                if (result == result.OK) {
+                    logger.info("this msg is ok time {}", endTime - startTime);
+                } else {
+                    logger.info("this msg is error ,{}", result);
+                }
+            }
+        } catch (Exception e) {
+            logger.error("{}", e.getMessage());
+            e.printStackTrace();
+
+        } finally {
+            if (reader != null) {
+                try {
+                    reader.close();
+                } catch (IOException e) {
+                    e.printStackTrace();
+                }
+            }
+        }
+
+    }
+}
diff --git a/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/http/InternalHttpSender.java b/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/http/InternalHttpSender.java
new file mode 100644
index 0000000..982f698
--- /dev/null
+++ b/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/http/InternalHttpSender.java
@@ -0,0 +1,278 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.dataproxy.http;
+
+import com.google.gson.JsonObject;
+import com.google.gson.JsonParser;
+
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.http.HttpHeaders;
+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.client.utils.URLEncodedUtils;
+import org.apache.http.entity.StringEntity;
+import org.apache.http.impl.client.CloseableHttpClient;
+import org.apache.http.impl.client.HttpClientBuilder;
+import org.apache.http.message.BasicNameValuePair;
+import org.apache.http.util.EntityUtils;
+import org.apache.inlong.dataproxy.ProxyClientConfig;
+import org.apache.inlong.dataproxy.SendResult;
+import org.apache.inlong.dataproxy.config.HostInfo;
+import org.apache.inlong.dataproxy.network.HttpMessage;
+import org.apache.inlong.dataproxy.network.Utils;
+import org.apache.inlong.dataproxy.utils.ConcurrentHashSet;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * internal http sender
+ */
+public class InternalHttpSender {
+    private static final Logger logger = LoggerFactory.getLogger(InternalHttpSender.class);
+
+    private final ProxyClientConfig proxyClientConfig;
+    private final ConcurrentHashSet<HostInfo> hostList;
+
+    private static final int DEFAULT_PORT = 8000;
+
+    private final LinkedBlockingQueue<HttpMessage> messageCache;
+    private final ExecutorService workerServices = Executors
+            .newCachedThreadPool();
+    private CloseableHttpClient httpClient;
+    private final JsonParser jsonParser = new JsonParser();
+    private boolean bShutDown = false;
+
+    public InternalHttpSender(ProxyClientConfig proxyClientConfig,
+                              ConcurrentHashSet<HostInfo> hostList,
+                              LinkedBlockingQueue<HttpMessage> messageCache) {
+        this.proxyClientConfig = proxyClientConfig;
+        this.hostList = hostList;
+        this.messageCache = messageCache;
+        submitWorkThread();
+    }
+
+    private void submitWorkThread() {
+        for (int i = 0; i < proxyClientConfig.getAsyncWorkerNumber(); i++) {
+            workerServices.execute(new WorkerRunner());
+        }
+    }
+
+    /**
+     * construct header
+     *
+     * @param bodies
+     * @param bid
+     * @param tid
+     * @param dt
+     * @return
+     */
+    private ArrayList<BasicNameValuePair> getHeaders(List<String> bodies,
+                                                     String bid, String tid, long dt) {
+        ArrayList<BasicNameValuePair> params = new ArrayList<BasicNameValuePair>();
+        params.add(new BasicNameValuePair("bid", bid));
+        params.add(new BasicNameValuePair("tid", tid));
+        params.add(new BasicNameValuePair("dt", String.valueOf(dt)));
+        params.add(new BasicNameValuePair("body", StringUtils.join(bodies, "\n")));
+        params.add(new BasicNameValuePair("cnt", String.valueOf(bodies.size())));
+
+        return params;
+    }
+
+    /**
+     * http client
+     *
+     * @param timeout
+     * @param timeUnit
+     * @return
+     */
+    private synchronized CloseableHttpClient constructHttpClient(long timeout, TimeUnit timeUnit) {
+        if (httpClient != null) {
+            return httpClient;
+        }
+        long timeoutInMs = timeUnit.toMillis(timeout);
+        RequestConfig requestConfig = RequestConfig.custom()
+                .setConnectTimeout((int) timeoutInMs)
+                .setSocketTimeout((int) timeoutInMs).build();
+        HttpClientBuilder httpClientBuilder = HttpClientBuilder.create();
+        httpClientBuilder.setDefaultRequestConfig(requestConfig);
+        return httpClientBuilder.build();
+    }
+
+    /**
+     * check cache runner
+     */
+    private class WorkerRunner implements Runnable {
+        @Override
+        public void run() {
+            // if not shutdown or queue is not empty
+            while (!bShutDown || !messageCache.isEmpty()) {
+                try {
+                    while (!messageCache.isEmpty()) {
+                        HttpMessage httpMessage = messageCache.poll();
+                        if (httpMessage != null) {
+                            SendResult result = sendMessageWithHostInfo(
+                                    httpMessage.getBodies(), httpMessage.getBid(),
+                                    httpMessage.getTid(), httpMessage.getDt(),
+                                    httpMessage.getTimeout(), httpMessage.getTimeUnit());
+                            httpMessage.getCallback().onMessageAck(result);
+                        }
+                    }
+                    TimeUnit.MILLISECONDS.sleep(proxyClientConfig.getAsyncWorkerInterval());
+                } catch (Exception exception) {
+                    logger.error("exception caught", exception);
+                }
+            }
+        }
+    }
+
+    /**
+     * get random ip
+     *
+     * @return list of host info
+     */
+    public List<HostInfo> getRandomHostInfo() {
+        List<HostInfo> tmpHostList = new ArrayList<>(hostList);
+        Collections.shuffle(tmpHostList);
+        // respect alive connection
+        int maxIndex = Math.min(proxyClientConfig.getAliveConnections(), tmpHostList.size());
+        return tmpHostList.subList(0, maxIndex);
+    }
+
+    /**
+     * send request by http
+     *
+     * @param bodies
+     * @param bid
+     * @param tid
+     * @param dt
+     * @param timeout
+     * @param timeUnit
+     * @param hostInfo
+     * @return
+     * @throws Exception
+     */
+    private SendResult sendByHttp(List<String> bodies, String bid, String tid, long dt,
+                                  long timeout, TimeUnit timeUnit, HostInfo hostInfo) throws Exception {
+        HttpPost httpPost = null;
+        CloseableHttpResponse response = null;
+        try {
+            if (httpClient == null) {
+                httpClient = constructHttpClient(timeout, timeUnit);
+            }
+
+
+            String url = "http://" + hostInfo.getHostName() + ":" + DEFAULT_PORT + "/manager/message";
+
+
+            httpPost = new HttpPost(url);
+            httpPost.setHeader(HttpHeaders.CONNECTION, "close");
+            httpPost.setHeader(HttpHeaders.CONTENT_TYPE, "application/x-www-form-urlencoded");
+            ArrayList<BasicNameValuePair> contents = getHeaders(bodies, bid, tid, dt);
+            String s = URLEncodedUtils.format(contents, StandardCharsets.UTF_8);
+            logger.info("encode string is {}", s);
+            httpPost.setEntity(new StringEntity(s));
+
+            response = httpClient.execute(httpPost);
+            String returnStr = EntityUtils.toString(response.getEntity());
+
+            if (Utils.isNotBlank(returnStr) && response.getStatusLine().getStatusCode() == 200) {
+                logger.debug("Get configure from manager is " + returnStr);
+                JsonObject jsonRes = jsonParser.parse(returnStr).getAsJsonObject();
+
+                if (jsonRes.has("code")) {
+                    int code = jsonRes.get("code").getAsInt();
+                    if (code == 1) {
+                        return SendResult.OK;
+                    } else {
+                        logger.debug("get error response {}", returnStr);
+                        return SendResult.INVALID_DATA;
+                    }
+                }
+
+            } else {
+                throw new Exception("exception to get response from request " + returnStr + " "
+                        + response.getStatusLine().getStatusCode());
+            }
+
+        } finally {
+            if (httpPost != null) {
+                httpPost.releaseConnection();
+            }
+            if (response != null) {
+                response.close();
+            }
+        }
+        return SendResult.UNKOWN_ERROR;
+    }
+
+    /**
+     * send message with host info
+     *
+     * @param bodies
+     * @param bid
+     * @param tid
+     * @param dt
+     * @param timeout
+     * @param timeUnit
+     * @return
+     */
+    public SendResult sendMessageWithHostInfo(List<String> bodies, String bid, String tid, long dt,
+                                              long timeout, TimeUnit timeUnit) {
+
+        List<HostInfo> randomHostList = getRandomHostInfo();
+        Exception tmpException = null;
+        for (HostInfo hostInfo : randomHostList) {
+            try {
+                return sendByHttp(bodies, bid, tid, dt, timeout, timeUnit, hostInfo);
+            } catch (Exception exception) {
+                tmpException = exception;
+                logger.debug("error while sending data, resending it", exception);
+            }
+        }
+        if (tmpException != null) {
+            logger.error("error while sending data", tmpException);
+        }
+        return SendResult.UNKOWN_ERROR;
+    }
+
+    /**
+     * close
+     *
+     * @throws Exception
+     */
+    public void close() throws Exception {
+        bShutDown = true;
+        if (proxyClientConfig.isCleanHttpCacheWhenClosing()) {
+            messageCache.clear();
+        }
+        if (httpClient != null) {
+            httpClient.close();
+        }
+        workerServices.shutdown();
+    }
+}
diff --git a/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/metric/MessageRecord.java b/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/metric/MessageRecord.java
new file mode 100644
index 0000000..b82842b
--- /dev/null
+++ b/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/metric/MessageRecord.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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.dataproxy.metric;
+
+/**
+ * message uuid
+ */
+public class MessageRecord {
+    private final String msgUUID;
+    private final int msgCount;
+    private final long startTime;
+    private final long dt;
+
+    private final String bid;
+    private final String tid;
+    private final String localIp;
+    private final long packTime;
+
+    public MessageRecord(String bid, String tid, String localIp, String msgId, long dt, long packTime, int msgCount) {
+        this.bid = bid;
+        this.tid = tid;
+        this.localIp = localIp;
+        this.msgUUID = msgId;
+        this.msgCount = msgCount;
+        this.packTime = packTime;
+        this.dt = dt;
+        startTime = System.currentTimeMillis();
+    }
+
+    public String getMsgUUID() {
+        return msgUUID;
+    }
+
+    public int getMsgCount() {
+        return msgCount;
+    }
+
+    public long getMessageTime() {
+        return System.currentTimeMillis() - startTime;
+    }
+
+    public long getPackTime() {
+        return packTime;
+    }
+
+    public long getDt() {
+        return dt;
+    }
+
+    public String getBid() {
+        return bid;
+    }
+
+    public String getTid() {
+        return tid;
+    }
+
+    public String getLocalIp() {
+        return localIp;
+    }
+}
+
diff --git a/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/metric/MetricTimeNumSummary.java b/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/metric/MetricTimeNumSummary.java
new file mode 100644
index 0000000..d4c3a2f
--- /dev/null
+++ b/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/metric/MetricTimeNumSummary.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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.dataproxy.metric;
+
+import java.util.concurrent.atomic.AtomicLong;
+
+public class MetricTimeNumSummary {
+
+    private final AtomicLong send10msBelow = new AtomicLong(0);
+
+    private final AtomicLong sendBetween10msAnd100ms = new AtomicLong(0);
+
+    private final AtomicLong sendBetween100msAnd500ms = new AtomicLong(0);
+
+    private final AtomicLong sendBetween500msAnd1s = new AtomicLong(0);
+
+    private final AtomicLong send1sAbove = new AtomicLong(0);
+
+    private final AtomicLong successNum = new AtomicLong(0);
+    private final AtomicLong failedNum = new AtomicLong(0);
+
+    private final long startCalculateTime;
+
+    public MetricTimeNumSummary(long startCalculateTime) {
+        this.startCalculateTime = startCalculateTime;
+    }
+
+    /**
+     * get summary time
+     *
+     * @return
+     */
+    public long getSummaryTime() {
+        return System.currentTimeMillis() - this.startCalculateTime;
+    }
+
+    /**
+     * record time
+     * 1. [-, 10)
+     * 2. [10, 100)
+     * 3. [100, 500)
+     * 4. [500, 1000)
+     * 5. [100, -)
+     *
+     * @param sendTimeInMs - send time
+     * @param sendNum      - send num
+     */
+    public void recordSuccessSendTime(long sendTimeInMs, int sendNum) {
+        if (sendTimeInMs < 10) {
+            send10msBelow.addAndGet(sendNum);
+        } else if (sendTimeInMs < 100) {
+            sendBetween10msAnd100ms.addAndGet(sendNum);
+        } else if (sendTimeInMs < 500) {
+            sendBetween100msAnd500ms.addAndGet(sendNum);
+        } else if (sendTimeInMs < 1000) {
+            sendBetween500msAnd1s.addAndGet(sendNum);
+        } else {
+            send1sAbove.addAndGet(sendNum);
+        }
+        increaseSuccessNum(sendNum);
+    }
+
+    public void increaseSuccessNum(int sendNum) {
+        successNum.addAndGet(sendNum);
+    }
+
+    public void increaseFailedNum(int sendNum) {
+        failedNum.addAndGet(sendNum);
+    }
+
+    public long getFailedNum() {
+        return failedNum.get();
+    }
+
+    public long getSuccessNum() {
+        return successNum.get();
+    }
+
+    public long getTotalNum() {
+        return failedNum.get() + successNum.get();
+    }
+
+    public String getTimeString() {
+        return send10msBelow.get() + "#" + sendBetween10msAnd100ms.get()
+                + "#" + sendBetween100msAnd500ms.get() + "#"
+                + sendBetween500msAnd1s.get() + "#" + send1sAbove.get();
+    }
+
+    public long getStartCalculateTime() {
+        return startCalculateTime;
+    }
+}
diff --git a/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/network/ClientHandler.java b/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/network/ClientHandler.java
new file mode 100644
index 0000000..5409f15
--- /dev/null
+++ b/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/network/ClientHandler.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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.dataproxy.network;
+
+import org.apache.inlong.dataproxy.codec.EncodeObject;
+import org.jboss.netty.channel.ChannelHandlerContext;
+import org.jboss.netty.channel.ChannelStateEvent;
+import org.jboss.netty.channel.ExceptionEvent;
+import org.jboss.netty.channel.MessageEvent;
+import org.jboss.netty.handler.timeout.IdleStateAwareChannelHandler;
+import org.jboss.netty.handler.timeout.IdleStateEvent;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ClientHandler extends IdleStateAwareChannelHandler {
+    private static final Logger logger = LoggerFactory
+            .getLogger(ClientHandler.class);
+
+    private final Sender sender;
+    private final ClientMgr clientMgr;
+
+    public ClientHandler(Sender sender, ClientMgr clientMgr) {
+        this.sender = sender;
+        this.clientMgr = clientMgr;
+    }
+
+    @Override
+    public void channelIdle(ChannelHandlerContext ctx, IdleStateEvent e) throws Exception {
+
+    }
+
+    @Override
+    public void messageReceived(ChannelHandlerContext ctx, MessageEvent e) {
+        try {
+            EncodeObject encodeObject = (EncodeObject) e.getMessage();
+            if (encodeObject.getMsgtype() != 8) {
+                sender.notifyFeedback(e.getChannel(), encodeObject);
+            } else {
+                clientMgr.notifyHBAck(e.getChannel(), encodeObject.getLoad());
+            }
+        } catch (Exception ex) {
+            logger.error("error :", ex);
+        }
+    }
+
+    @Override
+    public void exceptionCaught(ChannelHandlerContext ctx, ExceptionEvent e) {
+        logger.error("this channel {} has error! , reason is {} ", e.getChannel(), e.getCause());
+        try {
+            clientMgr.setConnectionFrozen(e.getChannel());
+        } catch (Exception e1) {
+            logger.error("exceptionCaught error :", e1);
+        }
+    }
+
+    @Override
+    public void channelDisconnected(ChannelHandlerContext ctx,
+                                    ChannelStateEvent e) {
+        // clientMgr.resetClient(e.getChannel());
+        logger.info("ClientHandler channelDisconnected {}", e.getChannel());
+        try {
+            sender.notifyConnectionDisconnected(e.getChannel());
+        } catch (Exception e1) {
+            logger.error("exceptionCaught error {}", e1.getMessage());
+        }
+    }
+
+    @Override
+    public void channelClosed(ChannelHandlerContext ctx, ChannelStateEvent e) {
+        // clientMgr.resetClient(e.getChannel());
+        logger.info("ClientHandler channelClosed {}", e.getChannel());
+        try {
+            sender.notifyConnectionDisconnected(e.getChannel());
+        } catch (Exception e1) {
+            logger.error("exceptionCaught error ", e1);
+        }
+    }
+}
diff --git a/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/network/ClientMgr.java b/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/network/ClientMgr.java
new file mode 100644
index 0000000..163d23e
--- /dev/null
+++ b/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/network/ClientMgr.java
@@ -0,0 +1,907 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.dataproxy.network;
+
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+import org.apache.inlong.dataproxy.ProxyClientConfig;
+import org.apache.inlong.dataproxy.ConfigConstants;
+import org.apache.inlong.dataproxy.codec.EncodeObject;
+import org.apache.inlong.dataproxy.config.ProxyConfigEntry;
+import org.apache.inlong.dataproxy.config.ProxyConfigManager;
+import org.apache.inlong.dataproxy.config.EncryptConfigEntry;
+import org.apache.inlong.dataproxy.config.HostInfo;
+import org.jboss.netty.bootstrap.ClientBootstrap;
+import org.jboss.netty.channel.Channel;
+import org.jboss.netty.channel.ChannelFactory;
+import org.jboss.netty.channel.socket.nio.NioClientSocketChannelFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ClientMgr {
+    private static final Logger logger = LoggerFactory
+            .getLogger(ClientMgr.class);
+
+
+    private final Map<HostInfo, NettyClient> clientMapData = new ConcurrentHashMap<HostInfo, NettyClient>();
+
+    private final ConcurrentHashMap<HostInfo, NettyClient> clientMapHB = new ConcurrentHashMap<HostInfo, NettyClient>();
+    // clientMapData + clientMapHB = clientMap
+    private final ConcurrentHashMap<HostInfo, NettyClient> clientMap = new ConcurrentHashMap<HostInfo, NettyClient>();
+
+    private final ConcurrentHashMap<HostInfo, AtomicLong> lastBadHostMap = new ConcurrentHashMap<>();
+
+    private final ArrayList<NettyClient> clientList = new ArrayList<NettyClient>();
+    private List<HostInfo> proxyInfoList = new ArrayList<HostInfo>();
+
+    //    private final Map<HostInfo, Integer> channelLoadMap = new ConcurrentHashMap<HostInfo, Integer>();
+    private final Map<HostInfo, int[]> channelLoadMapData = new ConcurrentHashMap<HostInfo, int[]>();
+    private final Map<HostInfo, int[]> channelLoadMapHB = new ConcurrentHashMap<HostInfo, int[]>();
+
+
+    private ClientBootstrap bootstrap;
+    private int currentIndex = 0;
+    private ProxyClientConfig configure;
+    private Sender sender;
+    private int aliveConnections;
+    private int realSize;
+    //    private ConnectionCheckThread connectionCheckThread;
+    private SendHBThread sendHBThread;
+    private ProxyConfigManager ipManager;
+
+    private int bidNum = 0;
+    private String bid = "";
+    private Map<String, Integer> tidMap = new HashMap<String, Integer>();
+    private int loadThreshold;
+    private int loadCycle = 0;
+    private static final int[] weight = {
+            1, 1, 1, 1, 1,
+            2, 2, 2, 2, 2,
+            3, 3, 3, 3, 3,
+            6, 6, 6, 6, 6,
+            12, 12, 12, 12, 12,
+            48, 96, 192, 384, 1000};
+//    private static final int total_weight = 240;
+    /**
+     * Lock to protect FSNamesystem.
+     */
+    private final ReentrantReadWriteLock fsLock = new ReentrantReadWriteLock(true);
+
+    public int getLoadThreshold() {
+        return loadThreshold;
+    }
+
+    public void setLoadThreshold(int loadThreshold) {
+        this.loadThreshold = loadThreshold;
+    }
+
+    public int getBidNum() {
+        return bidNum;
+    }
+
+    public void setBidNum(int bidNum) {
+        this.bidNum = bidNum;
+    }
+
+    public String getBid() {
+        return bid;
+    }
+
+    public void setBid(String bid) {
+        this.bid = bid;
+    }
+
+    public Map<String, Integer> getTidMap() {
+        return tidMap;
+    }
+
+    public void setTidMap(Map<String, Integer> tidMap) {
+        this.tidMap = tidMap;
+    }
+
+    public EncryptConfigEntry getEncryptConfigEntry() {
+        return this.ipManager.getEncryptConfigEntry(configure.getUserName());
+    }
+
+    public List<HostInfo> getProxyInfoList() {
+        return proxyInfoList;
+    }
+
+    public int getAliveConnections() {
+        return aliveConnections;
+    }
+
+    public void setAliveConnections(int aliveConnections) {
+        this.aliveConnections = aliveConnections;
+    }
+
+    public void readLock() {
+        this.fsLock.readLock().lock();
+    }
+
+    public void readUnlock() {
+        this.fsLock.readLock().unlock();
+    }
+
+    public void writeLock() {
+        this.fsLock.writeLock().lock();
+    }
+
+    public void writeLockInterruptibly() throws InterruptedException {
+        this.fsLock.writeLock().lockInterruptibly();
+    }
+
+    public void writeUnlock() {
+        this.fsLock.writeLock().unlock();
+    }
+
+    public boolean hasWriteLock() {
+        return this.fsLock.isWriteLockedByCurrentThread();
+    }
+
+    public boolean hasReadLock() {
+        return this.fsLock.getReadHoldCount() > 0;
+    }
+
+    public boolean hasReadOrWriteLock() {
+        return hasReadLock() || hasWriteLock();
+    }
+
+    public ClientMgr(ProxyClientConfig configure, Sender sender) throws Exception {
+        this(configure, sender, null);
+    }
+
+    // Build up the connection between the server and client.
+    public ClientMgr(ProxyClientConfig configure, Sender sender, ChannelFactory selfDefineFactory) throws Exception {
+        /* Initialize the bootstrap. */
+        if (selfDefineFactory == null) {
+            selfDefineFactory = new NioClientSocketChannelFactory(Executors.newCachedThreadPool(),
+                    Executors.newCachedThreadPool());
+        }
+        bootstrap = new ClientBootstrap(selfDefineFactory);
+        bootstrap.setPipelineFactory(new ClientPipelineFactory(this, sender));
+        bootstrap.setOption(ConfigConstants.RECEIVE_BUFFER_SIZE, ConfigConstants.DEFAULT_RECEIVE_BUFFER_SIZE);
+        bootstrap.setOption(ConfigConstants.SEND_BUFFER_SIZE, ConfigConstants.DEFAULT_SEND_BUFFER_SIZE);
+        if (configure.getNetTag().equals("bobcat")) {
+            bootstrap.setOption("trafficClass", 96);
+        }
+
+        /* ready to Start the thread which refreshes the proxy list. */
+        ipManager = new ProxyConfigManager(configure, Utils.getLocalIp(), this);
+        ipManager.setName("proxyConfigManager");
+        if (configure.getBid() != null) {
+            ipManager.setBusinessId(configure.getBid());
+            bid = configure.getBid();
+        }
+
+        /*
+         * Request the IP before starting, so that we already have three
+         * connections.
+         */
+        this.configure = configure;
+        this.sender = sender;
+        this.aliveConnections = configure.getAliveConnections();
+
+        try {
+            ipManager.doProxyEntryQueryWork();
+        } catch (IOException e) {
+            e.printStackTrace();
+            logger.info(e.getMessage());
+        }
+        ipManager.setDaemon(true);
+        ipManager.start();
+
+        this.sendHBThread = new SendHBThread();
+        this.sendHBThread.setName("SendHBThread");
+        this.sendHBThread.start();
+    }
+
+    public ProxyConfigEntry getBidConfigureInfo() throws Exception {
+        return ipManager.getBidConfigure();
+    }
+
+    private boolean initConnection(HostInfo host) {
+        NettyClient client = clientMap.get(host);
+        if (client != null && client.isActive()) {
+            logger.info("this client {} has open!", host.getHostName());
+            throw new IllegalStateException(
+                    "The channel has already been opened");
+        }
+        client = new NettyClient(bootstrap, host.getHostName(),
+                host.getPortNumber(), configure);
+        boolean bSuccess = client.connect();
+
+        if (clientMapData.size() < aliveConnections) {
+
+            if (bSuccess) {
+                clientMapData.put(host, client);
+                clientList.add(client);
+                clientMap.put(host, client);
+                logger.info("build a connection success! {},channel {}", host.getHostName(), client.getChannel());
+            } else {
+                logger.info("build a connection fail! {}", host.getHostName());
+            }
+            logger.info("client map size {},client list size {}", clientMapData.size(), clientList.size());
+        } else {
+
+            if (bSuccess) {
+                clientMapHB.put(host, client);
+                clientMap.put(host, client);
+                logger.info("build a HBconnection success! {},channel {}", host.getHostName(), client.getChannel());
+            } else {
+                logger.info("build a HBconnection fail! {}", host.getHostName());
+            }
+        }
+        return bSuccess;
+    }
+
+    public void resetClient(Channel channel) {
+        if (channel == null) {
+            return;
+        }
+        logger.info("reset this channel {}", channel);
+        for (HostInfo hostInfo : clientMap.keySet()) {
+            if (hostInfo == null) {
+                continue;
+            }
+            NettyClient client = clientMap.get(hostInfo);
+            if (client != null && client.getChannel() != null
+                    && client.getChannel().getId().equals(channel.getId())) {
+                client.reconnect();
+                break;
+            }
+        }
+    }
+
+    public void setConnectionFrozen(Channel channel) {
+        if (channel == null) {
+            return;
+        }
+        logger.info("set this channel {} frozen", channel);
+        for (HostInfo hostInfo : clientMap.keySet()) {
+            if (hostInfo == null) {
+                continue;
+            }
+            NettyClient client = clientMap.get(hostInfo);
+            if (client != null && client.getChannel() != null
+                    && client.getChannel().getId().equals(channel.getId())) {
+                client.setFrozen();
+                logger.info("end to froze this channel {}", client.getChannel().toString());
+                break;
+            }
+        }
+    }
+
+    public void setConnectionBusy(Channel channel) {
+        if (channel == null) {
+            return;
+        }
+        logger.info("set this channel {} busy", channel);
+        for (HostInfo hostInfo : clientMap.keySet()) {
+            if (hostInfo == null) {
+                continue;
+            }
+            NettyClient client = clientMap.get(hostInfo);
+            if (client != null && client.getChannel() != null
+                    && client.getChannel().getId().equals(channel.getId())) {
+                client.setBusy();
+                break;
+            }
+        }
+    }
+
+    public synchronized NettyClient getClientByRoundRobin() {
+        NettyClient client = null;
+        if (clientList.isEmpty()) {
+            return null;
+        }
+        int currSize = clientList.size();
+        for (int retryTime = 0; retryTime < currSize; retryTime++) {
+            currentIndex = (++currentIndex) % currSize;
+            client = clientList.get(currentIndex);
+            if (client != null && client.isActive()) {
+                break;
+            }
+        }
+        if (client == null || !client.isActive()) {
+            return null;
+        }
+        //logger.info("get a client {}", client.getChannel());
+        return client;
+    }
+
+    public NettyClient getContainProxy(String proxyip) {
+        if (proxyip == null) {
+            return null;
+        }
+        for (NettyClient tmpClient : clientList) {
+            if (tmpClient != null && tmpClient.getServerIP() != null && tmpClient.getServerIP().equals(proxyip)) {
+                return tmpClient;
+            }
+        }
+        return null;
+    }
+
+    public void shutDown() {
+        bootstrap.releaseExternalResources();
+//        bootstrap.shutdown();
+
+        ipManager.shutDown();
+
+//        connectionCheckThread.shutDown();
+        sendHBThread.shutDown();
+        closeAllConnection();
+
+    }
+
+    private void closeAllConnection() {
+        if (!clientMap.isEmpty()) {
+            logger.info("ready to close all connections!");
+            for (HostInfo hostInfo : clientMap.keySet()) {
+                if (hostInfo == null) {
+                    continue;
+                }
+                NettyClient client = clientMap.get(hostInfo);
+                if (client != null && client.isActive()) {
+                    sender.waitForAckForChannel(client.getChannel());
+                    client.close();
+                }
+            }
+        }
+        clientMap.clear();
+        clientMapData.clear();
+        clientMapHB.clear();
+
+        channelLoadMapData.clear();
+        channelLoadMapHB.clear();
+        clientList.clear();
+        sender.clearCallBack();
+    }
+
+    private void updateAllConnection(List<HostInfo> hostInfos) {
+        closeAllConnection();
+        /* Build new channels*/
+        for (HostInfo hostInfo : hostInfos) {
+            initConnection(hostInfo);
+        }
+    }
+
+    public void notifyHBAck(Channel channel, short loadvalue) {
+        try {
+            if (loadvalue == (-1) || loadCycle == 0) {
+                return;
+            } else {
+                for (Map.Entry<HostInfo, NettyClient> entry : clientMapData.entrySet()) {
+                    NettyClient client = entry.getValue();
+                    HostInfo hostInfo = entry.getKey();
+                    if (client != null && client.getChannel() != null
+                            && client.getChannel().getId().equals(channel.getId())) {
+//                        logger.info("channel" + channel + "; Load:" + load);
+                        if (!channelLoadMapData.containsKey(hostInfo)) {
+                            channelLoadMapData.put(hostInfo, new int[ConfigConstants.CYCLE]);
+                        }
+                        if ((loadCycle - 1) >= 0) {
+                            channelLoadMapData.get(hostInfo)[loadCycle - 1] = loadvalue;
+                        } else {
+                            return;
+                        }
+                        break;
+                    }
+                }
+
+                for (Map.Entry<HostInfo, NettyClient> entry : clientMapHB.entrySet()) {
+                    NettyClient client = entry.getValue();
+                    HostInfo hostInfo = entry.getKey();
+                    if (client != null && client.getChannel() != null
+                            && client.getChannel().getId().equals(channel.getId())) {
+//                        logger.info("HBchannel" + channel + "; Load:" + load);
+                        if (!channelLoadMapHB.containsKey(hostInfo)) {
+                            channelLoadMapHB.put(hostInfo, new int[ConfigConstants.CYCLE]);
+                        }
+                        if ((loadCycle - 1) >= 0) {
+                            channelLoadMapHB.get(hostInfo)[loadCycle - 1] = loadvalue;
+                        } else {
+                            return;
+                        }
+                        break;
+                    }
+                }
+            }
+        } catch (Exception e) {
+            logger.error("{} , {}", e.toString(), e.getStackTrace());
+        }
+    }
+
+
+    private void loadDataInfo(Map<HostInfo, Integer> loadData) {
+        for (Map.Entry<HostInfo, int[]> entry : channelLoadMapData.entrySet()) {
+            HostInfo key = entry.getKey();
+            int[] value = entry.getValue();
+            int numerator = 0;
+            int denominator = 0;
+            for (int i = 0; i < value.length; i++) {
+                if (value[i] > 0) {
+                    numerator = numerator + value[i] * weight[i];
+                    denominator = denominator + weight[i];
+                }
+            }
+            int sum = numerator / denominator;
+            loadData.put(key, sum);
+        }
+    }
+
+    private void loadHBInfo(Map<HostInfo, Integer> loadHB) {
+        for (Map.Entry<HostInfo, int[]> entry : channelLoadMapHB.entrySet()) {
+            HostInfo key = entry.getKey();
+            int[] value = entry.getValue();
+            int numerator = 0;
+            int denominator = 0;
+            for (int i = 0; i < value.length; i++) {
+                if (value[i] > 0) {
+                    numerator = numerator + value[i] * weight[i];
+                    denominator = denominator + weight[i];
+                }
+            }
+            int sum = numerator / denominator;
+            loadHB.put(key, sum);
+        }
+    }
+
+    public void notifyHBControl() {
+        try {
+            writeLock();
+            logger.info("check if there is need to start balancing!");
+
+            Map<HostInfo, Integer> loadData = new ConcurrentHashMap<HostInfo, Integer>();
+            Map<HostInfo, Integer> loadHB = new ConcurrentHashMap<HostInfo, Integer>();
+            loadDataInfo(loadData);
+            loadHBInfo(loadHB);
+
+            List<Map.Entry<HostInfo, Integer>> listData = new ArrayList<>(loadData.entrySet());
+            Collections.sort(listData, new Comparator<Map.Entry<HostInfo, Integer>>() {
+                @Override
+                public int compare(Map.Entry<HostInfo, Integer> o1, Map.Entry<HostInfo, Integer> o2) {
+                    if (o2.getValue() != null && o1.getValue() != null && o1.getValue() > o2.getValue()) {
+                        return -1;
+                    } else {
+                        return 1;
+                    }
+                }
+            });
+            List<Map.Entry<HostInfo, Integer>> listHB = new ArrayList<>(loadHB.entrySet());
+            Collections.sort(listHB, new Comparator<Map.Entry<HostInfo, Integer>>() {
+                @Override
+                public int compare(Map.Entry<HostInfo, Integer> o1, Map.Entry<HostInfo, Integer> o2) {
+                    if (o2.getValue() != null && o1.getValue() != null && o2.getValue() > o1.getValue()) {
+                        return -1;
+                    } else {
+                        return 1;
+                    }
+                }
+            });
+
+
+            logger.info("show info: last compute result!");
+            for (Map.Entry<HostInfo, Integer> item : listData) {
+//                System.out.println("listData:"+listData.get(i));
+                logger.info("Client:" + item.getKey() + ";" + item.getValue());
+            }
+            for (Map.Entry<HostInfo, Integer> item : listHB) {
+//                System.out.println("listHB:"+listHB.get(i));
+                logger.info("HBClient:" + item.getKey() + ";" + item.getValue());
+            }
+            boolean isLoadSwitch = false;
+
+//            int smallSize = listData.size() < listHB.size() ? listData.size() : listHB.size();
+            int smallSize = 1;
+            for (int i = 0; i < smallSize; i++) {
+                if ((listData.get(i).getValue() - listHB.get(i).getValue()) >= this.loadThreshold) {
+                    isLoadSwitch = true;
+                    HostInfo dataHost = listData.get(i).getKey();
+                    HostInfo hbHost = listHB.get(i).getKey();
+                    logger.info("balancing client:" + dataHost.getHostName() + ",load: " + listData.get(i).getValue()
+                            + "; HBclient:" + hbHost.getHostName() + ",load: " + listHB.get(i).getValue());
+
+                    NettyClient client = clientMapData.get(dataHost);
+                    client.setFrozen();
+                    sender.waitForAckForChannel(client.getChannel());
+                    client.close();
+
+                    clientList.remove(clientMapData.get(dataHost));
+                    clientMap.remove(dataHost);
+                    clientMapData.remove(dataHost);
+//                    channelLoadMapData.remove(dataHost);
+                    clientMapData.put(hbHost, clientMapHB.get(hbHost));
+//                    channelLoadMapData.put(hbHost,listHB.get(i).getValue());
+                    clientList.add(clientMapHB.get(hbHost));
+                    clientMapHB.remove(hbHost);
+                }
+            }
+
+
+            if (!isLoadSwitch) {
+                logger.info("Choose other HBClient because there is no load balancing! ");
+            }
+            for (Map.Entry<HostInfo, NettyClient> entry : clientMapHB.entrySet()) {
+                entry.getValue().close();
+                clientMap.remove(entry.getKey());
+            }
+            clientMapHB.clear();
+
+            int realSize = this.realSize - clientMap.size();
+            if (realSize > 0) {
+                List<HostInfo> hostInfoList = new ArrayList<>(proxyInfoList);
+                hostInfoList.removeAll(clientMap.keySet());
+                List<HostInfo> replaceHost = getRealHosts(hostInfoList, realSize);
+                for (HostInfo hostInfo : replaceHost) {
+                    initConnection(hostInfo);
+                }
+            }
+        } catch (Exception e) {
+            logger.error("notifyHBcontrol", e);
+        } finally {
+            writeUnlock();
+        }
+    }
+
+    private void sendHeartBeat() {
+        for (HostInfo hostInfo : clientMap.keySet()) {
+            if (hostInfo == null) {
+                continue;
+            }
+            NettyClient client = clientMap.get(hostInfo);
+            if (client == null) {
+                continue;
+            }
+            try {
+                if (client.isActive()) {
+                    //logger.info("active host to send heartbeat! {}", entry.getKey().getHostName());
+                    EncodeObject encodeObject = new EncodeObject("heartbeat".getBytes(StandardCharsets.UTF_8),
+                            8, false, false, false, System.currentTimeMillis() / 1000, 1, "", "", "");
+                    if (configure.isNeedAuthentication()) {
+                        encodeObject.setAuth(configure.isNeedAuthentication(),
+                                configure.getUserName(), configure.getSecretKey());
+                    }
+                    client.write(encodeObject);
+                }
+            } catch (Throwable e) {
+                logger.error("sendHeartBeat to " + hostInfo.getReferenceName()
+                        + " exception {}, {}", e.toString(), e.getStackTrace());
+            }
+        }
+    }
+
+    /**
+     * fill up client with hb client
+     */
+    private void fillUpWorkClientWithHBClient() {
+        if (clientMapHB.size() > 0) {
+            logger.info("fill up work client with HB, clientMapData {}, clientMapHB {}",
+                    clientMapData.size(), clientMapHB.size());
+        }
+        Iterator<Map.Entry<HostInfo, NettyClient>> it = clientMapHB.entrySet().iterator();
+        while (it.hasNext() && clientMapData.size() < aliveConnections) {
+            Map.Entry<HostInfo, NettyClient> entry = it.next();
+            clientMapData.put(entry.getKey(), entry.getValue());
+            clientList.add(entry.getValue());
+            channelLoadMapHB.remove(entry.getKey());
+            it.remove();
+        }
+    }
+
+    private void fillUpWorkClientWithLastBadClient() {
+
+
+        int currentRealSize = aliveConnections - clientMapData.size();
+
+        List<HostInfo> pendingBadList = new ArrayList<>();
+        for (Map.Entry<HostInfo, AtomicLong> entry : lastBadHostMap.entrySet()) {
+            if (pendingBadList.size() < currentRealSize) {
+                pendingBadList.add(entry.getKey());
+            } else {
+                for (int index = 0; index < pendingBadList.size(); index++) {
+
+                    if (entry.getValue().get() < lastBadHostMap
+                            .get(pendingBadList.get(index)).get()) {
+                        pendingBadList.set(index, entry.getKey());
+                    }
+                }
+            }
+        }
+        List<HostInfo> replaceHostLists = getRealHosts(pendingBadList, currentRealSize);
+        if (replaceHostLists.size() > 0) {
+            logger.info("replace bad connection, use last bad list, "
+                            + "last bad list {}, client Map data {}",
+                    lastBadHostMap.size(), clientMapData.size());
+        }
+        for (HostInfo hostInfo : replaceHostLists) {
+
+            boolean isSuccess = initConnection(hostInfo);
+
+            if (isSuccess) {
+                lastBadHostMap.remove(hostInfo);
+            }
+        }
+    }
+
+    private void removeBadRealClient(List<HostInfo> badHostLists, List<HostInfo> normalHostLists) {
+        for (HostInfo hostInfo : clientMapData.keySet()) {
+            if (hostInfo == null) {
+                continue;
+            }
+            NettyClient client = clientMapData.get(hostInfo);
+            if (client == null || !client.isActive()) {
+                logger.info("this host {} is bad! so remove it", hostInfo.getHostName());
+                badHostLists.add(hostInfo);
+            } else {
+                logger.info("this host {} is active! so keep it", hostInfo.getHostName());
+                normalHostLists.add(hostInfo);
+            }
+        }
+    }
+
+
+    private void removeBadHBClients(List<HostInfo> badHostLists, List<HostInfo> normalHostLists) {
+        for (HostInfo hostInfo : clientMapHB.keySet()) {
+            if (hostInfo == null) {
+                continue;
+            }
+            NettyClient client = clientMapHB.get(hostInfo);
+            if (client == null || !client.isActive()) {
+                logger.info("this HBhost {} is bad! so remove it", hostInfo.getHostName());
+                badHostLists.add(hostInfo);
+            } else {
+                logger.info("this HBhost {} is active! so keep it", hostInfo.getHostName());
+                normalHostLists.add(hostInfo);
+            }
+        }
+    }
+
+    private void removeBadClients(List<HostInfo> badHostLists) {
+        for (HostInfo hostInfo : badHostLists) {
+            if (hostInfo == null) {
+                continue;
+            }
+            NettyClient client = clientMapData.get(hostInfo);
+            if (client != null) {
+                sender.waitForAckForChannel(client.getChannel());
+                client.close();
+                clientMapData.remove(hostInfo);
+                clientMap.remove(hostInfo);
+                clientList.remove(client);
+
+                channelLoadMapData.remove(hostInfo);
+                logger.info("remove this client {}", hostInfo.getHostName());
+            }
+            client = clientMapHB.get(hostInfo);
+            if (client != null) {
+                clientMapHB.get(hostInfo).close();
+                clientMapHB.remove(hostInfo);
+                clientMap.remove(hostInfo);
+
+                channelLoadMapHB.remove(hostInfo);
+                logger.info("remove this HBclient {}", hostInfo.getHostName());
+            }
+        }
+    }
+
+
+    public void replaceBadConnectionHB() {
+        try {
+            writeLock();
+
+            List<HostInfo> badHostLists = new ArrayList<>();
+            List<HostInfo> normalHostLists = new ArrayList<>();
+            removeBadRealClient(badHostLists, normalHostLists);
+            removeBadHBClients(badHostLists, normalHostLists);
+            removeBadClients(badHostLists);
+
+            if (badHostLists.size() == 0 && normalHostLists.size() != 0 && clientMapData.size() >= aliveConnections) {
+                logger.info("hasn't bad host! so keep it");
+                if (loadCycle >= ConfigConstants.CYCLE) {
+                    if (loadThreshold == 0) {
+                        logger.info("the proxy cluster is being updated!");
+                    } else if (clientMapHB.size() != 0 && clientMapData.size() != 0) {
+                        notifyHBControl();
+                    } else if (this.realSize != clientMap.size()) {
+                        logger.info("make the amount of proxy to original value");
+                        int realSize = this.realSize - clientMap.size();
+                        if (realSize > 0) {
+                            List<HostInfo> hostInfoList = new ArrayList<>(proxyInfoList);
+                            hostInfoList.removeAll(clientMap.keySet());
+                            List<HostInfo> replaceHost = getRealHosts(hostInfoList, realSize);
+                            for (HostInfo hostInfo : replaceHost) {
+                                initConnection(hostInfo);
+                            }
+                        }
+                    }
+                    loadCycle = 0;
+                    channelLoadMapData.clear();
+                    channelLoadMapHB.clear();
+                }
+                return;
+            } else {
+                loadCycle = 0;
+                channelLoadMapData.clear();
+                channelLoadMapHB.clear();
+            }
+
+            List<HostInfo> hostLists = new ArrayList<HostInfo>(this.proxyInfoList);
+            hostLists.removeAll(badHostLists);
+            hostLists.removeAll(lastBadHostMap.keySet());
+            hostLists.removeAll(normalHostLists);
+
+            int realSize = this.realSize - clientMap.size();
+            if (realSize > hostLists.size()) {
+                realSize = hostLists.size();
+            }
+
+            if (realSize != 0) {
+                List<HostInfo> replaceHostLists = getRealHosts(hostLists, realSize);
+                /* Build new channels.*/
+                for (HostInfo hostInfo : replaceHostLists) {
+                    initConnection(hostInfo);
+                }
+            }
+
+
+            if (clientMapData.size() < aliveConnections) {
+                fillUpWorkClientWithHBClient();
+            }
+
+
+            if (clientMapData.size() < aliveConnections) {
+                fillUpWorkClientWithLastBadClient();
+            }
+
+
+            for (HostInfo hostInfo : badHostLists) {
+                AtomicLong tmpValue = new AtomicLong(0);
+                AtomicLong hostValue = lastBadHostMap.putIfAbsent(hostInfo, tmpValue);
+                if (hostValue == null) {
+                    hostValue = tmpValue;
+                }
+                hostValue.incrementAndGet();
+            }
+
+            for (HostInfo hostInfo : normalHostLists) {
+                lastBadHostMap.remove(hostInfo);
+            }
+
+            logger.info(
+                    "replace bad connection ,client map size {},client list size {}",
+                    clientMapData.size(), clientList.size());
+
+
+        } catch (Exception e) {
+            logger.error("replaceBadConnection exception {}, {}", e.toString(), e.getStackTrace());
+        } finally {
+            writeUnlock();
+        }
+
+    }
+
+    public void setProxyInfoList(List<HostInfo> proxyInfoList) {
+        try {
+            /* Close and remove old client. */
+            writeLock();
+            this.proxyInfoList = proxyInfoList;
+
+            if (loadThreshold == 0) {
+                if (aliveConnections >= proxyInfoList.size()) {
+                    realSize = proxyInfoList.size();
+                    aliveConnections = realSize;
+                    logger.error("there is no enough proxy to work!");
+                } else {
+                    realSize = aliveConnections;
+                }
+            } else {
+                if (aliveConnections >= proxyInfoList.size()) {
+                    realSize = proxyInfoList.size();
+                    aliveConnections = realSize;
+                    logger.error("there is no idle proxy to choose for balancing!");
+                } else if ((aliveConnections + 4) > proxyInfoList.size()) {
+                    realSize = proxyInfoList.size();
+                    logger.warn("there is only {} idle proxy to choose for balancing!",
+                            proxyInfoList.size() - aliveConnections);
+                } else {
+                    realSize = aliveConnections + 4;
+                }
+            }
+
+            List<HostInfo> hostInfos = getRealHosts(proxyInfoList, realSize);
+
+            /* Refresh the current channel connections. */
+            updateAllConnection(hostInfos);
+
+            logger.info(
+                    "update all connection ,client map size {},client list size {}",
+                    clientMapData.size(), clientList.size());
+
+        } catch (Exception e) {
+            logger.error(e.getMessage());
+        } finally {
+            writeUnlock();
+        }
+    }
+
+    private List<HostInfo> getRealHosts(List<HostInfo> hostList, int realSize) {
+        if (realSize > hostList.size()) {
+            return hostList;
+        }
+        Collections.shuffle(hostList);
+        List<HostInfo> resultHosts = new ArrayList<HostInfo>(realSize);
+        for (int i = 0; i < realSize; i++) {
+            resultHosts.add(hostList.get(i));
+            logger.info("host={}", hostList.get(i));
+        }
+        return resultHosts;
+    }
+
+
+    private class SendHBThread extends Thread {
+
+        private boolean bShutDown = false;
+        private final int[] random = {17, 19, 23, 31, 37};
+
+        public SendHBThread() {
+            bShutDown = false;
+        }
+
+        public void shutDown() {
+            logger.info("begin to shut down SendHBThread!");
+            bShutDown = true;
+        }
+
+        @Override
+        public void run() {
+            while (!bShutDown) {
+                try {
+                    loadCycle++;
+                    if (!clientMapHB.isEmpty()) {
+                        sendHeartBeat();
+                    }
+                    replaceBadConnectionHB();
+                    try {
+                        int index = (int) (Math.random() * random.length);
+                        Thread.sleep((random[index]) * 1000);
+                    } catch (InterruptedException e) {
+                        // TODO Auto-generated catch block
+                        logger.error(e.toString());
+                    }
+                } catch (Throwable e) {
+                    logger.error("SendHBThread throw exception: ", e);
+                }
+            }
+        }
+    }
+
+
+}
diff --git a/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/network/ClientPipelineFactory.java b/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/network/ClientPipelineFactory.java
new file mode 100644
index 0000000..a274c4e
--- /dev/null
+++ b/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/network/ClientPipelineFactory.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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.dataproxy.network;
+
+import org.apache.inlong.dataproxy.codec.ProtocolDecoder;
+import org.apache.inlong.dataproxy.codec.ProtocolEncoder;
+import org.jboss.netty.channel.ChannelPipeline;
+import org.jboss.netty.channel.ChannelPipelineFactory;
+import org.jboss.netty.channel.Channels;
+import org.jboss.netty.handler.codec.frame.LengthFieldBasedFrameDecoder;
+
+public class ClientPipelineFactory implements ChannelPipelineFactory {
+    private final ClientMgr clientMgr;
+    private final Sender sender;
+
+    public ClientPipelineFactory(ClientMgr clientMgr, Sender sender) {
+        this.clientMgr = clientMgr;
+        this.sender = sender;
+    }
+
+    public ChannelPipeline getPipeline() throws Exception {
+
+        ChannelPipeline pipeline = Channels.pipeline();
+
+        pipeline.addLast("frameDecoder", new LengthFieldBasedFrameDecoder(
+                65536, 0, 4, 0, 0));
+        pipeline.addLast("contentDecoder", new ProtocolDecoder());
+        pipeline.addLast("contentEncoder", new ProtocolEncoder());
+        pipeline.addLast("handler", new ClientHandler(sender, clientMgr));
+
+        return pipeline;
+    }
+}
diff --git a/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/network/HttpMessage.java b/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/network/HttpMessage.java
new file mode 100644
index 0000000..2476efe
--- /dev/null
+++ b/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/network/HttpMessage.java
@@ -0,0 +1,75 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.dataproxy.network;
+
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.inlong.dataproxy.SendMessageCallback;
+
+/**
+ * http message for cache.
+ */
+public class HttpMessage {
+    private final String bid;
+    private final String tid;
+    private final List<String> bodies;
+    private final SendMessageCallback callback;
+    private final long dt;
+    private final long timeout;
+    private final TimeUnit timeUnit;
+
+    public HttpMessage(List<String> bodies, String bid, String tid, long dt,
+                       long timeout, TimeUnit timeUnit, SendMessageCallback callback) {
+        this.bid = bid;
+        this.tid = tid;
+        this.bodies = bodies;
+        this.callback = callback;
+        this.dt = dt;
+        this.timeout = timeout;
+        this.timeUnit = timeUnit;
+    }
+
+    public String getBid() {
+        return bid;
+    }
+
+    public String getTid() {
+        return tid;
+    }
+
+    public List<String> getBodies() {
+        return bodies;
+    }
+
+    public SendMessageCallback getCallback() {
+        return callback;
+    }
+
+    public long getDt() {
+        return dt;
+    }
+
+    public long getTimeout() {
+        return timeout;
+    }
+
+    public TimeUnit getTimeUnit() {
+        return timeUnit;
+    }
+}
diff --git a/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/network/HttpProxySender.java b/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/network/HttpProxySender.java
new file mode 100644
index 0000000..507b830
--- /dev/null
+++ b/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/network/HttpProxySender.java
@@ -0,0 +1,230 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.dataproxy.network;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.inlong.dataproxy.ProxyClientConfig;
+import org.apache.inlong.dataproxy.SendMessageCallback;
+import org.apache.inlong.dataproxy.SendResult;
+import org.apache.inlong.dataproxy.config.ProxyConfigEntry;
+import org.apache.inlong.dataproxy.config.ProxyConfigManager;
+import org.apache.inlong.dataproxy.config.HostInfo;
+import org.apache.inlong.dataproxy.http.InternalHttpSender;
+import org.apache.inlong.dataproxy.utils.ConcurrentHashSet;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * http sender
+ */
+public class HttpProxySender extends Thread {
+    private static final Logger logger = LoggerFactory.getLogger(Sender.class);
+
+    private final ConcurrentHashSet<HostInfo> hostList = new ConcurrentHashSet<>();
+
+    private final ProxyClientConfig proxyClientConfig;
+    private ProxyConfigManager proxyConfigManager;
+
+
+    private boolean bShutDown = false;
+
+    private final InternalHttpSender internalHttpSender;
+    private final LinkedBlockingQueue<HttpMessage> messageCache;
+
+    public HttpProxySender(ProxyClientConfig configure) throws Exception {
+        this.proxyClientConfig = configure;
+        initTDMClientAndRequest(configure);
+        this.messageCache = new LinkedBlockingQueue<>(configure.getTotalAsyncCallbackSize());
+        internalHttpSender = new InternalHttpSender(configure, hostList, messageCache);
+    }
+
+    /**
+     * get proxy list
+     *
+     * @param configure
+     * @throws Exception
+     */
+    private void initTDMClientAndRequest(ProxyClientConfig configure) throws Exception {
+
+        try {
+            proxyConfigManager = new ProxyConfigManager(configure,
+                    Utils.getLocalIp(), null);
+            proxyConfigManager.setBusinessId(configure.getBid());
+            ProxyConfigEntry proxyConfigEntry = retryGettingProxyConfig();
+            hostList.addAll(proxyConfigEntry.getHostMap().values());
+
+            this.setDaemon(true);
+            this.start();
+        } catch (Throwable e) {
+            if (configure.isReadProxyIPFromLocal()) {
+                throw new Exception("Get local proxy configure failure!", e.getCause());
+            } else {
+                throw new Exception("Visit TDManager error!", e.getCause());
+            }
+        }
+        logger.info("http proxy sender starts");
+    }
+
+    /**
+     * retry fetching proxy config in case of network issue.
+     *
+     * @return proxy config entry.
+     */
+    private ProxyConfigEntry retryGettingProxyConfig() throws Exception {
+        return proxyConfigManager.getBidConfigure();
+    }
+
+    /**
+     * get proxy list
+     */
+    @Override
+    public void run() {
+        while (!bShutDown) {
+            try {
+                int rand = ThreadLocalRandom.current().nextInt(0, 600);
+                int randSleepTime = proxyClientConfig.getProxyHttpUpdateIntervalMinutes() * 60 + rand;
+                TimeUnit.MILLISECONDS.sleep(randSleepTime * 1000);
+                if (proxyConfigManager != null) {
+                    ProxyConfigEntry proxyConfigEntry = proxyConfigManager.getBidConfigure();
+                    hostList.addAll(proxyConfigEntry.getHostMap().values());
+                    hostList.retainAll(proxyConfigEntry.getHostMap().values());
+                } else {
+                    logger.error("manager is null, please check it!");
+                }
+                logger.info("get new proxy list " + hostList.toString());
+            } catch (InterruptedException ignored) {
+                // ignore it.
+            } catch (Exception ex) {
+                logger.error("managerFetcher get or save managerIpList occur error,", ex);
+            }
+        }
+    }
+
+
+    /**
+     * send by http
+     *
+     * @param body
+     * @param bid
+     * @param tid
+     * @param dt
+     * @param timeout
+     * @param timeUnit
+     * @return
+     */
+    public SendResult sendMessage(String body, String bid, String tid, long dt,
+                                  long timeout, TimeUnit timeUnit) {
+        return sendMessage(Collections.singletonList(body), bid, tid, dt, timeout, timeUnit);
+    }
+
+    /**
+     * send multiple messages.
+     *
+     * @param bodies   list of bodies
+     * @param bid
+     * @param tid
+     * @param dt
+     * @param timeout
+     * @param timeUnit
+     * @return
+     */
+    public SendResult sendMessage(List<String> bodies, String bid, String tid, long dt,
+                                  long timeout, TimeUnit timeUnit) {
+        if (hostList.isEmpty()) {
+            logger.error("proxy list is empty, maybe client has been "
+                    + "closed or bid is not assigned with proxy list");
+            return SendResult.NO_CONNECTION;
+        }
+        return internalHttpSender.sendMessageWithHostInfo(
+                bodies, bid, tid, dt, timeout, timeUnit);
+
+    }
+
+    /**
+     * async sender
+     *
+     * @param bodies
+     * @param bid
+     * @param tid
+     * @param dt
+     * @param timeout
+     * @param timeUnit
+     * @param callback
+     */
+    public void asyncSendMessage(List<String> bodies, String bid, String tid, long dt,
+                                 long timeout, TimeUnit timeUnit, SendMessageCallback callback) {
+        List<String> bodyList = new ArrayList<>(bodies);
+        HttpMessage httpMessage = new HttpMessage(bodyList, bid, tid, dt,
+                timeout, timeUnit, callback);
+        try {
+            if (!messageCache.offer(httpMessage)) {
+                if (!proxyClientConfig.isDiscardOldMessage()) {
+                    // put and wait for capacity available.
+                    messageCache.put(httpMessage);
+                } else {
+                    // discard old message and use new message instead.
+                    logger.debug("discard old message and use new message instead");
+                    HttpMessage oldMessage = messageCache.poll();
+                    if (oldMessage != null) {
+                        oldMessage.getCallback().onMessageAck(SendResult.TIMEOUT);
+                    }
+                    messageCache.offer(httpMessage);
+                }
+            }
+        } catch (Exception exception) {
+            logger.error("error async sending data", exception);
+        }
+    }
+
+    /**
+     * async send single message.
+     *
+     * @param body
+     * @param bid
+     * @param tid
+     * @param dt
+     * @param timeout
+     * @param timeUnit
+     * @param callback
+     */
+    public void asyncSendMessage(String body, String bid, String tid, long dt,
+                                 long timeout, TimeUnit timeUnit, SendMessageCallback callback) {
+        asyncSendMessage(Collections.singletonList(body), bid, tid,
+                dt, timeout, timeUnit, callback);
+    }
+
+    /**
+     * close
+     */
+    public void close() {
+        hostList.clear();
+        bShutDown = true;
+        try {
+            this.interrupt();
+            internalHttpSender.close();
+        } catch (Exception exception) {
+            logger.error("error while closing http client", exception);
+        }
+    }
+}
diff --git a/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/network/NettyClient.java b/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/network/NettyClient.java
new file mode 100644
index 0000000..768e6f3
--- /dev/null
+++ b/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/network/NettyClient.java
@@ -0,0 +1,215 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.dataproxy.network;
+
+import java.net.InetSocketAddress;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.ReentrantLock;
+
+import org.apache.inlong.dataproxy.ProxyClientConfig;
+import org.apache.inlong.dataproxy.codec.EncodeObject;
+import org.jboss.netty.bootstrap.ClientBootstrap;
+import org.jboss.netty.channel.Channel;
+import org.jboss.netty.channel.ChannelFuture;
+import org.jboss.netty.channel.ChannelFutureListener;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class NettyClient {
+    private static final Logger logger = LoggerFactory.getLogger(NettyClient.class);
+
+    private Channel channel = null;
+    private final ReentrantLock stateLock = new ReentrantLock();
+
+    private ConnState connState;
+    private ProxyClientConfig configure;
+    private ClientBootstrap bootstrap;
+    private String serverIP;
+    private int serverPort;
+
+    public String getServerIP() {
+        return serverIP;
+    }
+
+    public void setServerIP(String serverIP) {
+        this.serverIP = serverIP;
+    }
+
+    public NettyClient(ClientBootstrap bootstrap, String serverIP,
+                       int serverPort, ProxyClientConfig configure) {
+        this.bootstrap = bootstrap;
+        this.serverIP = serverIP;
+        this.serverPort = serverPort;
+        this.configure = configure;
+        setState(ConnState.INIT);
+    }
+
+    public Channel getChannel() {
+        return channel;
+    }
+
+    public void setChannel(Channel channel) {
+        this.channel = channel;
+    }
+
+    public boolean connect() {
+        // Connect to server.
+
+        setState(ConnState.INIT);
+        final CountDownLatch awaitLatch = new CountDownLatch(1);
+        ChannelFuture future = bootstrap.connect(new InetSocketAddress(
+                serverIP, serverPort));
+        future.addListener(new ChannelFutureListener() {
+            public void operationComplete(ChannelFuture arg0) throws Exception {
+                logger.info("connect ack! {}", serverIP);
+                awaitLatch.countDown();
+            }
+        });
+
+        try {
+            // Wait until the connection is built.
+            awaitLatch.await(configure.getConnectTimeoutMillis(),
+                    TimeUnit.MILLISECONDS);
+        } catch (Exception e) {
+            logger.error("create connect exception! {}", e.getMessage());
+            e.printStackTrace();
+            return false;
+        }
+
+        // Return if no connection is built.
+        if (!future.isSuccess()) {
+            return false;
+        }
+        channel = future.getChannel();
+        setState(ConnState.READY);
+        logger.info("ip {} stat {}", serverIP, connState);
+        return true;
+    }
+
+    public boolean close() {
+        logger.debug("begin to close this channel{}", channel);
+        final CountDownLatch awaitLatch = new CountDownLatch(1);
+        boolean ret = true;
+        try {
+            if (channel != null) {
+                ChannelFuture future = channel.close();
+                future.addListener(new ChannelFutureListener() {
+                    public void operationComplete(ChannelFuture arg0)
+                            throws Exception {
+                        logger.info("close client ack {}", serverIP);
+                        awaitLatch.countDown();
+                    }
+                });
+                // Wait until the connection is close.
+                awaitLatch.await(configure.getRequestTimeoutMillis(),
+                        TimeUnit.MILLISECONDS);
+                // Return if close this connection fail.
+                if (!future.isSuccess()) {
+                    ret = false;
+                }
+            }
+        } catch (Exception e) {
+            logger.error("close connect {" + serverIP + ":" + serverPort + "} exception! {}", e.getMessage());
+            e.printStackTrace();
+            ret = false;
+        } finally {
+            setState(ConnState.DEAD);
+        }
+        logger.info("end to close {" + serverIP + ":" + serverPort + "} 's channel, bSuccess = " + ret);
+        return ret;
+    }
+
+    public void reconnect() {
+        this.close();
+        this.connect();
+    }
+
+    public boolean isActive() {
+        stateLock.lock();
+        try {
+            return (connState == ConnState.READY && channel != null && channel.isOpen() && channel.isConnected());
+        } catch (Exception e) {
+            logger.error("channel maybe null!{}", e.getMessage());
+            return false;
+        } finally {
+            stateLock.unlock();
+        }
+        // channel.isOpen();
+    }
+
+    private void setState(ConnState newState) {
+        stateLock.lock();
+        try {
+            connState = newState;
+        } catch (Exception e) {
+            logger.error("setState maybe error!{}", e.getMessage());
+        } finally {
+            stateLock.unlock();
+        }
+    }
+
+    private enum ConnState {
+        INIT, READY, FROZEN, DEAD, BUSY
+    }
+
+    public ChannelFuture write(EncodeObject encodeObject) {
+        // TODO Auto-generated method stub
+        ChannelFuture future = null;
+        try {
+            future = channel.write(encodeObject);
+        } catch (Exception e) {
+            logger.error("channel write error {}", e.getMessage());
+            e.printStackTrace();
+        }
+        return future;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (this == obj) {
+            return true;
+        }
+        if (obj == null) {
+            return false;
+        }
+        if (getClass() != obj.getClass()) {
+            return false;
+        }
+        NettyClient other = (NettyClient) obj;
+        if (channel == null) {
+            if (other.channel != null) {
+                return false;
+            }
+        } else if (!channel.equals(other.channel)) {
+            return false;
+        }
+        return true;
+    }
+
+    public void setFrozen() {
+        // TODO Auto-generated method stub
+        setState(ConnState.FROZEN);
+
+    }
+
+    public void setBusy() {
+        setState(ConnState.BUSY);
+    }
+
+}
diff --git a/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/network/ProxysdkException.java b/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/network/ProxysdkException.java
new file mode 100644
index 0000000..b30c2a1
--- /dev/null
+++ b/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/network/ProxysdkException.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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.dataproxy.network;
+
+/**
+ * Created by jesseyzhou on 2018/6/22.
+ */
+public class ProxysdkException extends Exception {
+
+    public ProxysdkException() {
+    }
+
+    public ProxysdkException(String message) {
+        super(message);
+    }
+
+    public ProxysdkException(String message, Throwable cause) {
+        super(message, cause);
+    }
+
+    public ProxysdkException(Throwable cause) {
+        super(cause);
+    }
+}
diff --git a/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/network/QueueObject.java b/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/network/QueueObject.java
new file mode 100644
index 0000000..6730e72
--- /dev/null
+++ b/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/network/QueueObject.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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.dataproxy.network;
+
+import java.util.concurrent.TimeUnit;
+
+import org.apache.inlong.dataproxy.SendMessageCallback;
+
+public class QueueObject {
+    private final long sendTimeInMillis;
+    private final SendMessageCallback callback;
+    private final long timeoutInMillis;
+    private final int size;
+
+    public QueueObject(long sendTimeInMillis,
+                       SendMessageCallback callback,
+                       long timeout,
+                       TimeUnit timeUnit) {
+        this.sendTimeInMillis = sendTimeInMillis;
+        this.callback = callback;
+        this.timeoutInMillis = TimeUnit.MILLISECONDS.convert(timeout, timeUnit);
+        this.size = 1;
+    }
+
+    public QueueObject(long sendTimeInMillis,
+                       SendMessageCallback callback, int size,
+                       long timeout,
+                       TimeUnit timeUnit) {
+        this.sendTimeInMillis = sendTimeInMillis;
+        this.callback = callback;
+        this.timeoutInMillis = TimeUnit.MILLISECONDS.convert(timeout, timeUnit);
+        this.size = size;
+    }
+
+    public long getSendTimeInMillis() {
+        return sendTimeInMillis;
+    }
+
+    public SendMessageCallback getCallback() {
+        return callback;
+    }
+
+    public long getTimeoutInMillis() {
+        return timeoutInMillis;
+    }
+
+    public int getSize() {
+        return size;
+    }
+}
diff --git a/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/network/Sender.java b/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/network/Sender.java
new file mode 100644
index 0000000..2cc5ea1
--- /dev/null
+++ b/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/network/Sender.java
@@ -0,0 +1,694 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.dataproxy.network;
+
+import java.io.PrintWriter;
+import java.io.StringWriter;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.locks.ReentrantLock;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.inlong.dataproxy.ProxyClientConfig;
+import org.apache.inlong.dataproxy.FileCallback;
+import org.apache.inlong.dataproxy.SendMessageCallback;
+import org.apache.inlong.dataproxy.SendResult;
+import org.apache.inlong.dataproxy.codec.EncodeObject;
+import org.apache.inlong.dataproxy.config.ProxyConfigEntry;
+import org.apache.inlong.dataproxy.threads.MetricWorkerThread;
+import org.apache.inlong.dataproxy.threads.TimeoutScanThread;
+import org.jboss.netty.channel.Channel;
+import org.jboss.netty.channel.ChannelFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class Sender {
+
+    private static final Logger logger = LoggerFactory.getLogger(Sender.class);
+
+    /* Store the callback used by asynchronously message sending. */
+    private final ConcurrentHashMap<Channel, ConcurrentHashMap<String, QueueObject>> callbacks =
+            new ConcurrentHashMap<Channel, ConcurrentHashMap<String, QueueObject>>();
+    /* Store the synchronous message sending invocations. */
+    private final ConcurrentHashMap<String, SyncMessageCallable> syncCallables =
+            new ConcurrentHashMap<String, SyncMessageCallable>();
+    private final ConcurrentHashMap<String, NettyClient> chooseProxy = new ConcurrentHashMap<String, NettyClient>();
+    private final ReentrantLock stateLock = new ReentrantLock();
+    private final ExecutorService threadPool;
+    private final int asyncCallbackMaxSize;
+    private final AtomicInteger currentBufferSize = new AtomicInteger(0);
+    private final TimeoutScanThread scanThread;
+    private final ClientMgr clientMgr;
+    private final ProxyClientConfig configure;
+    private final boolean isFile;
+    private final MetricWorkerThread metricWorker;
+    private String clusterId;
+
+    public Sender(ProxyClientConfig configure) throws Exception {
+        this(configure, null);
+    }
+
+    public Sender(ProxyClientConfig configure, ChannelFactory selfDefineFactory) throws Exception {
+        this.configure = configure;
+        this.asyncCallbackMaxSize = configure.getTotalAsyncCallbackSize();
+        this.threadPool = Executors.newCachedThreadPool();
+        this.clientMgr = new ClientMgr(configure, this, selfDefineFactory);
+        ProxyConfigEntry proxyConfigEntry = null;
+        try {
+            proxyConfigEntry = this.clientMgr.getBidConfigureInfo();
+            setClusterId(proxyConfigEntry.getClusterId());
+        } catch (Throwable e) {
+            if (configure.isReadProxyIPFromLocal()) {
+                throw new Exception("Get local proxy configure failure!", e.getCause());
+            } else {
+                throw new Exception("Visit manager error!", e.getCause());
+            }
+        }
+        if (!proxyConfigEntry.isInterVisit()) {
+            if (!configure.isNeedAuthentication()) {
+                throw new Exception("In OutNetwork isNeedAuthentication must be true!");
+            }
+            if (!configure.isNeedDataEncry()) {
+                throw new Exception("In OutNetwork isNeedDataEncry must be true!");
+            }
+        }
+        this.isFile = configure.isFile();
+        scanThread = new TimeoutScanThread(callbacks, currentBufferSize, configure, clientMgr);
+        scanThread.start();
+
+        metricWorker = new MetricWorkerThread(configure, this);
+        metricWorker.start();
+        logger.info("proxy sdk is starting!");
+    }
+
+    private void checkCallbackList() {
+        // max wait for 1 min
+        logger.info("checking call back list before close, current size is {}",
+                currentBufferSize.get());
+        int count = 0;
+        try {
+            while (currentBufferSize.get() > 0 && count < 60) {
+                TimeUnit.SECONDS.sleep(1);
+                count += 1;
+            }
+            if (currentBufferSize.get() > 0) {
+                logger.warn("callback not empty {}, please check it", currentBufferSize.get());
+            }
+        } catch (Exception ex) {
+            logger.error("exception while checking callback list", ex);
+        }
+    }
+
+    public void close() {
+        checkCallbackList();
+        scanThread.shutDown();
+        clientMgr.shutDown();
+        threadPool.shutdown();
+        metricWorker.close();
+    }
+
+    public String getExceptionStack(Throwable e) {
+        StringWriter sw = new StringWriter();
+        PrintWriter pw = new PrintWriter(sw);
+        String exceptStr = null;
+        try {
+            e.printStackTrace(pw);
+            exceptStr = sw.toString();
+        } catch (Exception ex) {
+            logger.error(getExceptionStack(ex));
+        } finally {
+            try {
+                pw.close();
+                sw.close();
+            } catch (Exception ex) {
+                logger.error(getExceptionStack(ex));
+            }
+        }
+        return exceptStr;
+    }
+
+    /*Used for asynchronously message sending.*/
+    public void notifyCallback(Channel channel, String messageId, SendResult result) {
+        if (channel == null) {
+            return;
+        }
+        ConcurrentHashMap<String, QueueObject> callBackMap = callbacks.get(channel);
+        if (callBackMap == null) {
+            return;
+        }
+        QueueObject callback = callBackMap.remove(messageId);
+        if (callback == null) {
+            return;
+        }
+        if (isFile) {
+            String proxyip = channel.getRemoteAddress().toString();
+            ((FileCallback) callback.getCallback()).onMessageAck(result.toString()
+                    + "=" + proxyip.substring(1, proxyip.indexOf(':')));
+            currentBufferSize.addAndGet(-callback.getSize());
+        } else {
+            callback.getCallback().onMessageAck(result);
+            currentBufferSize.decrementAndGet();
+        }
+    }
+
+    private SendResult syncSendInternalMessage(NettyClient client,
+        EncodeObject encodeObject, String msgUUID, long timeout, TimeUnit timeUnit)
+        throws ExecutionException, InterruptedException, TimeoutException {
+        client = clientMgr.getClientByRoundRobin();
+        if (client == null) {
+            return SendResult.NO_CONNECTION;
+        }
+        if (isNotValidateAttr(encodeObject.getCommonattr(), encodeObject.getAttributes())) {
+            logger.error("error attr format {} {}", encodeObject.getCommonattr(),
+                encodeObject.getAttributes());
+            return SendResult.INVALID_ATTRIBUTES;
+        }
+        if (encodeObject.getMsgtype() == 7) {
+            int bidnum = 0;
+            int tidnum = 0;
+            if (encodeObject.getBid().equals(clientMgr.getBid())) {
+                bidnum = clientMgr.getBidNum();
+                tidnum = clientMgr.getTidMap().get(encodeObject.getTid()) != null
+                    ? clientMgr.getTidMap().get(encodeObject.getTid()) : 0;
+            }
+            encodeObject.setBidNum(bidnum);
+            encodeObject.setTidNum(tidnum);
+            if (bidnum == 0 || tidnum == 0) {
+                encodeObject.setBidTransfer(false);
+            }
+        }
+        if (this.configure.isNeedDataEncry()) {
+            encodeObject.setEncryptEntry(true, configure.getUserName(), clientMgr.getEncryptConfigEntry());
+        } else {
+            encodeObject.setEncryptEntry(false, null, null);
+        }
+        encodeObject.setMsgUUID(msgUUID);
+        SyncMessageCallable callable = new SyncMessageCallable(client, encodeObject, timeout, timeUnit);
+        syncCallables.put(encodeObject.getMessageId(), callable);
+
+        Future<SendResult> future = threadPool.submit(callable);
+        return future.get(timeout, timeUnit);
+    }
+
+    /* Following methods used by synchronously message sending. */
+    public SendResult syncSendMessage(EncodeObject encodeObject, String msgUUID,
+                                      long timeout, TimeUnit timeUnit) {
+        metricWorker.recordNumByKey(encodeObject.getMessageId(),
+                encodeObject.getBid(), encodeObject.getTid(),
+                Utils.getLocalIp(), encodeObject.getDt(), encodeObject.getPackageTime(), encodeObject.getRealCnt());
+        NettyClient client = null;
+        SendResult message = null;
+        try {
+            message = syncSendInternalMessage(client, encodeObject, msgUUID, timeout, timeUnit);
+        } catch (InterruptedException e) {
+            // TODO Auto-generated catch block
+            logger.error("send message error {} ", getExceptionStack(e));
+            syncCallables.remove(encodeObject.getMessageId());
+            return SendResult.THREAD_INTERRUPT;
+        } catch (ExecutionException e) {
+            // TODO Auto-generated catch block
+            logger.error("ExecutionException {} ", getExceptionStack(e));
+            syncCallables.remove(encodeObject.getMessageId());
+            return SendResult.UNKOWN_ERROR;
+        } catch (TimeoutException e) {
+            // TODO Auto-generated catch block
+            logger.error("TimeoutException {} ", getExceptionStack(e));
+            //e.printStackTrace();
+            SyncMessageCallable syncMessageCallable = syncCallables.remove(encodeObject.getMessageId());
+            if (syncMessageCallable != null) {
+                NettyClient tmpClient = syncMessageCallable.getClient();
+                if (tmpClient != null) {
+                    Channel curChannel = tmpClient.getChannel();
+                    if (curChannel != null) {
+                        logger.error("channel maybe busy {}", curChannel);
+                        scanThread.addTimeoutChannel(curChannel);
+                    }
+                }
+            }
+            return SendResult.TIMEOUT;
+        } catch (Throwable e) {
+            logger.error("syncSendMessage exception {} ", getExceptionStack(e));
+            syncCallables.remove(encodeObject.getMessageId());
+            return SendResult.UNKOWN_ERROR;
+        }
+        if (message == null) {
+            syncCallables.remove(encodeObject.getMessageId());
+            return SendResult.UNKOWN_ERROR;
+        }
+        scanThread.resetTimeoutChannel(client.getChannel());
+        if (message == SendResult.OK) {
+            metricWorker.recordSuccessByMessageId(encodeObject.getMessageId());
+        }
+        return message;
+    }
+
+
+    private SendResult syncSendMessageIndexInternal(NettyClient client,
+        EncodeObject encodeObject, String msgUUID, long timeout,
+        TimeUnit timeUnit) throws ExecutionException, InterruptedException, TimeoutException {
+        if (client == null || !client.isActive()) {
+            chooseProxy.remove(encodeObject.getMessageId());
+            client = clientMgr.getClientByRoundRobin();
+            if (client == null) {
+                return SendResult.NO_CONNECTION;
+            }
+            chooseProxy.put(encodeObject.getMessageId(), client);
+        }
+
+        if (encodeObject.getMsgtype() == 7) {
+            int bidnum = 0;
+            int tidnum = 0;
+            if (encodeObject.getBid().equals(clientMgr.getBid())) {
+                bidnum = clientMgr.getBidNum();
+                tidnum = clientMgr.getTidMap().get(encodeObject.getTid()) != null
+                    ? clientMgr.getTidMap().get(encodeObject.getTid()) : 0;
+            }
+            encodeObject.setBidNum(bidnum);
+            encodeObject.setTidNum(tidnum);
+            if (bidnum == 0 || tidnum == 0) {
+                encodeObject.setBidTransfer(false);
+            }
+        }
+        if (this.configure.isNeedDataEncry()) {
+            encodeObject.setEncryptEntry(true, configure.getUserName(), clientMgr.getEncryptConfigEntry());
+        } else {
+            encodeObject.setEncryptEntry(false, null, null);
+        }
+        encodeObject.setMsgUUID(msgUUID);
+        SyncMessageCallable callable = new SyncMessageCallable(client, encodeObject, timeout, timeUnit);
+        syncCallables.put(encodeObject.getMessageId(), callable);
+
+        Future<SendResult> future = threadPool.submit(callable);
+        return future.get(timeout, timeUnit);
+    }
+
+    /**
+     * sync send
+     * @param encodeObject
+     * @param msgUUID
+     * @param timeout
+     * @param timeUnit
+     * @return
+     */
+    public String syncSendMessageIndex(EncodeObject encodeObject, String msgUUID, long timeout,
+                                       TimeUnit timeUnit) {
+        try {
+            SendResult message = null;
+            NettyClient client = chooseProxy.get(encodeObject.getMessageId());
+            String proxyip = encodeObject.getProxyIp();
+            if (proxyip != null && proxyip.length() != 0) {
+                client = clientMgr.getContainProxy(proxyip);
+            }
+            if (isNotValidateAttr(encodeObject.getCommonattr(), encodeObject.getAttributes())) {
+                logger.error("error attr format {} {}", encodeObject.getCommonattr(),
+                        encodeObject.getAttributes());
+                return SendResult.INVALID_ATTRIBUTES.toString();
+            }
+            try {
+                message = syncSendMessageIndexInternal(client, encodeObject,
+                    msgUUID, timeout, timeUnit);
+            } catch (InterruptedException e) {
+                // TODO Auto-generated catch block
+                logger.error("send message error {}", getExceptionStack(e));
+                syncCallables.remove(encodeObject.getMessageId());
+                return SendResult.THREAD_INTERRUPT.toString();
+            } catch (ExecutionException e) {
+                // TODO Auto-generated catch block
+                logger.error("ExecutionException {}", getExceptionStack(e));
+                syncCallables.remove(encodeObject.getMessageId());
+                return SendResult.UNKOWN_ERROR.toString();
+            } catch (TimeoutException e) {
+                // TODO Auto-generated catch block
+                logger.error("TimeoutException {}", getExceptionStack(e));
+                //e.printStackTrace();
+                SyncMessageCallable syncMessageCallable = syncCallables.remove(encodeObject.getMessageId());
+                if (syncMessageCallable != null) {
+                    NettyClient tmpClient = syncMessageCallable.getClient();
+                    if (tmpClient != null) {
+                        Channel curChannel = tmpClient.getChannel();
+                        if (curChannel != null) {
+                            logger.error("channel maybe busy {}", curChannel);
+                            scanThread.addTimeoutChannel(curChannel);
+                        }
+                    }
+                }
+                return SendResult.TIMEOUT.toString();
+            } catch (Throwable e) {
+                logger.error("syncSendMessage exception {}", getExceptionStack(e));
+                syncCallables.remove(encodeObject.getMessageId());
+                return SendResult.UNKOWN_ERROR.toString();
+            }
+            scanThread.resetTimeoutChannel(client.getChannel());
+            return message.toString() + "=" + client.getServerIP();
+        } catch (Exception e) {
+            logger.error("agent send error {}", getExceptionStack(e));
+            syncCallables.remove(encodeObject.getMessageId());
+            return SendResult.UNKOWN_ERROR.toString();
+        }
+    }
+
+    /**
+     * async send message index
+     * @param encodeObject
+     * @param callback
+     * @param msgUUID
+     * @param timeout
+     * @param timeUnit
+     * @throws ProxysdkException
+     */
+    public void asyncSendMessageIndex(EncodeObject encodeObject,
+                                      FileCallback callback, String msgUUID,
+                                      long timeout, TimeUnit timeUnit) throws ProxysdkException {
+        NettyClient client = chooseProxy.get(encodeObject.getMessageId());
+        String proxyip = encodeObject.getProxyIp();
+        if (proxyip != null && proxyip.length() != 0) {
+            client = clientMgr.getContainProxy(proxyip);
+        }
+        if (client == null || !client.isActive()) {
+            chooseProxy.remove(encodeObject.getMessageId());
+            client = clientMgr.getClientByRoundRobin();
+            if (client == null) {
+                throw new ProxysdkException(SendResult.NO_CONNECTION.toString());
+            }
+            chooseProxy.put(encodeObject.getMessageId(), client);
+        }
+        if (currentBufferSize.get() >= asyncCallbackMaxSize) {
+            throw new ProxysdkException("ASYNC_CALLBACK_BUFFER_FULL");
+        }
+        int size = 1;
+        if (isFile) {
+            if (encodeObject.getBodyBytes() != null) {
+                size = encodeObject.getBodyBytes().length;
+            } else {
+                for (byte[] bytes : encodeObject.getBodylist()) {
+                    size = size + bytes.length;
+                }
+            }
+            if (currentBufferSize.addAndGet(size) >= asyncCallbackMaxSize) {
+                currentBufferSize.addAndGet(-size);
+                throw new ProxysdkException("ASYNC_CALLBACK_BUFFER_FULL");
+            }
+
+        } else {
+            if (currentBufferSize.incrementAndGet() >= asyncCallbackMaxSize) {
+                currentBufferSize.decrementAndGet();
+                throw new ProxysdkException("ASYNC_CALLBACK_BUFFER_FULL");
+            }
+        }
+        ConcurrentHashMap<String, QueueObject> tmpCallBackMap = new ConcurrentHashMap<>();
+        ConcurrentHashMap<String, QueueObject> msgQueueMap = callbacks.putIfAbsent(
+                client.getChannel(), tmpCallBackMap);
+        if (msgQueueMap == null) {
+            msgQueueMap = tmpCallBackMap;
+        }
+        msgQueueMap.put(encodeObject.getMessageId(), new QueueObject(System.currentTimeMillis(),
+                callback, size, timeout, timeUnit));
+        if (encodeObject.getMsgtype() == 7) {
+            int bidnum = 0;
+            int tidnum = 0;
+            if ((clientMgr.getBid().length() != 0) && (encodeObject.getBid().equals(clientMgr.getBid()))) {
+                bidnum = clientMgr.getBidNum();
+                tidnum = (clientMgr.getTidMap().get(encodeObject.getTid()) != null)
+                        ? clientMgr.getTidMap().get(encodeObject.getTid()) : 0;
+            }
+            encodeObject.setBidNum(bidnum);
+            encodeObject.setTidNum(tidnum);
+            if (bidnum == 0 || tidnum == 0) {
+                encodeObject.setBidTransfer(false);
+            }
+        }
+        if (this.configure.isNeedDataEncry()) {
+            encodeObject.setEncryptEntry(true, configure.getUserName(), clientMgr.getEncryptConfigEntry());
+        } else {
+            encodeObject.setEncryptEntry(false, null, null);
+        }
+        encodeObject.setMsgUUID(msgUUID);
+        client.write(encodeObject);
+    }
+
+    /**
+     * whether is validate
+     *
+     * @param commonAttr
+     * @param oldAttr
+     * @return
+     */
+    private boolean isNotValidateAttr(String commonAttr, String oldAttr) {
+        if (!StringUtils.isEmpty(commonAttr) && !validAttribute(commonAttr)) {
+            return true;
+        }
+        return !StringUtils.isEmpty(oldAttr) && !validAttribute(oldAttr);
+    }
+
+    /**
+     * validate attribute
+     *
+     * @param attr
+     * @return
+     */
+    private boolean validAttribute(String attr) {
+        boolean needEqual = true;
+        boolean needAnd = false;
+        for (int i = 0; i < attr.length(); i++) {
+            char item = attr.charAt(i);
+            if (item == '=') {
+                // if not must equal, then return false
+                if (!needEqual) {
+                    return false;
+                }
+                needEqual = false;
+                needAnd = true;
+            } else if (item == '&') {
+                // if not must and, then return false
+                if (!needAnd) {
+                    return false;
+                }
+                needAnd = false;
+                needEqual = true;
+            }
+        }
+        return !needEqual;
+    }
+
+
+    /* Following methods used by asynchronously message sending. */
+    public void asyncSendMessage(EncodeObject encodeObject,
+                                 SendMessageCallback callback, String msgUUID,
+                                 long timeout, TimeUnit timeUnit) throws ProxysdkException {
+        metricWorker.recordNumByKey(encodeObject.getMessageId(), encodeObject.getBid(),
+                encodeObject.getTid(), Utils.getLocalIp(), encodeObject.getPackageTime(),
+                encodeObject.getDt(), encodeObject.getRealCnt());
+
+        // send message package time
+
+        NettyClient client = clientMgr.getClientByRoundRobin();
+        if (client == null) {
+            throw new ProxysdkException(SendResult.NO_CONNECTION.toString());
+        }
+        if (currentBufferSize.get() >= asyncCallbackMaxSize) {
+            throw new ProxysdkException("ASYNC_CALLBACK_BUFFER_FULL");
+        }
+        if (isNotValidateAttr(encodeObject.getCommonattr(), encodeObject.getAttributes())) {
+            logger.error("error attr format {} {}", encodeObject.getCommonattr(),
+                    encodeObject.getAttributes());
+            throw new ProxysdkException(SendResult.INVALID_ATTRIBUTES.toString());
+        }
+        int size = 1;
+        if (isFile) {
+            if (encodeObject.getBodyBytes() != null) {
+                size = encodeObject.getBodyBytes().length;
+            } else {
+                for (byte[] bytes : encodeObject.getBodylist()) {
+                    size = size + bytes.length;
+                }
+            }
+            if (currentBufferSize.addAndGet(size) >= asyncCallbackMaxSize) {
+                currentBufferSize.addAndGet(-size);
+                throw new ProxysdkException("ASYNC_CALLBACK_BUFFER_FULL");
+            }
+
+        } else {
+            if (currentBufferSize.incrementAndGet() >= asyncCallbackMaxSize) {
+                currentBufferSize.decrementAndGet();
+                throw new ProxysdkException("ASYNC_CALLBACK_BUFFER_FULL");
+            }
+        }
+        ConcurrentHashMap<String, QueueObject> tmpQueueMap = new ConcurrentHashMap<>();
+        ConcurrentHashMap<String, QueueObject> msgQueueMap =
+                callbacks.putIfAbsent(client.getChannel(), tmpQueueMap);
+        if (msgQueueMap == null) {
+            msgQueueMap = tmpQueueMap;
+        }
+        QueueObject queueObject = msgQueueMap.putIfAbsent(
+                encodeObject.getMessageId(),
+                new QueueObject(System.currentTimeMillis(), callback, size, timeout, timeUnit));
+        if (queueObject != null) {
+            logger.warn("message id {} has existed.", encodeObject.getMessageId());
+        }
+        if (encodeObject.getMsgtype() == 7) {
+            int bidnum = 0;
+            int tidnum = 0;
+            if ((clientMgr.getBid().length() != 0) && (encodeObject.getBid().equals(clientMgr.getBid()))) {
+                bidnum = clientMgr.getBidNum();
+                tidnum = (clientMgr.getTidMap().get(encodeObject.getTid()) != null) ? clientMgr
+                        .getTidMap().get(encodeObject.getTid()) : 0;
+            }
+            encodeObject.setBidNum(bidnum);
+            encodeObject.setTidNum(tidnum);
+            if (bidnum == 0 || tidnum == 0) {
+                encodeObject.setBidTransfer(false);
+            }
+        }
+        if (this.configure.isNeedDataEncry()) {
+            encodeObject.setEncryptEntry(true, configure.getUserName(), clientMgr.getEncryptConfigEntry());
+        } else {
+            encodeObject.setEncryptEntry(false, null, null);
+        }
+        encodeObject.setMsgUUID(msgUUID);
+        client.write(encodeObject);
+    }
+
+    /* Deal with feedback. */
+    public void notifyFeedback(Channel channel, EncodeObject response) {
+        String messageId = response.getMessageId();
+        chooseProxy.remove(messageId);
+        SyncMessageCallable callable = syncCallables.remove(messageId);
+        SendResult result = response.isException() ? SendResult.INVALID_ATTRIBUTES : SendResult.OK;
+        if (result == SendResult.OK) {
+            metricWorker.recordSuccessByMessageId(messageId);
+        }
+        if (callable != null) {
+            callable.update(result);
+        }
+        if (response.isException()) {
+            logger.error("{} exception happens, error message {}", channel,
+                    response.getExceptionError());
+        }
+        notifyCallback(channel, messageId, result);
+    }
+
+    /*
+     * deal with connection disconnection, should we restore it and
+     * re-send on a new channel?
+     */
+    public void notifyConnectionDisconnected(Channel channel) {
+        if (channel == null) {
+            return;
+        }
+        logger.info("channel {} connection is disconnected!", channel);
+        try {
+            ConcurrentHashMap<String, QueueObject> msgQueueMap = callbacks.remove(channel);
+            if (msgQueueMap != null) {
+                for (String messageId : msgQueueMap.keySet()) {
+                    QueueObject queueObject = msgQueueMap.remove(messageId);
+                    if (queueObject == null) {
+                        continue;
+                    }
+                    if (isFile) {
+                        ((FileCallback) queueObject.getCallback()).onMessageAck(SendResult.CONNECTION_BREAK.toString());
+                        currentBufferSize.addAndGet(-queueObject.getSize());
+                    } else {
+                        queueObject.getCallback().onMessageAck(SendResult.CONNECTION_BREAK);
+                        currentBufferSize.decrementAndGet();
+                    }
+                }
+                msgQueueMap.clear();
+            }
+        } catch (Throwable e2) {
+            logger.info("process channel {} disconnected callbacks throw error,", channel, e2);
+        }
+
+        try {
+            for (String messageId : syncCallables.keySet()) {
+                if (messageId == null) {
+                    continue;
+                }
+                SyncMessageCallable messageCallable = syncCallables.get(messageId);
+                if (messageCallable == null) {
+                    continue;
+                }
+                NettyClient nettyClient = messageCallable.getClient();
+                if (nettyClient == null) {
+                    continue;
+                }
+                Channel netChannel1 = nettyClient.getChannel();
+                if (netChannel1 == null) {
+                    continue;
+                }
+                if (netChannel1.getId().equals(channel.getId())) {
+                    messageCallable.update(SendResult.CONNECTION_BREAK);
+                    syncCallables.remove(messageId);
+                    break;
+                }
+            }
+        } catch (Throwable e) {
+            logger.info("process channel {} disconnected syncCallables throw error,", channel, e);
+        }
+    }
+
+    /* Deal with unexpected exception. only used for asyc send*/
+    public void waitForAckForChannel(Channel channel) {
+        if (channel == null) {
+            return;
+        }
+        logger.info("wait for ack for channel {}", channel);
+        try {
+            ConcurrentHashMap<String, QueueObject> queueObjMap = callbacks.get(channel);
+            if (queueObjMap != null) {
+                while (true) {
+                    if (queueObjMap.isEmpty()) {
+                        logger.info("this channel {} is empty!", channel);
+                        break;
+                    }
+                    try {
+                        Thread.sleep(1000);
+                    } catch (InterruptedException e) {
+                        // TODO Auto-generated catch block
+                        logger.error("wait for ack for channel {}, error {}",
+                                channel, e.getMessage());
+                        e.printStackTrace();
+                    }
+                }
+            }
+            logger.info("waitForAckForChannel finished , channel is {}", channel);
+        } catch (Throwable e) {
+            logger.error("waitForAckForChannel exception, channel is {}", channel, e);
+        }
+    }
+
+    public void clearCallBack() {
+        currentBufferSize.set(0);
+        callbacks.clear();
+    }
+
+    public String getClusterId() {
+        return clusterId;
+    }
+
+    public void setClusterId(String clusterId) {
+        this.clusterId = clusterId;
+    }
+
+
+}
diff --git a/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/network/SequentialID.java b/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/network/SequentialID.java
new file mode 100644
index 0000000..636b0ca
--- /dev/null
+++ b/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/network/SequentialID.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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.dataproxy.network;
+
+import java.util.concurrent.atomic.AtomicLong;
+
+public class SequentialID {
+    private static final long maxId = 2000000000;
+    private String ip = null;
+    private AtomicLong id = new AtomicLong(0);
+
+    public SequentialID(String theIp) {
+        ip = theIp;
+    }
+
+    public synchronized String getNextId() {
+        if (id.get() > maxId) {
+            id.set(0);
+        }
+        id.incrementAndGet();
+        return ip + "#" + id.toString() + "#" + System.currentTimeMillis();
+    }
+
+    public synchronized long getNextInt() {
+        if (id.get() > maxId) {
+            id.set(0);
+        }
+        id.incrementAndGet();
+        return id.get();
+    }
+
+}
diff --git a/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/network/SyncMessageCallable.java b/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/network/SyncMessageCallable.java
new file mode 100644
index 0000000..80bb466
--- /dev/null
+++ b/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/network/SyncMessageCallable.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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.dataproxy.network;
+
+import java.util.concurrent.Callable;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.inlong.dataproxy.SendResult;
+import org.apache.inlong.dataproxy.codec.EncodeObject;
+import org.jboss.netty.channel.ChannelFuture;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SyncMessageCallable implements Callable<SendResult> {
+    private static final Logger logger = LoggerFactory
+            .getLogger(SyncMessageCallable.class);
+
+    private final NettyClient client;
+    private final CountDownLatch awaitLatch = new CountDownLatch(1);
+    private final EncodeObject encodeObject;
+    private final long timeout;
+    private final TimeUnit timeUnit;
+
+    private SendResult message;
+
+    public SyncMessageCallable(NettyClient client, EncodeObject encodeObject,
+                               long timeout, TimeUnit timeUnit) {
+        this.client = client;
+        this.encodeObject = encodeObject;
+        this.timeout = timeout;
+        this.timeUnit = timeUnit;
+    }
+
+    public void update(SendResult message) {
+        this.message = message;
+        awaitLatch.countDown();
+    }
+
+    public SendResult call() throws Exception {
+        // TODO Auto-generated method stub
+        try {
+            ChannelFuture channelFuture = client.write(encodeObject);
+            awaitLatch.await(timeout, timeUnit);
+        } catch (Exception e) {
+            logger.error("SendResult call", e);
+            e.printStackTrace();
+            return SendResult.UNKOWN_ERROR;
+        }
+        return message;
+    }
+
+    public NettyClient getClient() {
+        return client;
+    }
+}
\ No newline at end of file
diff --git a/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/network/TimeScanObject.java b/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/network/TimeScanObject.java
new file mode 100644
index 0000000..36c4574
--- /dev/null
+++ b/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/network/TimeScanObject.java
@@ -0,0 +1,59 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.dataproxy.network;
+
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ * Created by jesseyzhou on 2017/11/2.
+ */
+public class TimeScanObject {
+
+    private AtomicInteger count = new AtomicInteger(0);
+    private AtomicLong time = new AtomicLong(0);
+
+    public TimeScanObject() {
+        this.count.set(0);
+        this.time.set(System.currentTimeMillis());
+    }
+
+    public int incrementAndGet() {
+        this.time.set(System.currentTimeMillis());
+        return this.count.incrementAndGet();
+    }
+
+    public long getTime() {
+        return time.get();
+    }
+
+    public void updateCountToZero() {
+        long oldValue = this.time.get();
+        int oldCnt = this.count.get();
+        if (System.currentTimeMillis() > oldValue) {
+            if (this.time.compareAndSet(oldValue, System.currentTimeMillis())) {
+                this.count.compareAndSet(oldCnt, 0);
+            }
+        }
+    }
+
+    public int getCurTimeoutCount() {
+        return this.count.get();
+    }
+
+}
diff --git a/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/network/Utils.java b/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/network/Utils.java
new file mode 100644
index 0000000..95acdaa
--- /dev/null
+++ b/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/network/Utils.java
@@ -0,0 +1,188 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.dataproxy.network;
+
+import java.io.UnsupportedEncodingException;
+import java.net.DatagramSocket;
+import java.net.InetAddress;
+import java.net.URLEncoder;
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.commons.codec.binary.Base64;
+import org.apache.commons.codec.digest.HmacUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class Utils {
+    private static final Logger logger = LoggerFactory.getLogger(Utils.class);
+    private static String userIp;
+
+    static {
+        userIp = getLocalIp();
+    }
+
+    public static String getLocalIp() {
+        if (userIp != null) {
+            return userIp;
+        }
+        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 ignored) {
+            logger.warn("getLocalIp ", ignored);
+        }
+        userIp = ip;
+        return ip;
+    }
+
+    public static boolean validLocalIp(String currLocalHost) throws ProxysdkException {
+        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);
+        }
+        if (!ip.equals(currLocalHost)) {
+            logger.warn("ip is not equal {} {}", currLocalHost, ip);
+        }
+        userIp = ip;
+        return true;
+    }
+
+    public static String getMD5String(String source) {
+        final char[] hexDigits = {'0', '1', '2', '3', '4', '5', '6', '7', '8', '9', 'a', 'b', 'c', 'd', 'e', 'f'};
+        String retString = null;
+
+        if (source == null) {
+            return retString;
+        }
+
+        try {
+            StringBuilder sb = new StringBuilder();
+            MessageDigest md = MessageDigest.getInstance("MD5");
+            try {
+                md.update(source.getBytes("utf8"), 0, source.length());
+            } catch (UnsupportedEncodingException e) {
+                e.printStackTrace();
+            }
+            byte[] retBytes = md.digest();
+            for (byte b : retBytes) {
+                sb.append(hexDigits[(b >> 4) & 0x0f]);
+                sb.append(hexDigits[b & 0x0f]);
+            }
+
+            retString = sb.toString();
+        } catch (NoSuchAlgorithmException e) {
+            logger.error("" + e);
+        }
+
+        return retString;
+    }
+
+    public static boolean isNotBlank(String str) {
+        return !isBlank(str);
+    }
+
+    public static boolean isBlank(String str) {
+        int strLen;
+        if (str == null || (strLen = str.length()) == 0) {
+            return true;
+        }
+        for (int i = 0; i < strLen; i++) {
+            if ((!Character.isWhitespace(str.charAt(i)))) {
+                return false;
+            }
+        }
+        return true;
+    }
+
+    public static byte[] toBytes(String ipAddr) {
+        byte[] ret = new byte[4];
+        try {
+            String[] ipArr = ipAddr.split("\\.");
+            ret[0] = (byte) (Integer.parseInt(ipArr[0]) & 0xFF);
+            ret[1] = (byte) (Integer.parseInt(ipArr[1]) & 0xFF);
+            ret[2] = (byte) (Integer.parseInt(ipArr[2]) & 0xFF);
+            ret[3] = (byte) (Integer.parseInt(ipArr[3]) & 0xFF);
+            return ret;
+        } catch (Exception e) {
+            throw new IllegalArgumentException(ipAddr + " is invalid IP");
+        }
+    }
+
+    public static int bytesToInt(byte[] bytes) {
+        int addr = bytes[3] & 0xFF;
+        addr |= ((bytes[2] << 8) & 0xFF00);
+        addr |= ((bytes[1] << 16) & 0xFF0000);
+        addr |= ((bytes[0] << 24) & 0xFF000000);
+        return addr;
+    }
+
+    public static String convertListToString(List<String> list, Character ch) {
+        if (list == null || list.isEmpty()) {
+            return "";
+        }
+        StringBuilder sb = new StringBuilder();
+        Iterator itr = list.iterator();
+        sb.append(itr.next());
+        while (itr.hasNext()) {
+            sb.append(ch).append(itr.next());
+        }
+        return sb.toString();
+    }
+
+    public static String generateSignature(String secureId, long timestamp, int randomValue, String secureKey) {
+        Base64 base64 = new Base64();
+        byte[] baseStr = base64.encode(HmacUtils.hmacSha1(secureKey, secureId + timestamp + randomValue));
+        String result = "";
+        try {
+            result = URLEncoder.encode(new String(baseStr), "UTF-8");
+        } catch (UnsupportedEncodingException e) {
+            e.printStackTrace();
+        }
+        return result;
+    }
+
+
+    public static String getAuthorizenInfo(final String secretId, final String secretKey, long timestamp, int nonce) {
+        String signature = generateSignature(secretId, timestamp, nonce, secretKey);
+        return "manager " + secretId + " " + timestamp + " " + nonce + " " + signature;
+    }
+
+
+    public static String convertSetToString(Set<String> list, Character ch) {
+        if (list == null || list.isEmpty()) {
+            return "";
+        }
+        StringBuilder sb = new StringBuilder();
+        Iterator itr = list.iterator();
+        sb.append(itr.next());
+        while (itr.hasNext()) {
+            sb.append(ch).append(itr.next());
+        }
+        return sb.toString();
+    }
+
+}
diff --git a/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/threads/IndexCollectThread.java b/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/threads/IndexCollectThread.java
new file mode 100644
index 0000000..5ec7322
--- /dev/null
+++ b/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/threads/IndexCollectThread.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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.dataproxy.threads;
+
+import java.text.DateFormat;
+import java.text.SimpleDateFormat;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * index collector
+ */
+public class IndexCollectThread extends Thread {
+    private static final Logger logger = LoggerFactory.getLogger(IndexCollectThread.class);
+
+    private volatile boolean bShutDown;
+    private final Map<String, Long> storeIndex;
+
+    public IndexCollectThread(Map<String, Long> storeIndex) {
+        bShutDown = false;
+        this.storeIndex = storeIndex;
+        this.setDaemon(true);
+        this.setName("IndexCollectThread");
+    }
+
+    public void shutDown() {
+        logger.info("begin to shut down IndexCollectThread!");
+        bShutDown = true;
+    }
+
+    @Override
+    public void run() {
+        logger.info("IndexCollectThread Thread=" + Thread.currentThread().getId() + " started !");
+        while (!bShutDown) {
+            try {
+                TimeUnit.MILLISECONDS.sleep(60 * 1000);
+                DateFormat sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
+
+                for (Map.Entry<String, Long> entry : storeIndex.entrySet()) {
+                    String key = entry.getKey();
+                    Long val = entry.getValue();
+                    key = "|" + sdf.format(System.currentTimeMillis()) + "|" + key;
+                    logger.info("Monitor {} send message {}", key, val);
+                    entry.setValue(0L);
+                }
+            } catch (Exception e) {
+                if (!bShutDown) {
+                    logger.error("IndexCollectThread exception", e);
+                }
+            }
+        }
+    }
+}
diff --git a/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/threads/ManagerFetcherThread.java b/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/threads/ManagerFetcherThread.java
new file mode 100644
index 0000000..4eb90bc
--- /dev/null
+++ b/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/threads/ManagerFetcherThread.java
@@ -0,0 +1,66 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.dataproxy.threads;
+
+import java.util.concurrent.TimeUnit;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.inlong.dataproxy.ProxyClientConfig;
+import org.apache.inlong.dataproxy.utils.ServiceDiscoveryUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * manager fetch thread
+ */
+public class ManagerFetcherThread extends Thread {
+    private final Logger logger = LoggerFactory.getLogger(ManagerFetcherThread.class);
+    private volatile boolean isShutdown;
+    private final ProxyClientConfig proxyClientConfig;
+
+    public ManagerFetcherThread(ProxyClientConfig proxyClientConfig) {
+        isShutdown = false;
+        this.proxyClientConfig = proxyClientConfig;
+        this.setDaemon(true);
+        this.setName("ManagerFetcherThread");
+    }
+
+    public void shutdown() {
+        logger.info("Begin to shutdown ManagerFetcherThread.");
+        isShutdown = true;
+    }
+
+    @Override
+    public void run() {
+        logger.info("ManagerFetcherThread Thread=" + Thread.currentThread().getId() + " started !");
+        while (!isShutdown) {
+            try {
+                String managerIpList = ServiceDiscoveryUtils.getManagerIpList(proxyClientConfig);
+                if (StringUtils.isBlank(managerIpList)) {
+                    logger.error("ManagerFetcher get managerIpList is blank.");
+                } else {
+                    ServiceDiscoveryUtils.updateManagerInfo2Local(managerIpList,
+                        proxyClientConfig.getManagerIpLocalPath());
+                }
+                TimeUnit.MILLISECONDS.sleep((long) proxyClientConfig.getProxyUpdateIntervalMinutes() * 60 * 1000);
+            } catch (Throwable e) {
+                logger.error("ManagerFetcher get or save managerIpList occur error,", e);
+            }
+        }
+    }
+}
\ No newline at end of file
diff --git a/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/threads/MetricWorkerThread.java b/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/threads/MetricWorkerThread.java
new file mode 100644
index 0000000..fbca3bb
--- /dev/null
+++ b/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/threads/MetricWorkerThread.java
@@ -0,0 +1,320 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.dataproxy.threads;
+
+import java.io.Closeable;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+import org.apache.inlong.dataproxy.ProxyClientConfig;
+import org.apache.inlong.dataproxy.FileCallback;
+import org.apache.inlong.dataproxy.SendResult;
+import org.apache.inlong.dataproxy.codec.EncodeObject;
+import org.apache.inlong.dataproxy.metric.MessageRecord;
+import org.apache.inlong.dataproxy.metric.MetricTimeNumSummary;
+import org.apache.inlong.dataproxy.network.Sender;
+import org.apache.inlong.dataproxy.network.SequentialID;
+import org.apache.inlong.dataproxy.network.Utils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * metric worker
+ */
+public class MetricWorkerThread extends Thread implements Closeable {
+    private final Logger logger = LoggerFactory.getLogger(MetricWorkerThread.class);
+
+    private final SequentialID idGenerator = new SequentialID(Utils.getLocalIp());
+
+
+    private final ConcurrentHashMap<String, MessageRecord> metricValueCache = new ConcurrentHashMap<>();
+
+
+    private final ConcurrentHashMap<String, MetricTimeNumSummary> metricPackTimeMap = new ConcurrentHashMap<>();
+
+
+    private final ConcurrentHashMap<String, MetricTimeNumSummary> metricDtMap = new ConcurrentHashMap<>();
+
+
+    private static final String DEFAULT_KEY_ITEM = "";
+    private static final String DEFAULT_KEY_SPLITTER = "#";
+    private final ProxyClientConfig proxyClientConfig;
+    private volatile boolean bShutdown = false;
+    private final long delayTime;
+    private final Sender sender;
+    private final boolean enableSlaMetric;
+    private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+
+    public MetricWorkerThread(ProxyClientConfig proxyClientConfig, Sender sender) {
+        this.proxyClientConfig = proxyClientConfig;
+        this.enableSlaMetric = proxyClientConfig.isEnableSlaMetric();
+
+        this.delayTime = 20 * 1000;
+        this.sender = sender;
+        this.setDaemon(true);
+        this.setName("MetricWorkerThread");
+    }
+
+    public long getFormatKeyTime(long keyTime) {
+        return keyTime - keyTime % proxyClientConfig.getMetricIntervalInMs();
+    }
+
+    /**
+     * get string key
+     *
+     * @param bid     - bid
+     * @param tid     - tid
+     * @param localIp - ip
+     * @return
+     */
+    private String getKeyStringByConfig(String bid, String tid, String localIp, long keyTime) {
+        StringBuilder builder = new StringBuilder();
+        String bidStr = proxyClientConfig.isUseBidAsKey() ? bid : DEFAULT_KEY_ITEM;
+        String tidStr = proxyClientConfig.isUseTidAsKey() ? tid : DEFAULT_KEY_ITEM;
+        String localIpStr = proxyClientConfig.isUseLocalIpAsKey() ? localIp : DEFAULT_KEY_ITEM;
+
+        builder.append(bidStr).append(DEFAULT_KEY_SPLITTER)
+                .append(tidStr).append(DEFAULT_KEY_SPLITTER)
+                .append(localIpStr).append(DEFAULT_KEY_SPLITTER)
+                .append(keyTime);
+        return builder.toString();
+    }
+
+    /**
+     * record num
+     *
+     * @param msgId    - msg uuid
+     * @param bid      - bid
+     * @param tid      - tid
+     * @param localIp  - ip
+     * @param packTime - package time
+     * @param dt       - dt
+     * @param num      - num
+     */
+    public void recordNumByKey(String msgId, String bid, String tid, String localIp, long packTime, long dt, int num) {
+        if (!enableSlaMetric) {
+            return;
+        }
+        MessageRecord messageRecord = new MessageRecord(bid, tid, localIp, msgId,
+                getFormatKeyTime(dt), getFormatKeyTime(packTime), num);
+
+        metricValueCache.putIfAbsent(msgId, messageRecord);
+    }
+
+    private MetricTimeNumSummary getMetricSummary(String keyName, MetricTimeNumSummary summary,
+                                                  ConcurrentHashMap<String, MetricTimeNumSummary> cacheMap) {
+        MetricTimeNumSummary finalSummary = cacheMap.putIfAbsent(keyName, summary);
+        if (finalSummary == null) {
+            finalSummary = summary;
+        }
+        return finalSummary;
+    }
+
+    /**
+     * record success num
+     *
+     * @param msgId - msg id
+     */
+    public void recordSuccessByMessageId(String msgId) {
+        if (!enableSlaMetric) {
+            return;
+        }
+        MessageRecord messageRecord = metricValueCache.remove(msgId);
+        if (messageRecord != null) {
+
+            String packTimeKeyName = getKeyStringByConfig(messageRecord.getBid(), messageRecord.getTid(),
+                    messageRecord.getLocalIp(), messageRecord.getPackTime());
+            String dtKeyName = getKeyStringByConfig(messageRecord.getBid(), messageRecord.getTid(),
+                    messageRecord.getLocalIp(), messageRecord.getDt());
+
+
+            MetricTimeNumSummary packTimeSummary = getMetricSummary(packTimeKeyName,
+                    new MetricTimeNumSummary(messageRecord.getPackTime()), metricPackTimeMap);
+
+            MetricTimeNumSummary dtSummary = getMetricSummary(dtKeyName,
+                    new MetricTimeNumSummary(messageRecord.getDt()), metricDtMap);
+
+            packTimeSummary.recordSuccessSendTime(messageRecord.getMessageTime(), messageRecord.getMsgCount());
+            dtSummary.increaseSuccessNum(messageRecord.getMsgCount());
+        }
+    }
+
+    /**
+     * record failed num
+     *
+     * @param msgId - msg id
+     */
+    public void recordFailedByMessageId(String msgId) {
+        MessageRecord messageRecord = metricValueCache.remove(msgId);
+        if (messageRecord != null) {
+
+            String packTimeKeyName = getKeyStringByConfig(messageRecord.getBid(), messageRecord.getTid(),
+                    messageRecord.getLocalIp(), messageRecord.getPackTime());
+            String dtKeyName = getKeyStringByConfig(messageRecord.getBid(), messageRecord.getTid(),
+                    messageRecord.getLocalIp(), messageRecord.getDt());
+
+
+            MetricTimeNumSummary packTimeSummary = getMetricSummary(packTimeKeyName,
+                    new MetricTimeNumSummary(messageRecord.getMessageTime()), metricPackTimeMap);
+
+            MetricTimeNumSummary dtSummary = getMetricSummary(dtKeyName,
+                    new MetricTimeNumSummary(messageRecord.getDt()), metricDtMap);
+
+            packTimeSummary.increaseFailedNum(messageRecord.getMsgCount());
+            dtSummary.increaseFailedNum(messageRecord.getMsgCount());
+        }
+    }
+
+    @Override
+    public void close() {
+        bShutdown = false;
+        flushMetric(true);
+    }
+
+    @Override
+    public void run() {
+        logger.info("MetricWorkerThread Thread=" + Thread.currentThread().getId() + " started !");
+        while (!bShutdown) {
+            // check metric
+            try {
+                checkCacheRecords();
+                flushMetric(false);
+                TimeUnit.MILLISECONDS.sleep(proxyClientConfig.getMetricIntervalInMs());
+            } catch (Exception ex) {
+                // exception happens
+            }
+        }
+    }
+
+    private void tryToSendMetricToManager(EncodeObject encodeObject, MetricSendCallBack callBack) {
+        callBack.increaseRetry();
+        try {
+
+            if (callBack.getRetryCount() < 4) {
+                sender.asyncSendMessageIndex(encodeObject, callBack,
+                        String.valueOf(System.currentTimeMillis()), 20, TimeUnit.SECONDS);
+            } else {
+                logger.error("error while sending {} {}", encodeObject.getBodyBytes(), encodeObject.getBodylist());
+            }
+        } catch (Exception ex) {
+            logger.warn("exception caught {}", ex.getMessage());
+            tryToSendMetricToManager(encodeObject, callBack);
+        }
+    }
+
+    private void sendSingleLine(String line, String tid, long dtTime) {
+        EncodeObject encodeObject = new EncodeObject(line.getBytes(), 7,
+                false, false, false,
+                dtTime, idGenerator.getNextInt(),
+                proxyClientConfig.getMetricBid(), tid, "", "", Utils.getLocalIp());
+        MetricSendCallBack callBack = new MetricSendCallBack(encodeObject);
+        tryToSendMetricToManager(encodeObject, callBack);
+    }
+
+    private void flushMapRecords(boolean isClosing, ConcurrentHashMap<String, MetricTimeNumSummary> cacheMap) {
+        for (String keyName : cacheMap.keySet()) {
+            MetricTimeNumSummary summary = cacheMap.get(keyName);
+            if (isClosing || (summary != null && summary.getSummaryTime()
+                    + delayTime > proxyClientConfig.getMetricIntervalInMs())) {
+                summary = cacheMap.remove(keyName);
+                if (summary != null) {
+                    long metricDtTime = summary.getStartCalculateTime() / 1000;
+                    // send to manager cluster.
+                    String countLine = keyName + DEFAULT_KEY_SPLITTER + summary.getSuccessNum()
+                            + DEFAULT_KEY_SPLITTER + summary.getFailedNum()
+                            + DEFAULT_KEY_SPLITTER + summary.getTotalNum();
+                    String timeLine = keyName + DEFAULT_KEY_SPLITTER + summary.getTimeString();
+
+                    logger.info("sending {}", countLine);
+                    logger.info("sending {}", timeLine);
+                    sendSingleLine(countLine, "count", metricDtTime);
+                    sendSingleLine(timeLine, "time", metricDtTime);
+                }
+            }
+        }
+    }
+
+    /**
+     * flush records
+     */
+    private void flushRecords(boolean isClosing) {
+        flushMapRecords(isClosing, metricDtMap);
+        flushMapRecords(isClosing, metricPackTimeMap);
+    }
+
+    /**
+     * check cache records
+     */
+    private void checkCacheRecords() {
+        for (String msgId : metricValueCache.keySet()) {
+            MessageRecord record = metricValueCache.get(msgId);
+
+            if (record != null && record.getMessageTime() + delayTime > proxyClientConfig.getMetricIntervalInMs()) {
+                recordFailedByMessageId(msgId);
+            }
+        }
+    }
+
+    /**
+     * flush metric
+     *
+     * @param isClosing - whether is closing
+     */
+    private void flushMetric(boolean isClosing) {
+        lock.writeLock().lock();
+        try {
+
+            flushRecords(isClosing);
+        } finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    private class MetricSendCallBack extends FileCallback {
+
+        private final EncodeObject encodeObject;
+        private int retryCount = 0;
+
+        public void increaseRetry() {
+            retryCount += 1;
+        }
+
+        public int getRetryCount() {
+            return retryCount;
+        }
+
+        public MetricSendCallBack(EncodeObject encodeObject) {
+            this.encodeObject = encodeObject;
+        }
+
+        @Override
+        public void onMessageAck(String result) {
+            if (!SendResult.OK.toString().equals(result)) {
+                tryToSendMetricToManager(encodeObject, this);
+            } else {
+                logger.info("metric is ok");
+            }
+        }
+
+        @Override
+        public void onMessageAck(SendResult result) {
+
+        }
+    }
+}
diff --git a/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/threads/TimeoutScanThread.java b/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/threads/TimeoutScanThread.java
new file mode 100644
index 0000000..5b35309
--- /dev/null
+++ b/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/threads/TimeoutScanThread.java
@@ -0,0 +1,186 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.dataproxy.threads;
+
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.inlong.dataproxy.ProxyClientConfig;
+import org.apache.inlong.dataproxy.FileCallback;
+import org.apache.inlong.dataproxy.SendResult;
+import org.apache.inlong.dataproxy.network.ClientMgr;
+import org.apache.inlong.dataproxy.network.QueueObject;
+import org.apache.inlong.dataproxy.network.TimeScanObject;
+import org.jboss.netty.channel.Channel;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Daemon threads to check timeout for asynchronous callback.
+ */
+public class TimeoutScanThread extends Thread {
+    private final Logger logger = LoggerFactory.getLogger(TimeoutScanThread.class);
+    private volatile boolean bShutDown = false;
+    private long printCount = 0;
+
+    private final ConcurrentHashMap<Channel, ConcurrentHashMap<String, QueueObject>> callbacks;
+    private final AtomicInteger currentBufferSize;
+    private final ProxyClientConfig config;
+    private final ClientMgr clientMgr;
+    private final ConcurrentHashMap<Channel, TimeScanObject> timeoutChannelStat = new ConcurrentHashMap<>();
+    private static final int MAX_CHANNEL_TIMEOUT = 5 * 60 * 1000;
+
+    public TimeoutScanThread(ConcurrentHashMap<Channel, ConcurrentHashMap<String, QueueObject>> callbacks,
+                             AtomicInteger currentBufferSize, ProxyClientConfig config, ClientMgr clientMgr) {
+        bShutDown = false;
+        printCount = 0;
+        this.callbacks = callbacks;
+        this.currentBufferSize = currentBufferSize;
+        this.config = config;
+        this.clientMgr = clientMgr;
+        this.setDaemon(true);
+        this.setName("TimeoutScanThread");
+    }
+
+    public void shutDown() {
+        logger.info("begin to shut down TimeoutScanThread!");
+        bShutDown = true;
+    }
+
+    /**
+     * add timeout channel
+     *
+     * @param channel
+     */
+    public void addTimeoutChannel(Channel channel) {
+        if (channel != null) {
+            TimeScanObject timeScanObject = timeoutChannelStat.get(channel);
+            if (timeScanObject == null) {
+                TimeScanObject tmpTimeObj = new TimeScanObject();
+                timeScanObject = timeoutChannelStat.putIfAbsent(channel, tmpTimeObj);
+                if (timeScanObject == null) {
+                    timeScanObject = tmpTimeObj;
+                }
+            }
+            timeScanObject.incrementAndGet();
+        }
+    }
+
+    /**
+     * reset channel timeout
+     *
+     * @param channel
+     */
+    public void resetTimeoutChannel(Channel channel) {
+        if (channel != null) {
+            TimeScanObject timeScanObject = timeoutChannelStat.get(channel);
+            if (timeScanObject != null) {
+                timeScanObject.updateCountToZero();
+            }
+        }
+    }
+
+    /**
+     * check timeout
+     */
+    private void checkTimeoutChannel() {
+        //if timeout >3,set channel busy
+        for (Channel tmpChannel : timeoutChannelStat.keySet()) {
+            TimeScanObject timeScanObject = tmpChannel != null ? timeoutChannelStat.get(tmpChannel) : null;
+            if (timeScanObject == null) {
+                continue;
+            }
+
+            if (System.currentTimeMillis() - timeScanObject.getTime() > MAX_CHANNEL_TIMEOUT) {
+                timeoutChannelStat.remove(tmpChannel);
+            } else {
+
+                if (timeScanObject.getCurTimeoutCount() > config.getMaxTimeoutCnt()) {
+                    timeoutChannelStat.remove(tmpChannel);
+                    if (tmpChannel.isOpen() && tmpChannel.isConnected()) {
+                        clientMgr.setConnectionBusy(tmpChannel);
+                        logger.error("this client {} is busy!", tmpChannel);
+                    }
+                }
+            }
+        }
+    }
+
+    /**
+     * check message id
+     *
+     * @param channel
+     * @param messageIdCallbacks
+     */
+    private void checkMessageIdBasedCallbacks(Channel channel,
+                                              ConcurrentHashMap<String, QueueObject> messageIdCallbacks) {
+        for (String messageId : messageIdCallbacks.keySet()) {
+            QueueObject queueObject = messageId != null ? messageIdCallbacks.get(messageId) : null;
+            if (queueObject == null) {
+                continue;
+            }
+            // if queueObject timeout
+            if (System.currentTimeMillis() - queueObject.getSendTimeInMillis()
+                    >= queueObject.getTimeoutInMillis()) {
+                // remove it before callback
+                QueueObject queueObject1 = messageIdCallbacks.remove(messageId);
+                if (queueObject1 != null) {
+                    if (config.isFile()) {
+                        ((FileCallback) queueObject1.getCallback()).onMessageAck(SendResult.TIMEOUT.toString());
+                        currentBufferSize.addAndGet(-queueObject1.getSize());
+                    } else {
+                        queueObject1.getCallback().onMessageAck(SendResult.TIMEOUT);
+                        currentBufferSize.decrementAndGet();
+                    }
+                }
+                addTimeoutChannel(channel);
+            }
+        }
+    }
+
+    @Override
+    public void run() {
+        logger.info("TimeoutScanThread Thread=" + Thread.currentThread().getId() + " started !");
+        while (!bShutDown) {
+            try {
+                for (Channel channel : callbacks.keySet()) {
+                    ConcurrentHashMap<String, QueueObject> msgQueueMap =
+                            channel != null ? callbacks.get(channel) : null;
+                    if (msgQueueMap == null) {
+                        continue;
+                    }
+                    checkMessageIdBasedCallbacks(channel, msgQueueMap);
+                }
+                checkTimeoutChannel();
+                TimeUnit.SECONDS.sleep(1);
+            } catch (Throwable e) {
+                if (!bShutDown) {
+                    logger.error("TimeoutScanThread exception {}", e.getMessage());
+                } else {
+                    logger.warn("TimeoutScanThread exception {}", e.getMessage());
+                }
+            }
+            if (printCount++ % 20 == 0) {
+                logger.info("TimeoutScanThread thread=" + Thread.currentThread().getId()
+                        + "'s currentBufferSize = " + currentBufferSize.get());
+            }
+        }
+        logger.info("TimeoutScanThread Thread=" + Thread.currentThread().getId() + " existed !");
+    }
+}
diff --git a/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/utils/ConcurrentHashSet.java b/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/utils/ConcurrentHashSet.java
new file mode 100644
index 0000000..b15cbaf
--- /dev/null
+++ b/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/utils/ConcurrentHashSet.java
@@ -0,0 +1,45 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.dataproxy.utils;
+
+import java.util.Collection;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+/**
+ * A {@link ConcurrentHashMap}-backed {@link Set}.
+ */
+public class ConcurrentHashSet<E> extends MapBackedSet<E> {
+
+    private static final long serialVersionUID = 8518578988740277828L;
+
+    public ConcurrentHashSet() {
+        super(new ConcurrentHashMap<E, Boolean>());
+    }
+
+    public ConcurrentHashSet(Collection<E> c) {
+        super(new ConcurrentHashMap<E, Boolean>(), c);
+    }
+
+    @Override
+    public boolean add(E o) {
+        Boolean answer = ((ConcurrentMap<E, Boolean>) map).putIfAbsent(o, Boolean.TRUE);
+        return answer == null;
+    }
+}
diff --git a/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/utils/EncryptUtil.java b/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/utils/EncryptUtil.java
new file mode 100644
index 0000000..c093486
--- /dev/null
+++ b/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/utils/EncryptUtil.java
@@ -0,0 +1,478 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.dataproxy.utils;
+
+import java.io.BufferedInputStream;
+import java.io.BufferedReader;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileReader;
+import java.io.IOException;
+import java.security.InvalidKeyException;
+import java.security.KeyFactory;
+import java.security.NoSuchAlgorithmException;
+import java.security.SecureRandom;
+import java.security.interfaces.RSAPrivateKey;
+import java.security.interfaces.RSAPublicKey;
+import java.security.spec.InvalidKeySpecException;
+import java.security.spec.PKCS8EncodedKeySpec;
+import java.security.spec.X509EncodedKeySpec;
+import javax.crypto.BadPaddingException;
+import javax.crypto.Cipher;
+import javax.crypto.IllegalBlockSizeException;
+import javax.crypto.KeyGenerator;
+import javax.crypto.NoSuchPaddingException;
+import javax.crypto.SecretKey;
+import javax.crypto.SecretKeyFactory;
+import javax.crypto.spec.DESKeySpec;
+
+import org.apache.commons.codec.binary.Base64;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class EncryptUtil {
+    private static final Logger logger =
+            LoggerFactory.getLogger(EncryptUtil.class);
+
+    public static final int MAX_ENCRYPT_BLOCK = 117;
+
+    public static final int MAX_DECRYPT_BLOCK = 128;
+
+    public static final String DES = "DES";
+
+
+    /**
+     * load key
+     *
+     * @param path path
+     * @throws Exception exception
+     */
+    public static String loadPublicKeyByFileText(String path) throws Exception {
+        try {
+            BufferedReader br = new BufferedReader(new FileReader(path));
+            String readLine = null;
+            StringBuilder sb = new StringBuilder();
+            while ((readLine = br.readLine()) != null) {
+                sb.append(readLine);
+            }
+            br.close();
+            return sb.toString();
+        } catch (IOException e) {
+            throw new Exception("key error");
+        } catch (NullPointerException e) {
+            throw new Exception("npe error");
+        }
+    }
+
+    public static byte[] loadPublicKeyByFileBinary(String path) throws Exception {
+        try {
+            File file = new File(path);
+            int len = 0;
+            if (file.exists()) {
+                len = (int) file.length();
+            } else {
+                // error
+            }
+
+            DataInputStream dis = new DataInputStream(new BufferedInputStream(new FileInputStream(path)));
+            byte[] content = new byte[len];
+            while (dis.read(content) != -1) {
+
+            }
+
+            dis.close();
+            return content;
+        } catch (IOException e) {
+            throw new Exception("key error");
+        } catch (NullPointerException e) {
+            throw new Exception("npe error");
+        }
+    }
+
+    /**
+     * get key from public
+     *
+     * @param publicKeyStr get key string
+     */
+    public static RSAPublicKey loadPublicKeyByText(String publicKeyStr) {
+        try {
+            byte[] buffer = Base64.decodeBase64(publicKeyStr);
+//            byte[] buffer = publicKeyStr.getBytes();
+            KeyFactory keyFactory = KeyFactory.getInstance("RSA");
+            X509EncodedKeySpec keySpec = new X509EncodedKeySpec(buffer);
+            return (RSAPublicKey) keyFactory.generatePublic(keySpec);
+        } catch (NoSuchAlgorithmException e) {
+            logger.error("no such algorithm", e);
+        } catch (InvalidKeySpecException e) {
+            logger.error("invalid key spec", e);
+        } catch (NullPointerException e) {
+            logger.error("public key is null", e);
+        }
+        return null;
+    }
+
+    public static RSAPublicKey loadPublicKeyByBinary(byte[] publicKeyByte) {
+        try {
+            //byte[] buffer = Base64.decodeBase64(publicKeyStr);
+            KeyFactory keyFactory = KeyFactory.getInstance("RSA");
+            X509EncodedKeySpec keySpec = new X509EncodedKeySpec(publicKeyByte);
+            return (RSAPublicKey) keyFactory.generatePublic(keySpec);
+        } catch (NoSuchAlgorithmException e) {
+            logger.error("no such algorithm", e);
+        } catch (InvalidKeySpecException e) {
+            logger.error("invalid key spec", e);
+        } catch (NullPointerException e) {
+            logger.error("public key is null", e);
+        }
+
+        return null;
+    }
+
+    /**
+     * get key from file
+     *
+     * @param path key path
+     * @return whether success
+     * @throws Exception
+     */
+    public static String loadPrivateKeyByFileText(String path) throws Exception {
+        try {
+            BufferedReader br = new BufferedReader(new FileReader(path));
+            String readLine = null;
+            StringBuilder sb = new StringBuilder();
+            while ((readLine = br.readLine()) != null) {
+                sb.append(readLine);
+            }
+            br.close();
+            return sb.toString();
+        } catch (IOException e) {
+            throw new Exception("key error");
+        } catch (NullPointerException e) {
+            throw new Exception("npe error");
+        }
+    }
+
+    public static byte[] loadPrivateKeyByFileBinary(String path) throws Exception {
+        try {
+            File file = new File(path);
+            int len = 0;
+            if (file.exists()) {
+                len = (int) file.length();
+            } else {
+                // error
+            }
+
+            DataInputStream dis = new DataInputStream(new BufferedInputStream(new FileInputStream(path)));
+            byte[] content = new byte[len];
+            while (dis.read(content) != -1) {
+
+            }
+
+            dis.close();
+            return content;
+        } catch (IOException e) {
+            throw new Exception("key error");
+        } catch (NullPointerException e) {
+            throw new Exception("npe error");
+        }
+    }
+
+    /**
+     * load private key by text
+     *
+     * @param privateKeyStr private key
+     * @throws Exception exception
+     */
+    public static RSAPrivateKey loadPrivateKeyByText(String privateKeyStr)
+            throws Exception {
+        try {
+            //byte[] buffer = Base64.decodeBase64(privateKeyStr);
+            byte[] buffer = privateKeyStr.getBytes();
+            PKCS8EncodedKeySpec keySpec = new PKCS8EncodedKeySpec(buffer);
+            KeyFactory keyFactory = KeyFactory.getInstance("RSA");
+            return (RSAPrivateKey) keyFactory.generatePrivate(keySpec);
+        } catch (NoSuchAlgorithmException e) {
+            throw new Exception("no such algorithm");
+        } catch (InvalidKeySpecException e) {
+            throw new Exception("key error");
+        } catch (NullPointerException e) {
+            throw new Exception("npe error");
+        }
+    }
+
+    public static RSAPrivateKey loadPrivateKeyByBinary(byte[] privateKeyByte)
+            throws Exception {
+        try {
+            //byte[] buffer = Base64.decodeBase64(privateKeyStr);
+            PKCS8EncodedKeySpec keySpec = new PKCS8EncodedKeySpec(privateKeyByte);
+            KeyFactory keyFactory = KeyFactory.getInstance("RSA");
+            return (RSAPrivateKey) keyFactory.generatePrivate(keySpec);
+        } catch (NoSuchAlgorithmException e) {
+            throw new Exception("no such algorithm");
+        } catch (InvalidKeySpecException e) {
+            throw new Exception("key error");
+        } catch (NullPointerException e) {
+            throw new Exception("npe error");
+        }
+    }
+
+    /**
+     * key encrypt
+     *
+     * @param publicKey public key
+     * @param data      data
+     * @return
+     * @throws Exception exception
+     */
+    public static byte[] rsaEncrypt(RSAPublicKey publicKey, byte[] data)
+            throws Exception {
+
+        Cipher cipher = Cipher.getInstance("RSA");
+        cipher.init(Cipher.ENCRYPT_MODE, publicKey);
+        int inputLen = data.length;
+        ByteArrayOutputStream out = new ByteArrayOutputStream();
+        int offSet = 0;
+        byte[] cache;
+        int i = 0;
+
+        while (inputLen - offSet > 0) {
+            if (inputLen - offSet > MAX_ENCRYPT_BLOCK) {
+                cache = cipher.doFinal(data, offSet, MAX_ENCRYPT_BLOCK);
+            } else {
+                cache = cipher.doFinal(data, offSet, inputLen - offSet);
+            }
+            out.write(cache, 0, cache.length);
+            i++;
+            offSet = i * MAX_ENCRYPT_BLOCK;
+        }
+        byte[] encryptedData = out.toByteArray();
+        out.close();
+        return encryptedData;
+
+    }
+
+
+    /**
+     * key encrypt
+     *
+     * @param privateKey    key
+     * @param encryptedData data
+     * @return
+     * @throws Exception exception
+     */
+    public static byte[] decryptByPrivateKey(RSAPrivateKey privateKey, byte[] encryptedData)
+            throws Exception {
+        Cipher cipher = Cipher.getInstance("RSA");
+        cipher.init(Cipher.DECRYPT_MODE, privateKey);
+        int inputLen = encryptedData.length;
+        ByteArrayOutputStream out = new ByteArrayOutputStream();
+        int offSet = 0;
+        byte[] cache;
+        int i = 0;
+
+        while (inputLen - offSet > 0) {
+            if (inputLen - offSet > MAX_DECRYPT_BLOCK) {
+                cache = cipher.doFinal(encryptedData, offSet, MAX_DECRYPT_BLOCK);
+            } else {
+                cache = cipher.doFinal(encryptedData, offSet, inputLen - offSet);
+            }
+            out.write(cache, 0, cache.length);
+            i++;
+            offSet = i * MAX_DECRYPT_BLOCK;
+        }
+        byte[] decryptedData = out.toByteArray();
+        out.close();
+        return decryptedData;
+    }
+
+    /**
+     * rsa decrypt
+     *
+     * @param privateKey private key
+     * @param cipherData data
+     * @return message
+     * @throws Exception exception
+     */
+    public static byte[] rsaDecrypt(RSAPrivateKey privateKey, byte[] cipherData)
+            throws Exception {
+        if (privateKey == null) {
+            throw new Exception("private key is null");
+        }
+        Cipher cipher = null;
+        try {
+            cipher = Cipher.getInstance("RSA");
+            // cipher= Cipher.getInstance("RSA", new BouncyCastleProvider());
+            cipher.init(Cipher.DECRYPT_MODE, privateKey);
+            byte[] output = cipher.doFinal(cipherData);
+            return output;
+        } catch (NoSuchAlgorithmException e) {
+            throw new Exception("no such algorithm");
+        } catch (NoSuchPaddingException e) {
+            e.printStackTrace();
+            return null;
+        } catch (InvalidKeyException e) {
+            throw new Exception("invalid key");
+        } catch (IllegalBlockSizeException e) {
+            throw new Exception("illegal size");
+        } catch (BadPaddingException e) {
+            throw new Exception("bad padding");
+        }
+    }
+
+    /**
+     * rsa decrypt
+     *
+     * @param publicKey  public key
+     * @param cipherData cipher data
+     * @return
+     * @throws Exception exception
+     */
+    public static byte[] rsaDecrypt(RSAPublicKey publicKey, byte[] cipherData)
+            throws Exception {
+        if (publicKey == null) {
+            throw new Exception("public key is null");
+        }
+        Cipher cipher = null;
+        try {
+
+            cipher = Cipher.getInstance("RSA");
+            // cipher= Cipher.getInstance("RSA", new BouncyCastleProvider());
+            cipher.init(Cipher.DECRYPT_MODE, publicKey);
+            byte[] output = cipher.doFinal(cipherData);
+            return output;
+        } catch (NoSuchAlgorithmException e) {
+            throw new Exception("no such algorithm");
+        } catch (NoSuchPaddingException e) {
+            e.printStackTrace();
+            return null;
+        } catch (InvalidKeyException e) {
+            throw new Exception("invalid key");
+        } catch (IllegalBlockSizeException e) {
+            throw new Exception("illegal block");
+        } catch (BadPaddingException e) {
+            throw new Exception("bad padding");
+        }
+    }
+
+
+
+    /**
+     * generate des key
+     *
+     * @return base64 key
+     */
+    public static byte[] generateDesKey() {
+
+        KeyGenerator kg = null;
+        try {
+            kg = KeyGenerator.getInstance("DES");
+        } catch (NoSuchAlgorithmException e) {
+            logger.error("generate Des key error {}", e);
+        }
+
+        kg.init(56);
+
+        SecretKey secretKey = kg.generateKey();
+        return secretKey.getEncoded();
+    }
+
+
+    /**
+     * des encrypt
+     *
+     * @param plainText
+     * @param desKey
+     * @return
+     */
+    public static byte[] desEncrypt(byte[] plainText, byte[] desKey) {
+        try {
+//            byte[] buffer = Base64.decodeBase64(DesKey);
+
+            SecureRandom sr = SecureRandom.getInstance("SHA1PRNG");
+
+            DESKeySpec dks = new DESKeySpec(desKey);
+
+            SecretKeyFactory keyFactory = SecretKeyFactory.getInstance(DES);
+            SecretKey key = keyFactory.generateSecret(dks);
+
+            Cipher cipher = Cipher.getInstance(DES);
+            cipher.init(Cipher.ENCRYPT_MODE, key);
+
+            byte[] encryptedData = cipher.doFinal(plainText);
+
+            return encryptedData;
+        } catch (Exception e) {
+            logger.error("desEncrypt error {}", e);
+            return null;
+        }
+    }
+
+
+    /**
+     * des decrypt
+     *
+     * @param plainText
+     * @param desKey
+     * @return des decrypt
+     */
+    public static byte[] dESDecrypt(byte[] plainText, byte[] desKey) {
+        try {
+            SecureRandom sr = SecureRandom.getInstance("SHA1PRNG");
+
+            DESKeySpec dks = new DESKeySpec(desKey);
+            SecretKeyFactory keyFactory = SecretKeyFactory.getInstance(DES);
+            SecretKey key = keyFactory.generateSecret(dks);
+
+            Cipher cipher = Cipher.getInstance(DES);
+
+            cipher.init(Cipher.DECRYPT_MODE, key);
+
+            byte[] decryptedData = cipher.doFinal(plainText);
+
+//            System.out.println("decrypted data");
+//            System.out.println(new String(decryptedData));
+
+            return decryptedData;
+        } catch (Exception e) {
+//            e.printStackTrace();
+            logger.error("dESDecrypt error {}", e);
+            return null;
+        }
+    }
+
+
+    public static void main(String[] args) {
+        String plainText = "TDB-30001 Create Tdw Table Error26880 FAILED: "
+                + "TDWServer run SQL error (session: 6425308280519064 query: CREATE TABLE a";
+        System.out.println("plainText: \n" + plainText);
+        byte[] key = new byte[0];
+        try {
+            key = generateDesKey();
+        } catch (Exception e) {
+            e.printStackTrace();
+        }
+        byte[] encryptedData = desEncrypt(plainText.getBytes(), key);
+        System.out.println("after encrypted: \n" + new String(encryptedData));
+        byte[] decryptedData = dESDecrypt(encryptedData, key);
+        System.out.println("after decrypted: \n" + new String(decryptedData));
+
+    }
+}
diff --git a/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/utils/MapBackedSet.java b/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/utils/MapBackedSet.java
new file mode 100644
index 0000000..0ca2bd0
--- /dev/null
+++ b/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/utils/MapBackedSet.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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.dataproxy.utils;
+
+import java.io.Serializable;
+import java.util.AbstractSet;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * A {@link Map}-backed {@link Set}.
+ */
+public class MapBackedSet<E> extends AbstractSet<E> implements Serializable {
+
+    private static final long serialVersionUID = -8347878570391674042L;
+
+    protected final Map<E, Boolean> map;
+
+    public MapBackedSet(Map<E, Boolean> map) {
+        this.map = map;
+    }
+
+    public MapBackedSet(Map<E, Boolean> map, Collection<E> c) {
+        this.map = map;
+        addAll(c);
+    }
+
+    @Override
+    public int size() {
+        return map.size();
+    }
+
+    @Override
+    public boolean contains(Object o) {
+        return map.containsKey(o);
+    }
+
+    @Override
+    public Iterator<E> iterator() {
+        return map.keySet().iterator();
+    }
+
+    @Override
+    public boolean add(E o) {
+        return map.put(o, Boolean.TRUE) == null;
+    }
+
+    @Override
+    public boolean remove(Object o) {
+        return map.remove(o) != null;
+    }
+
+    @Override
+    public void clear() {
+        map.clear();
+    }
+}
diff --git a/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/utils/ProxyUtils.java b/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/utils/ProxyUtils.java
new file mode 100644
index 0000000..4d25bc8
--- /dev/null
+++ b/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/utils/ProxyUtils.java
@@ -0,0 +1,151 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.dataproxy.utils;
+
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+
+import java.util.Set;
+import org.apache.inlong.dataproxy.ProxyClientConfig;
+import org.apache.inlong.dataproxy.network.Utils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ProxyUtils {
+    private static final Logger logger = LoggerFactory.getLogger(ProxyUtils.class);
+    private static final int TIME_LENGTH = 13;
+    private static final Set<String> invalidAttr = new HashSet<>();
+
+    static {
+        Collections.addAll(invalidAttr, "bid", "tid", "dt", "msgUUID", "cp",
+            "cnt", "mt", "m", "sid", "t", "NodeIP", "messageId", "_file_status_check", "_secretId",
+            "_signature", "_timeStamp", "_nonce", "_userName", "_clientIP", "_encyVersion", "_encyDesKey");
+    }
+
+    public static boolean isAttrKeysValid(Map<String, String> attrsMap) {
+        if (attrsMap == null || attrsMap.size() == 0) {
+            return false;
+        }
+        for (String key : attrsMap.keySet()) {
+            if (invalidAttr.contains(key)) {
+                logger.error("the attributes is invalid ,please check ! {}", key);
+                return false;
+            }
+        }
+        return true;
+    }
+
+    public static boolean isDtValid(long dt) {
+        if (String.valueOf(dt).length() != TIME_LENGTH) {
+            logger.error("dt {} is error", dt);
+            return false;
+        }
+        return true;
+    }
+
+    /**
+     * check body valid
+     *
+     * @param body
+     * @return
+     */
+    public static boolean isBodyValid(byte[] body) {
+        if (body == null || body.length == 0) {
+            logger.error("body is error {}", body);
+            return false;
+        }
+        return true;
+    }
+
+    /**
+     * check body valid
+     *
+     * @param bodyList
+     * @return
+     */
+    public static boolean isBodyValid(List<byte[]> bodyList) {
+        if (bodyList == null || bodyList.size() == 0) {
+            logger.error("body  is error");
+            return false;
+        }
+        return true;
+    }
+
+
+
+    public static long covertZeroDt(long dt) {
+        if (dt == 0) {
+            return System.currentTimeMillis();
+        }
+        return dt;
+    }
+
+    public static StringBuilder convertAttrToStr(Map<String, String> extraAttrMap) {
+        StringBuilder attrs = new StringBuilder();
+        for (Map.Entry<String, String> entry : extraAttrMap.entrySet()) {
+            String key = entry.getKey();
+            String value = entry.getValue();
+            attrs.append(key).append("=");
+            attrs.append(value).append("&");
+        }
+        attrs.deleteCharAt(attrs.length() - 1);
+        return attrs;
+    }
+
+    /**
+     * valid client config
+     *
+     * @param clientConfig
+     */
+    public static void validClientConfig(ProxyClientConfig clientConfig) {
+        if (clientConfig.isNeedAuthentication()) {
+            if (Utils.isBlank(clientConfig.getUserName())) {
+                throw new IllegalArgumentException("Authentication require userName not Blank!");
+            }
+            if (Utils.isBlank(clientConfig.getSecretKey())) {
+                throw new IllegalArgumentException("Authentication require secretKey not Blank!");
+            }
+        }
+        if (!clientConfig.isLocalVisit()) {
+            //if(!clientConfig.isNeedDataEncry()) {
+            //    throw new IllegalArgumentException("OutNetwork visit isNeedDataEncry must be true!");
+            //}
+            if (!clientConfig.isNeedAuthentication()) {
+                throw new IllegalArgumentException("OutNetwork visit isNeedAuthentication must be true!");
+            }
+            if (Utils.isBlank(clientConfig.getUserName())) {
+                throw new IllegalArgumentException("Authentication require userName not Blank!");
+            }
+            if (Utils.isBlank(clientConfig.getSecretKey())) {
+                throw new IllegalArgumentException("Authentication require secretKey not Blank!");
+            }
+            if (!clientConfig.isNeedVerServer()) {
+                throw new IllegalArgumentException("OutNetwork visit need https, please set https parameters!");
+            }
+            if (Utils.isBlank(clientConfig.getTlsServerCertFilePathAndName())) {
+                throw new IllegalArgumentException("OutNetwork visit need https, "
+                        + "TlsServerCertFilePathAndName is Blank!");
+            }
+            if (Utils.isBlank(clientConfig.getTlsServerKey())) {
+                throw new IllegalArgumentException("OutNetwork visit need https, tlsServerKey is Blank!");
+            }
+        }
+    }
+}
diff --git a/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/utils/ServiceDiscoveryUtils.java b/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/utils/ServiceDiscoveryUtils.java
new file mode 100644
index 0000000..141c17f
--- /dev/null
+++ b/inlong-dataproxy-sdk/src/main/java/org/apache/inlong/dataproxy/utils/ServiceDiscoveryUtils.java
@@ -0,0 +1,292 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.dataproxy.utils;
+
+import static org.apache.inlong.dataproxy.ConfigConstants.REQUEST_HEADER_AUTHORIZATION;
+
+import com.google.gson.JsonObject;
+import com.google.gson.JsonParser;
+
+import java.io.BufferedReader;
+import java.io.BufferedWriter;
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStreamWriter;
+import java.io.UnsupportedEncodingException;
+import java.nio.charset.StandardCharsets;
+import java.security.SecureRandom;
+import java.util.ArrayList;
+import javax.net.ssl.SSLContext;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.http.Header;
+import org.apache.http.HttpResponse;
+import org.apache.http.client.config.RequestConfig;
+import org.apache.http.client.entity.UrlEncodedFormEntity;
+import org.apache.http.client.methods.HttpPost;
+import org.apache.http.conn.ssl.SSLConnectionSocketFactory;
+import org.apache.http.impl.client.CloseableHttpClient;
+import org.apache.http.impl.client.DefaultHttpClient;
+import org.apache.http.impl.client.HttpClients;
+import org.apache.http.message.BasicHeader;
+import org.apache.http.message.BasicNameValuePair;
+import org.apache.http.params.BasicHttpParams;
+import org.apache.http.params.HttpConnectionParams;
+import org.apache.http.params.HttpParams;
+import org.apache.http.ssl.SSLContexts;
+import org.apache.http.util.EntityUtils;
+import org.apache.inlong.dataproxy.ProxyClientConfig;
+import org.apache.inlong.dataproxy.network.ProxysdkException;
+import org.apache.inlong.dataproxy.network.Utils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Created by elroyzhang on 2018/5/10.
+ */
+public class ServiceDiscoveryUtils {
+    private static final Logger log = LoggerFactory.getLogger(ServiceDiscoveryUtils.class);
+
+    private static String latestManagerIPList = "";
+    private static String arraySed = ",";
+
+    public static String getManagerIpList(ProxyClientConfig proxyClientConfig) throws ProxysdkException {
+        String managerIpList;
+        String managerAddress = proxyClientConfig.getManagerIP() + ":" + proxyClientConfig.getManagerPort();
+        if (StringUtils.isBlank(managerAddress)) {
+            log.error("managerAddress is blank.");
+            return null;
+        }
+
+
+        managerIpList = getManagerIpListByHttp(managerAddress, proxyClientConfig);
+        if (!StringUtils.isBlank(managerIpList)) {
+            latestManagerIPList = managerIpList;
+            return managerIpList;
+        }
+        log.error("ServiceDiscovery get managerIpList from "
+            + "managerHost occur error, will try to get from managerIpList.");
+
+
+        String[] managerIps = latestManagerIPList.split(arraySed);
+        if (managerIps.length > 0) {
+            for (String managerIp : managerIps) {
+                if (!StringUtils.isBlank(managerIp)) {
+                    String currentAddress = managerIp + ":" + proxyClientConfig.getManagerPort();
+                    managerIpList = getManagerIpListByHttp(currentAddress, proxyClientConfig);
+                    if (!StringUtils.isBlank(managerIpList)) {
+                        latestManagerIPList = managerIpList;
+                        return managerIpList;
+                    } else {
+                        log.error("ServiceDiscovery request " + managerIp
+                                + " got by latestManagerIPList[" + latestManagerIPList
+                                + "] got nothing, will try next ip.");
+                    }
+                } else {
+                    log.error("ServiceDiscovery managerIp is null, "
+                        + "lastestManagerIPList is [" + latestManagerIPList
+                        + "].");
+                }
+            }
+        } else {
+            log.error("ServiceDiscovery latestManagerIpList["
+                + latestManagerIPList + "] format error, or not contain ip");
+        }
+
+
+
+
+
+        String existedTdmIpList = getLocalManagerIpList(proxyClientConfig.getManagerIpLocalPath());
+        if (!StringUtils.isBlank(existedTdmIpList)) {
+            String[] existedTdmIps = existedTdmIpList.split(arraySed);
+            if (existedTdmIps.length > 0) {
+                for (String existedTdmIp : existedTdmIps) {
+                    if (!StringUtils.isBlank(existedTdmIp)) {
+                        String currentAddress = existedTdmIp + ":" + proxyClientConfig.getManagerPort();
+                        managerIpList = getManagerIpListByHttp(currentAddress, proxyClientConfig);
+                        if (!StringUtils.isBlank(managerIpList)) {
+                            latestManagerIPList = managerIpList;
+                            return managerIpList;
+                        } else {
+                            log.error("ServiceDiscovery request " + existedTdmIp + " got by local file["
+                                    + proxyClientConfig.getManagerIpLocalPath() + "] got nothing, will try next ip.");
+                        }
+                    } else {
+                        log.error("ServiceDiscovery get illegal format TdmIpList from local file, "
+                                + "exist one ip is empty, managerIpList is ["
+                                + existedTdmIpList + "], "
+                                + "local file is [" + proxyClientConfig.getManagerIpLocalPath() + "]");
+                    }
+                }
+            } else {
+                log.error("ServiceDiscovery get illegal format TdmIpList from local file, "
+                        + "managerIpList is [" + existedTdmIpList + "], "
+                        + "local file is [" + proxyClientConfig.getManagerIpLocalPath() + "]");
+            }
+        } else {
+            log.error("ServiceDiscovery get empty TdmIpList from local file, "
+                    + "file path is [" + proxyClientConfig.getManagerIpLocalPath() + "].");
+        }
+
+        return managerIpList;
+    }
+
+    public static String getManagerIpListByHttp(String managerIp,
+        ProxyClientConfig proxyClientConfig) throws ProxysdkException {
+        String url = (proxyClientConfig.isLocalVisit() ? "http://" : "https://") + managerIp + "/api/getmanagervirtualip";
+        ArrayList<BasicNameValuePair> params = new ArrayList<BasicNameValuePair>();
+        params.add(new BasicNameValuePair("operation", "query"));
+        params.add(new BasicNameValuePair("username", proxyClientConfig.getUserName()));
+        log.info("Begin to get configure from manager {}, param is {}", url, params);
+        CloseableHttpClient httpClient = null;
+        HttpPost httpPost = null;
+        String returnStr = null;
+        HttpParams myParams = new BasicHttpParams();
+        HttpConnectionParams.setConnectionTimeout(myParams, proxyClientConfig.getManagerConnectionTimeout());
+        HttpConnectionParams.setSoTimeout(myParams, proxyClientConfig.getManagerSocketTimeout());
+        if (proxyClientConfig.isLocalVisit()) {
+            httpClient = new DefaultHttpClient(myParams);
+        } else {
+            try {
+                ArrayList<Header> headers = new ArrayList<Header>();
+                for (BasicNameValuePair paramItem : params) {
+                    headers.add(new BasicHeader(paramItem.getName(), paramItem.getValue()));
+                }
+                RequestConfig requestConfig = RequestConfig.custom()
+                        .setConnectTimeout(10000).setSocketTimeout(30000).build();
+                SSLContext sslContext = SSLContexts.custom().build();
+                SSLConnectionSocketFactory sslsf = new SSLConnectionSocketFactory(sslContext,
+                        new String[]{"TLSv1"}, null,
+                        SSLConnectionSocketFactory.getDefaultHostnameVerifier());
+                httpClient = HttpClients.custom().setDefaultHeaders(headers)
+                        .setDefaultRequestConfig(requestConfig).setSSLSocketFactory(sslsf).build();
+            } catch (Throwable eHttps) {
+                log.error("Create Https cliet failure, error 1 is ", eHttps);
+                eHttps.printStackTrace();
+                return null;
+            }
+        }
+
+        try {
+            httpPost = new HttpPost(url);
+            if (proxyClientConfig.isNeedAuthentication()) {
+                long timestamp = System.currentTimeMillis();
+                int nonce = new SecureRandom(String.valueOf(timestamp).getBytes()).nextInt(Integer.MAX_VALUE);
+                httpPost.setHeader(REQUEST_HEADER_AUTHORIZATION,
+                        Utils.getAuthorizenInfo(proxyClientConfig.getUserName(),
+                                proxyClientConfig.getSecretKey(), timestamp, nonce));
+            }
+            UrlEncodedFormEntity se = new UrlEncodedFormEntity(params);
+            httpPost.setEntity(se);
+            HttpResponse response = httpClient.execute(httpPost);
+            returnStr = EntityUtils.toString(response.getEntity());
+            if (Utils.isNotBlank(returnStr) && response.getStatusLine().getStatusCode() == 200) {
+                log.info("Get configure from manager is " + returnStr);
+                JsonParser jsonParser = new JsonParser();
+                JsonObject jb = jsonParser.parse(returnStr).getAsJsonObject();
+                JsonObject rd = jb.get("resultData").getAsJsonObject();
+                String ip = rd.get("ip").getAsString();
+                log.info("ServiceDiscovery updated managerVirtualIP success, ip : " + ip + ", retStr : " + returnStr);
+                return ip;
+            }
+            return null;
+        } catch (Throwable e) {
+            log.error("Connect Manager error, {}", e.getMessage());
+            return null;
+        } finally {
+            if (httpPost != null) {
+                httpPost.releaseConnection();
+            }
+            if (httpClient != null) {
+                httpClient.getConnectionManager().shutdown();
+            }
+        }
+    }
+
+    public static String getLocalManagerIpList(String managerIpLocalPath) {
+        log.info("ServiceDiscovery start loading config for :{} from file ...", managerIpLocalPath);
+        BufferedReader reader = null;
+        String newestIp = null;
+        try {
+            File managerIpListFile = new File(managerIpLocalPath);
+            if (!managerIpListFile.exists()) {
+                log.error("ServiceDiscovery no found local bidInfo file, path is [" + managerIpLocalPath + "].");
+                return null;
+            }
+            byte[] serialized;
+            serialized = FileUtils.readFileToByteArray(managerIpListFile);
+            if (serialized == null) {
+                return null;
+            }
+            newestIp = new String(serialized, "UTF-8");
+            log.info("ServiceDiscovery get manager ip list from local success, result is : {}", newestIp);
+        } catch (FileNotFoundException e) {
+            log.error("ServiceDiscovery load manager config error, file not found. Exception info : {}", e);
+        } catch (IOException e) {
+            log.error("ServiceDiscovery load manager config error. Exception info : {}", e);
+        } finally {
+            if (reader != null) {
+                try {
+                    reader.close();
+                } catch (IOException e) {
+                    log.error("ServiceDiscovery close bufferedReader "
+                            + "error after loading TDM config . Exception info : {}.", e);
+                }
+            }
+        }
+        return newestIp;
+    }
+
+
+    public static void updateManagerInfo2Local(String storeString, String path) {
+        if (StringUtils.isBlank(storeString)) {
+            log.warn("ServiceDiscovery updateTdmInfo2Local error, configMap is empty or managerIpList is blank.");
+            return;
+        }
+        BufferedWriter writer = null;
+        try {
+
+            File localPath = new File(path);
+            if (!localPath.getParentFile().exists()) {
+                localPath.getParentFile().mkdirs();
+            }
+            writer = new BufferedWriter(new OutputStreamWriter(
+                    new FileOutputStream(localPath), StandardCharsets.UTF_8));
+            writer.write(storeString);
+            writer.flush();
+        } catch (UnsupportedEncodingException e) {
+            log.error("ServiceDiscovery save manager config error1 .", e);
+        } catch (FileNotFoundException e) {
+            log.error("ServiceDiscovery save manager config error2 .", e);
+        } catch (IOException e) {
+            log.error("ServiceDiscovery save manager config error3 .", e);
+        } finally {
+            if (writer != null) {
+                try {
+                    writer.close();
+                } catch (IOException e) {
+                    log.error("ServiceDiscovery close manager writer error.", e);
+                }
+            }
+        }
+    }
+}
diff --git a/inlong-dataproxy-sdk/src/test/java/org/apache/inlong/dataproxy/AppTest.java b/inlong-dataproxy-sdk/src/test/java/org/apache/inlong/dataproxy/AppTest.java
new file mode 100644
index 0000000..72e4c06
--- /dev/null
+++ b/inlong-dataproxy-sdk/src/test/java/org/apache/inlong/dataproxy/AppTest.java
@@ -0,0 +1,51 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.dataproxy;
+
+import junit.framework.Test;
+import junit.framework.TestCase;
+import junit.framework.TestSuite;
+
+/**
+ * Unit test for simple App.
+ */
+public class AppTest
+        extends TestCase {
+    /**
+     * Create the test case
+     *
+     * @param testName name of the test case
+     */
+    public AppTest(String testName) {
+        super(testName);
+    }
+
+    /**
+     * @return the suite of tests being tested
+     */
+    public static Test suite() {
+        return new TestSuite(AppTest.class);
+    }
+
+    /**
+     * Rigourous Test :-)
+     */
+    public void testApp() {
+        assertTrue(true);
+    }
+}
diff --git a/inlong-dataproxy-sdk/src/test/java/org/apache/inlong/dataproxy/TestMetricWorkerThread.java b/inlong-dataproxy-sdk/src/test/java/org/apache/inlong/dataproxy/TestMetricWorkerThread.java
new file mode 100644
index 0000000..39f102e
--- /dev/null
+++ b/inlong-dataproxy-sdk/src/test/java/org/apache/inlong/dataproxy/TestMetricWorkerThread.java
@@ -0,0 +1,69 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.dataproxy;
+
+import java.util.concurrent.TimeUnit;
+
+import org.apache.inlong.dataproxy.ProxyClientConfig;
+import org.apache.inlong.dataproxy.network.Utils;
+import org.apache.inlong.dataproxy.threads.MetricWorkerThread;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+
+public class TestMetricWorkerThread {
+
+    private static MetricWorkerThread workerThread;
+
+    @BeforeClass
+    public static void setup() throws Exception {
+        ProxyClientConfig config = new ProxyClientConfig(Utils.getLocalIp(),
+                true, "127.0.0.1", 8099, "test", "all");
+        workerThread = new MetricWorkerThread(config, null);
+        workerThread.start();
+    }
+
+    @Test
+    public void testMetricCount() throws Exception {
+        for (int i = 0; i < 10000; i++) {
+            workerThread.recordNumByKey(String.valueOf(i), "test", "test1", "127.0.0.1",
+                    System.currentTimeMillis(), System.currentTimeMillis(), i);
+        }
+        TimeUnit.SECONDS.sleep(59);
+        for (int i = 0; i < 2000; i++) {
+            workerThread.recordSuccessByMessageId(String.valueOf(i));
+        }
+        for (int i = 2000; i < 2060; i++) {
+            workerThread.recordFailedByMessageId(String.valueOf(i));
+        }
+        TimeUnit.SECONDS.sleep(10);
+    }
+
+    @Test
+    public void getLocalIp() {
+        String ip = Utils.getLocalIp();
+        System.out.println(ip);
+    }
+
+    @AfterClass
+    public static void teardown() {
+        workerThread.close();
+    }
+
+}
diff --git a/pom.xml b/pom.xml
index 4b95840..0611334 100644
--- a/pom.xml
+++ b/pom.xml
@@ -87,6 +87,7 @@
     <module>inlong-common</module>
     <module>inlong-tubemq</module>
     <module>inlong-dataproxy</module>
+    <module>inlong-dataproxy-sdk</module>
     <module>inlong-manager</module>
     <module>inlong-sort</module>
   </modules>