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 2023/06/29 03:02:27 UTC

[inlong] branch master updated: [INLONG-8356][DataProxy] Replace source2 to source (#8359)

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

gosonzhang pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/inlong.git


The following commit(s) were added to refs/heads/master by this push:
     new e785da9ce3 [INLONG-8356][DataProxy] Replace source2 to source (#8359)
e785da9ce3 is described below

commit e785da9ce34dd4b03b947a659197dae1043af4b9
Author: Goson Zhang <46...@qq.com>
AuthorDate: Thu Jun 29 11:02:22 2023 +0800

    [INLONG-8356][DataProxy] Replace source2 to source (#8359)
---
 .../conf/dataproxy-mulit-pulsar-http-example.conf  |    6 +-
 .../conf/dataproxy-mulit-pulsar-udp-example.conf   |    2 -
 inlong-dataproxy/conf/dataproxy-tubemq.conf        |    8 +-
 inlong-dataproxy/conf/dataproxy.conf               |    8 +-
 .../org/apache/inlong/dataproxy/http/Context.java  |   39 -
 .../inlong/dataproxy/http/HttpBaseSource.java      |  155 ---
 .../inlong/dataproxy/http/MappedContext.java       |   72 --
 .../inlong/dataproxy/http/MessageFilter.java       |  176 ----
 .../inlong/dataproxy/http/MessageHandler.java      |   31 -
 .../dataproxy/http/MessageProcessServlet.java      |   73 --
 .../inlong/dataproxy/http/SimpleHttpSource.java    |  195 ----
 .../dataproxy/http/SimpleMessageHandler.java       |  236 -----
 .../http/exception/MessageProcessException.java    |   38 -
 .../dataproxy/sink/mq/SimplePackProfile.java       |    6 +-
 .../apache/inlong/dataproxy/source/BaseSource.java |  629 ++++++++----
 .../dataproxy/source/DefaultServiceDecoder.java    |  513 ----------
 .../dataproxy/source/ServerMessageFactory.java     |  131 +--
 .../dataproxy/source/ServerMessageHandler.java     | 1015 +++++++++++---------
 .../inlong/dataproxy/source/ServiceDecoder.java    |   41 -
 .../{source2 => source}/SimpleHttpSource.java      |    2 +-
 .../dataproxy/source/SimpleMessageHandler.java     |  673 -------------
 .../inlong/dataproxy/source/SimpleTcpSource.java   |  172 +---
 .../inlong/dataproxy/source/SimpleUdpSource.java   |   60 +-
 .../{source2 => source}/SourceConstants.java       |    2 +-
 .../inlong/dataproxy/source/SourceContext.java     |  270 ------
 .../httpMsg/HttpMessageHandler.java}               |   10 +-
 .../source/tcp/InlongTcpChannelHandler.java        |  358 -------
 .../tcp/InlongTcpChannelPipelineFactory.java       |  107 ---
 .../dataproxy/source/tcp/InlongTcpSource.java      |  173 ----
 .../source/tcp/InlongTcpSourceCallback.java        |  113 ---
 .../{source2 => source}/v0msg/AbsV0MsgCodec.java   |    4 +-
 .../{source2 => source}/v0msg/CodecBinMsg.java     |   34 +-
 .../{source2 => source}/v0msg/CodecTextMsg.java    |   12 +-
 .../{source2 => source}/v0msg/MsgFieldConsts.java  |    2 +-
 .../v1msg/InlongTcpSourceCallback.java             |    2 +-
 .../inlong/dataproxy/source2/BaseSource.java       |  582 -----------
 .../dataproxy/source2/InLongMessageFactory.java    |   85 --
 .../dataproxy/source2/InLongMessageHandler.java    |  700 --------------
 .../inlong/dataproxy/source2/SimpleTcpSource.java  |  127 ---
 .../inlong/dataproxy/source2/SimpleUdpSource.java  |   89 --
 40 files changed, 1143 insertions(+), 5808 deletions(-)

diff --git a/inlong-dataproxy/conf/dataproxy-mulit-pulsar-http-example.conf b/inlong-dataproxy/conf/dataproxy-mulit-pulsar-http-example.conf
index 56d2d7a858..7d02fa45d0 100644
--- a/inlong-dataproxy/conf/dataproxy-mulit-pulsar-http-example.conf
+++ b/inlong-dataproxy/conf/dataproxy-mulit-pulsar-http-example.conf
@@ -28,13 +28,11 @@ agent1.channels = ch-msg1 ch-msg2 ch-msg5 ch-msg6
 agent1.sinks = pulsar-sink-msg1 pulsar-sink-msg2 pulsar-sink-msg5 pulsar-sink-msg6
 
 agent1.sources.http-source.channels = ch-msg1 ch-msg2 ch-msg5 ch-msg6
-agent1.sources.http-source.type = org.apache.inlong.dataproxy.http.SimpleHttpSource
-agent1.sources.http-source.message-handler-name = org.apache.inlong.dataproxy.http.SimpleMessageHandler
+agent1.sources.http-source.type = org.apache.inlong.dataproxy.source.SimpleHttpSource
+agent1.sources.http-source.message-handler-name = org.apache.inlong.dataproxy.source.httpMsg.HttpMessageHandler
 agent1.sources.http-source.host = 0.0.0.0
 agent1.sources.http-source.port = 46802
 agent1.sources.http-source.max-msg-length = 524288
-#agent1.sources.http-source.topic = persistent://public/default/dataproxy-default-topic
-agent1.sources.http-source.attr = m=9
 agent1.sources.http-source.connections = 30000
 agent1.sources.http-source.max-threads = 64
 agent1.sources.http-source.receiveBufferSize = 1048576
diff --git a/inlong-dataproxy/conf/dataproxy-mulit-pulsar-udp-example.conf b/inlong-dataproxy/conf/dataproxy-mulit-pulsar-udp-example.conf
index 827db35a13..cbe473266b 100644
--- a/inlong-dataproxy/conf/dataproxy-mulit-pulsar-udp-example.conf
+++ b/inlong-dataproxy/conf/dataproxy-mulit-pulsar-udp-example.conf
@@ -33,8 +33,6 @@ agent1.sources.upd-source.msg-factory-name = org.apache.inlong.dataproxy.source.
 agent1.sources.upd-source.host = 0.0.0.0
 agent1.sources.upd-source.port = 46803
 agent1.sources.upd-source.max-msg-length = 524288
-#agent1.sources.upd-source.topic = persistent://public/default/dataproxy-default-topic
-agent1.sources.upd-source.attr = m=9
 agent1.sources.upd-source.connections = 30000
 agent1.sources.upd-source.max-threads = 64
 agent1.sources.upd-source.receiveBufferSize = 1048576
diff --git a/inlong-dataproxy/conf/dataproxy-tubemq.conf b/inlong-dataproxy/conf/dataproxy-tubemq.conf
index 9e41801655..86e9cff217 100644
--- a/inlong-dataproxy/conf/dataproxy-tubemq.conf
+++ b/inlong-dataproxy/conf/dataproxy-tubemq.conf
@@ -32,8 +32,6 @@ agent1.sources.tcp-source.port = 46801
 agent1.sources.tcp-source.highWaterMark=2621440
 agent1.sources.tcp-source.enableExceptionReturn=true
 agent1.sources.tcp-source.max-msg-length = 524288
-agent1.sources.tcp-source.topic = test_token
-agent1.sources.tcp-source.attr = m=9
 agent1.sources.tcp-source.connections = 5000
 agent1.sources.tcp-source.max-threads = 64
 agent1.sources.tcp-source.receiveBufferSize = 524288
@@ -55,14 +53,12 @@ agent1.sources.tcp-source.max-monitor-cnt=500000
 
 # http-source
 agent1.sources.http-source.channels = ch-msg1 ch-msg2 ch-msg3 ch-msg5 ch-msg6 ch-msg7 ch-msg8 ch-msg9 ch-msg10 ch-back
-agent1.sources.http-source.type = org.apache.inlong.dataproxy.http.SimpleHttpSource
-agent1.sources.http-source.message-handler-name = org.apache.inlong.dataproxy.http.SimpleMessageHandler
+agent1.sources.http-source.type = org.apache.inlong.dataproxy.source.SimpleHttpSource
+agent1.sources.http-source.message-handler-name = org.apache.inlong.dataproxy.source.httpMsg.SimpleMessageHandler
 # replace it with the ip you specified
 agent1.sources.http-source.host = localhost
 agent1.sources.http-source.port = 46802
 agent1.sources.http-source.max-msg-length = 524288
-#agent1.sources.http-source.topic = persistent://public/default/dataproxy-default-topic
-agent1.sources.http-source.attr = m=9
 agent1.sources.http-source.connections = 50000
 agent1.sources.http-source.max-threads = 64
 agent1.sources.http-source.receiveBufferSize = 1048576
diff --git a/inlong-dataproxy/conf/dataproxy.conf b/inlong-dataproxy/conf/dataproxy.conf
index c9853fbea6..35ad6f8d03 100644
--- a/inlong-dataproxy/conf/dataproxy.conf
+++ b/inlong-dataproxy/conf/dataproxy.conf
@@ -43,8 +43,6 @@ agent1.sources.tcp-source.msg-factory-name = org.apache.inlong.dataproxy.source.
 agent1.sources.tcp-source.host = localhost
 agent1.sources.tcp-source.port = 46801
 agent1.sources.tcp-source.max-msg-length = 524288
-# agent1.sources.tcp-source.topic = persistent://public/default/dataproxy-default-topic
-agent1.sources.tcp-source.attr = m=9
 agent1.sources.tcp-source.connections = 30000
 agent1.sources.tcp-source.max-threads = 64
 agent1.sources.tcp-source.receiveBufferSize = 8388608
@@ -62,14 +60,12 @@ agent1.sources.tcp-source.max-monitor-cnt=500000
 
 # http-source
 agent1.sources.http-source.channels = ch-msg1 ch-msg2 ch-msg5 ch-msg6
-agent1.sources.http-source.type = org.apache.inlong.dataproxy.http.SimpleHttpSource
-agent1.sources.http-source.message-handler-name = org.apache.inlong.dataproxy.http.SimpleMessageHandler
+agent1.sources.http-source.type = org.apache.inlong.dataproxy.source.SimpleHttpSource
+agent1.sources.http-source.message-handler-name = org.apache.inlong.dataproxy.source.httpMsg.HttpMessageHandler
 # replace it with the ip you specified
 agent1.sources.http-source.host = localhost
 agent1.sources.http-source.port = 46802
 agent1.sources.http-source.max-msg-length = 524288
-#agent1.sources.http-source.topic = persistent://public/default/dataproxy-default-topic
-agent1.sources.http-source.attr = m=9
 agent1.sources.http-source.connections = 30000
 agent1.sources.http-source.max-threads = 64
 agent1.sources.http-source.receiveBufferSize = 1048576
diff --git a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/http/Context.java b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/http/Context.java
deleted file mode 100644
index c40ff05539..0000000000
--- a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/http/Context.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.inlong.dataproxy.http;
-
-import java.util.Set;
-
-public interface Context {
-
-    void init();
-
-    void put(String key, Object value);
-
-    Object get(String key);
-
-    void remove(String key);
-
-    void clear();
-
-    void destroy();
-
-    boolean containsKey(String key);
-
-    Set<String> keySet();
-}
diff --git a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/http/HttpBaseSource.java b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/http/HttpBaseSource.java
deleted file mode 100644
index 2dce0d7e25..0000000000
--- a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/http/HttpBaseSource.java
+++ /dev/null
@@ -1,155 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.inlong.dataproxy.http;
-
-import org.apache.inlong.common.metric.MetricRegister;
-import org.apache.inlong.common.monitor.MonitorIndex;
-import org.apache.inlong.common.monitor.MonitorIndexExt;
-import org.apache.inlong.dataproxy.channel.FailoverChannelProcessor;
-import org.apache.inlong.dataproxy.config.CommonConfigHolder;
-import org.apache.inlong.dataproxy.consts.ConfigConstants;
-import org.apache.inlong.dataproxy.metrics.DataProxyMetricItemSet;
-import org.apache.inlong.dataproxy.utils.ConfStringUtils;
-
-import com.google.common.base.Preconditions;
-import org.apache.flume.ChannelSelector;
-import org.apache.flume.Context;
-import org.apache.flume.EventDrivenSource;
-import org.apache.flume.conf.Configurable;
-import org.apache.flume.conf.Configurables;
-import org.apache.flume.source.AbstractSource;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class HttpBaseSource extends AbstractSource implements EventDrivenSource, Configurable {
-
-    private static final Logger logger = LoggerFactory.getLogger(HttpBaseSource.class);
-    private static final String CONNECTIONS = "connections";
-    protected int port;
-    protected String host = null;
-    protected int maxMsgLength;
-    protected String topic;
-    protected String attr;
-    protected String messageHandlerName;
-    protected boolean filterEmptyMsg;
-    protected int maxConnections = Integer.MAX_VALUE;
-    protected boolean customProcessor = false;
-    protected Context context;
-    // statistic
-    protected MonitorIndex monitorIndex = null;
-    protected MonitorIndexExt monitorIndexExt = null;
-    private int statIntervalSec = 60;
-    private int maxMonitorCnt = ConfigConstants.DEF_MONITOR_STAT_CNT;
-    // audit
-    protected DataProxyMetricItemSet metricItemSet;
-
-    public HttpBaseSource() {
-        super();
-    }
-
-    @Override
-    public synchronized void start() {
-        logger.info("{} starting...", this.getName());
-        if (customProcessor) {
-            ChannelSelector selector = getChannelProcessor().getSelector();
-            FailoverChannelProcessor newProcessor = new FailoverChannelProcessor(selector);
-            newProcessor.configure(this.context);
-            setChannelProcessor(newProcessor);
-        }
-        if (statIntervalSec > 0) {
-            monitorIndex = new MonitorIndex("Source",
-                    statIntervalSec, maxMonitorCnt);
-            monitorIndexExt = new MonitorIndexExt("DataProxy_monitors#http",
-                    statIntervalSec, maxMonitorCnt);
-        }
-        // register metrics
-        String clusterId = CommonConfigHolder.getInstance().getClusterName();
-        this.metricItemSet =
-                new DataProxyMetricItemSet(clusterId, this.getName(), String.valueOf(port));
-        MetricRegister.register(metricItemSet);
-        super.start();
-        logger.info("{} started!", this.getName());
-    }
-
-    @Override
-    public synchronized void stop() {
-        logger.info("{} stopping...", this.getName());
-        if (statIntervalSec > 0) {
-            try {
-                monitorIndex.shutDown();
-            } catch (Exception e) {
-                logger.warn("Stats runner exception ", e);
-            }
-        }
-        super.stop();
-        logger.info("{} stopped!", this.getName());
-    }
-
-    /**
-     * configure
-     */
-    public void configure(Context context) {
-        this.context = context;
-        port = context.getInteger(ConfigConstants.CONFIG_PORT);
-        host = context.getString(ConfigConstants.CONFIG_HOST, "0.0.0.0");
-
-        Configurables.ensureRequiredNonNull(context, ConfigConstants.CONFIG_PORT);
-
-        Preconditions.checkArgument(ConfStringUtils.isValidIp(host), "ip config not valid");
-        Preconditions.checkArgument(ConfStringUtils.isValidPort(port), "port config not valid");
-
-        maxMsgLength = context.getInteger(ConfigConstants.MAX_MSG_LENGTH, 1024 * 64);
-        Preconditions.checkArgument(
-                (maxMsgLength >= 4 && maxMsgLength <= ConfigConstants.MSG_MAX_LENGTH_BYTES),
-                "maxMsgLength must be >= 4 and <= 65536");
-
-        topic = context.getString(ConfigConstants.TOPIC);
-        attr = context.getString(ConfigConstants.ATTR);
-
-        attr = attr.trim();
-        Preconditions.checkArgument(!attr.isEmpty(), "attr is empty");
-
-        messageHandlerName = context.getString(ConfigConstants.MESSAGE_HANDLER_NAME,
-                "org.apache.inlong.dataproxy.source.ServerMessageHandler");
-        messageHandlerName = messageHandlerName.trim();
-        Preconditions.checkArgument(!messageHandlerName.isEmpty(), "messageHandlerName is empty");
-
-        filterEmptyMsg = context.getBoolean(ConfigConstants.FILTER_EMPTY_MSG, false);
-        // get statistic interval
-        statIntervalSec = context.getInteger(ConfigConstants.STAT_INTERVAL_SEC, 60);
-        Preconditions.checkArgument((statIntervalSec >= 0), "statIntervalSec must be >= 0");
-        // get maxMonitorCnt's configure value
-        try {
-            maxMonitorCnt = context.getInteger(
-                    ConfigConstants.MAX_MONITOR_CNT, ConfigConstants.DEF_MONITOR_STAT_CNT);
-        } catch (NumberFormatException e) {
-            logger.warn("Property {} must specify an integer value: {}",
-                    ConfigConstants.MAX_MONITOR_CNT, context.getString(ConfigConstants.MAX_MONITOR_CNT));
-        }
-        Preconditions.checkArgument(maxMonitorCnt >= 0, "maxMonitorCnt must be >= 0");
-
-        customProcessor = context.getBoolean(ConfigConstants.CUSTOM_CHANNEL_PROCESSOR, false);
-
-        try {
-            maxConnections = context.getInteger(CONNECTIONS, 5000);
-        } catch (NumberFormatException e) {
-            logger.warn("BaseSource connections property must specify an integer value {}",
-                    context.getString(CONNECTIONS));
-        }
-    }
-}
diff --git a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/http/MappedContext.java b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/http/MappedContext.java
deleted file mode 100644
index 550208de5c..0000000000
--- a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/http/MappedContext.java
+++ /dev/null
@@ -1,72 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.inlong.dataproxy.http;
-
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Set;
-
-public class MappedContext implements Context {
-
-    private Map<String, Object> mapContext;
-
-    public MappedContext() {
-        init();
-    }
-
-    @Override
-    public void init() {
-        mapContext = new HashMap<>();
-    }
-
-    @Override
-    public void put(String key, Object value) {
-        mapContext.put(key, value);
-    }
-
-    @Override
-    public Object get(String key) {
-        return mapContext.get(key);
-    }
-
-    @Override
-    public void remove(String key) {
-        mapContext.remove(key);
-    }
-
-    @Override
-    public void clear() {
-        mapContext.clear();
-    }
-
-    @Override
-    public void destroy() {
-        mapContext = Collections.emptyMap();
-    }
-
-    @Override
-    public boolean containsKey(String key) {
-        return mapContext.containsKey(key);
-    }
-
-    @Override
-    public Set<String> keySet() {
-        return mapContext.keySet();
-    }
-}
diff --git a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/http/MessageFilter.java b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/http/MessageFilter.java
deleted file mode 100644
index e2272cd5c0..0000000000
--- a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/http/MessageFilter.java
+++ /dev/null
@@ -1,176 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.inlong.dataproxy.http;
-
-import org.apache.inlong.common.enums.DataProxyErrCode;
-import org.apache.inlong.common.msg.AttributeConstants;
-import org.apache.inlong.dataproxy.config.CommonConfigHolder;
-import org.apache.inlong.dataproxy.config.ConfigManager;
-import org.apache.inlong.dataproxy.consts.AttrConstants;
-
-import org.apache.commons.lang3.StringUtils;
-import org.apache.flume.ChannelException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import javax.servlet.Filter;
-import javax.servlet.FilterChain;
-import javax.servlet.FilterConfig;
-import javax.servlet.ServletException;
-import javax.servlet.ServletRequest;
-import javax.servlet.ServletResponse;
-import javax.servlet.http.HttpServletRequest;
-import javax.servlet.http.HttpServletResponse;
-
-import java.io.IOException;
-
-public class MessageFilter implements Filter {
-
-    private static final Logger LOG = LoggerFactory.getLogger(MessageFilter.class);
-
-    private final int maxMsgLength;
-
-    public MessageFilter(int maxMsgLength) {
-        this.maxMsgLength = maxMsgLength;
-    }
-
-    @Override
-    public void init(FilterConfig filterConfig) throws ServletException {
-    }
-
-    @Override
-    public void doFilter(ServletRequest request,
-            ServletResponse response,
-            FilterChain chain) throws IOException {
-        HttpServletRequest req = (HttpServletRequest) request;
-        HttpServletResponse resp = (HttpServletResponse) response;
-
-        String pathInfo = req.getPathInfo();
-        if (pathInfo.startsWith("/")) {
-            pathInfo = pathInfo.substring(1);
-        }
-        // check illegal ip
-        String strRemoteIP = req.getRemoteAddr();
-        if (ConfigManager.getInstance().needChkIllegalIP()
-                && ConfigManager.getInstance().isIllegalIP(strRemoteIP)) {
-            returnRspPackage(resp, req.getCharacterEncoding(),
-                    DataProxyErrCode.ILLEGAL_VISIT_IP.getErrCode(),
-                    DataProxyErrCode.ILLEGAL_VISIT_IP.getErrMsg() + " " + strRemoteIP);
-            return;
-        }
-        // process heartbeat request
-        if ("heartbeat".equals(pathInfo)) {
-            returnRspPackage(resp, req.getCharacterEncoding(),
-                    DataProxyErrCode.SUCCESS.getErrCode(),
-                    DataProxyErrCode.SUCCESS.getErrMsg());
-            return;
-        }
-        // check sink service status
-        if (!ConfigManager.getInstance().isMqClusterReady()) {
-            returnRspPackage(resp, req.getCharacterEncoding(),
-                    DataProxyErrCode.SINK_SERVICE_UNREADY.getErrCode(),
-                    DataProxyErrCode.SINK_SERVICE_UNREADY.getErrMsg());
-            return;
-        }
-        // get and check groupId
-        String groupId = req.getParameter(AttributeConstants.GROUP_ID);
-        if (StringUtils.isEmpty(groupId)) {
-            returnRspPackage(resp, req.getCharacterEncoding(),
-                    DataProxyErrCode.MISS_REQUIRED_GROUPID_ARGUMENT.getErrCode(),
-                    DataProxyErrCode.MISS_REQUIRED_GROUPID_ARGUMENT.getErrMsg());
-            return;
-        }
-        // get and check streamId
-        String streamId = req.getParameter(AttributeConstants.STREAM_ID);
-        if (StringUtils.isEmpty(streamId)) {
-            returnRspPackage(resp, req.getCharacterEncoding(),
-                    DataProxyErrCode.MISS_REQUIRED_STREAMID_ARGUMENT.getErrCode(),
-                    DataProxyErrCode.MISS_REQUIRED_STREAMID_ARGUMENT.getErrMsg());
-            return;
-        }
-        // get and check topicName
-        String topicName = ConfigManager.getInstance().getTopicName(groupId, streamId);
-        if (StringUtils.isBlank(topicName)
-                && !CommonConfigHolder.getInstance().isEnableUnConfigTopicAccept()) {
-            returnRspPackage(resp, req.getCharacterEncoding(),
-                    DataProxyErrCode.TOPIC_IS_BLANK.getErrCode(),
-                    DataProxyErrCode.TOPIC_IS_BLANK.getErrMsg());
-            return;
-        }
-        // get and check dt
-        String dt = req.getParameter(AttributeConstants.DATA_TIME);
-        if (StringUtils.isEmpty(dt)) {
-            returnRspPackage(resp, req.getCharacterEncoding(),
-                    DataProxyErrCode.MISS_REQUIRED_DT_ARGUMENT.getErrCode(),
-                    DataProxyErrCode.MISS_REQUIRED_DT_ARGUMENT.getErrMsg());
-            return;
-        }
-        // get and check body
-        String body = req.getParameter(AttrConstants.BODY);
-        if (StringUtils.isEmpty(body)) {
-            returnRspPackage(resp, req.getCharacterEncoding(),
-                    DataProxyErrCode.MISS_REQUIRED_BODY_ARGUMENT.getErrCode(),
-                    DataProxyErrCode.MISS_REQUIRED_BODY_ARGUMENT.getErrMsg());
-            return;
-        }
-        // check body length
-        if (body.length() <= 0) {
-            returnRspPackage(resp, req.getCharacterEncoding(),
-                    DataProxyErrCode.EMPTY_MSG.getErrCode(),
-                    "Bad request, body length <= 0");
-            return;
-        }
-        if (body.length() > maxMsgLength) {
-            returnRspPackage(resp, req.getCharacterEncoding(),
-                    DataProxyErrCode.BODY_EXCEED_MAX_LEN.getErrCode(),
-                    "Bad request, body length exceeds the limit:" + maxMsgLength);
-            return;
-        }
-        try {
-            chain.doFilter(request, response);
-            returnRspPackage(resp, req.getCharacterEncoding(),
-                    DataProxyErrCode.SUCCESS.getErrCode(),
-                    DataProxyErrCode.SUCCESS.getErrMsg());
-        } catch (Throwable t) {
-            String errMsg;
-            if ((t instanceof ChannelException)) {
-                errMsg = "Channel error! " + t.getMessage();
-            } else {
-                errMsg = "Service error! " + t.getMessage();
-            }
-            LOG.error("Request error!", t);
-            returnRspPackage(resp, req.getCharacterEncoding(),
-                    DataProxyErrCode.UNKNOWN_ERROR.getErrCode(), errMsg);
-        }
-    }
-
-    @Override
-    public void destroy() {
-    }
-
-    private void returnRspPackage(HttpServletResponse resp, String charEncoding,
-            int errCode, String errMsg) throws IOException {
-        StringBuilder builder =
-                new StringBuilder().append("{\"code\":\"").append(errCode)
-                        .append("\",\"msg\":\"").append(errMsg).append("\"}");
-        resp.setCharacterEncoding(charEncoding);
-        resp.setStatus(HttpServletResponse.SC_OK);
-        resp.getWriter().write(builder.toString());
-        resp.flushBuffer();
-    }
-}
diff --git a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/http/MessageHandler.java b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/http/MessageHandler.java
deleted file mode 100644
index 59f16ac1f2..0000000000
--- a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/http/MessageHandler.java
+++ /dev/null
@@ -1,31 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.inlong.dataproxy.http;
-
-import org.apache.inlong.dataproxy.http.exception.MessageProcessException;
-
-import org.apache.flume.conf.Configurable;
-
-public interface MessageHandler extends Configurable {
-
-    void init();
-
-    void processMessage(Context context) throws MessageProcessException;
-
-    void destroy();
-}
diff --git a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/http/MessageProcessServlet.java b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/http/MessageProcessServlet.java
deleted file mode 100644
index 5f8f125605..0000000000
--- a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/http/MessageProcessServlet.java
+++ /dev/null
@@ -1,73 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.inlong.dataproxy.http;
-
-import org.apache.inlong.common.enums.DataProxyErrCode;
-import org.apache.inlong.common.monitor.LogCounter;
-import org.apache.inlong.common.msg.AttributeConstants;
-import org.apache.inlong.dataproxy.consts.AttrConstants;
-import org.apache.inlong.dataproxy.http.exception.MessageProcessException;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import javax.servlet.ServletException;
-import javax.servlet.http.HttpServlet;
-import javax.servlet.http.HttpServletRequest;
-import javax.servlet.http.HttpServletResponse;
-
-import java.io.IOException;
-
-public class MessageProcessServlet extends HttpServlet {
-
-    private static final Logger LOG = LoggerFactory.getLogger(MessageProcessServlet.class);
-    private static final LogCounter logCounter = new LogCounter(10, 100000, 60 * 1000);
-
-    private MessageHandler messageHandler;
-
-    public MessageProcessServlet(MessageHandler messageHandler) {
-        this.messageHandler = messageHandler;
-    }
-
-    @Override
-    protected void doGet(HttpServletRequest req, HttpServletResponse resp)
-            throws ServletException, IOException {
-        doPost(req, resp);
-    }
-
-    @Override
-    protected void doPost(HttpServletRequest req, HttpServletResponse resp) {
-        try {
-            Context context = new MappedContext();
-            context.put(AttributeConstants.GROUP_ID, req.getParameter(AttributeConstants.GROUP_ID));
-            context.put(AttributeConstants.STREAM_ID, req.getParameter(AttributeConstants.STREAM_ID));
-            context.put(AttributeConstants.DATA_TIME, req.getParameter(AttributeConstants.DATA_TIME));
-            context.put(AttrConstants.BODY, req.getParameter(AttrConstants.BODY));
-
-            context.put(AttrConstants.HTTP_REQUEST, req);
-            context.put(AttrConstants.HTTP_RESPONSE, resp);
-
-            messageHandler.processMessage(context);
-        } catch (MessageProcessException e) {
-            if (logCounter.shouldPrint()) {
-                LOG.error("Received bad request from client. ", e);
-            }
-            req.setAttribute("code", DataProxyErrCode.UNKNOWN_ERROR.getErrCode());
-        }
-    }
-}
diff --git a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/http/SimpleHttpSource.java b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/http/SimpleHttpSource.java
deleted file mode 100644
index a847f8c932..0000000000
--- a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/http/SimpleHttpSource.java
+++ /dev/null
@@ -1,195 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.inlong.dataproxy.http;
-
-import org.apache.inlong.common.monitor.MonitorIndex;
-import org.apache.inlong.common.monitor.MonitorIndexExt;
-import org.apache.inlong.dataproxy.config.ConfigManager;
-import org.apache.inlong.dataproxy.config.remote.ConfigMessageServlet;
-import org.apache.inlong.dataproxy.metrics.DataProxyMetricItemSet;
-import org.apache.inlong.dataproxy.source.ServiceDecoder;
-
-import com.google.common.base.Preconditions;
-import com.google.common.base.Throwables;
-import org.apache.flume.Context;
-import org.apache.flume.channel.ChannelProcessor;
-import org.apache.flume.source.http.HTTPSource;
-import org.apache.flume.source.http.HTTPSourceConfigurationConstants;
-import org.eclipse.jetty.server.Connector;
-import org.eclipse.jetty.server.HttpConfiguration;
-import org.eclipse.jetty.server.HttpConnectionFactory;
-import org.eclipse.jetty.server.SecureRequestCustomizer;
-import org.eclipse.jetty.server.Server;
-import org.eclipse.jetty.server.ServerConnector;
-import org.eclipse.jetty.server.SslConnectionFactory;
-import org.eclipse.jetty.servlet.FilterHolder;
-import org.eclipse.jetty.servlet.ServletContextHandler;
-import org.eclipse.jetty.servlet.ServletHolder;
-import org.eclipse.jetty.util.ssl.SslContextFactory;
-import org.eclipse.jetty.util.thread.QueuedThreadPool;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import javax.servlet.DispatcherType;
-
-import java.lang.reflect.Constructor;
-import java.util.EnumSet;
-import java.util.Map;
-
-public class SimpleHttpSource extends HttpBaseSource {
-
-    private static final Logger LOG = LoggerFactory.getLogger(HTTPSource.class);
-
-    public static final String POOL_SIZE = "poolSize";
-    public static final String IDEL_TIME = "idelTime";
-    public static final String BUFFER_SIZE = "bufferSize";
-    public static final String BACKLOG = "backlog";
-
-    private volatile Integer port;
-    private volatile Server srv;
-    private Map<String, String> subProps;
-    private MessageHandler messageHandler;
-
-    // SSL configuration variable
-    private volatile String keyStorePath;
-    private volatile String keyStorePassword;
-    private volatile Boolean sslEnabled;
-
-    private int threadPoolSize = 512;
-    private int maxIdelTime = 600000;
-    private int requestBufferSize = 10000;
-    private int backlog = 2048;
-
-    @Override
-    public void configure(Context context) {
-        super.configure(context);
-        try {
-            port = context.getInteger(HTTPSourceConfigurationConstants.CONFIG_PORT);
-            threadPoolSize = context.getInteger(POOL_SIZE, 512);
-            maxIdelTime = context.getInteger(IDEL_TIME, 600000);
-            requestBufferSize = context.getInteger(BUFFER_SIZE, 10000);
-            backlog = context.getInteger(BACKLOG, 2048);
-            LOG.info("http backlog set to {}", backlog);
-            checkPort();
-
-            // SSL related config
-            sslEnabled = context.getBoolean(HTTPSourceConfigurationConstants.SSL_ENABLED, false);
-            if (sslEnabled) {
-                LOG.debug("SSL configuration enabled");
-                keyStorePath = context.getString(HTTPSourceConfigurationConstants.SSL_KEYSTORE);
-                Preconditions.checkArgument(keyStorePath != null && !keyStorePath.isEmpty(),
-                        "Keystore is required for SSL Conifguration");
-                keyStorePassword =
-                        context.getString(HTTPSourceConfigurationConstants.SSL_KEYSTORE_PASSWORD);
-                Preconditions.checkArgument(keyStorePassword != null,
-                        "Keystore password is required for SSL Configuration");
-            }
-
-            // ref: http://docs.codehaus.org/display/JETTY/Embedding+Jetty
-            // ref: http://jetty.codehaus.org/jetty/jetty-6/apidocs/org/mortbay/jetty/servlet
-            // /Context.html
-            subProps = context.getSubProperties(
-                    HTTPSourceConfigurationConstants.CONFIG_HANDLER_PREFIX);
-        } catch (Exception ex) {
-            LOG.error("Error configuring HTTPSource!", ex);
-            Throwables.propagate(ex);
-        }
-    }
-
-    @Override
-    public synchronized void stop() {
-        super.stop();
-    }
-
-    @Override
-    public synchronized void start() {
-        super.start();
-        try {
-
-            @SuppressWarnings("unchecked")
-            Class<? extends MessageHandler> clazz =
-                    (Class<? extends MessageHandler>) Class.forName(messageHandlerName);
-            Constructor ctor = clazz.getConstructor(ChannelProcessor.class,
-                    MonitorIndex.class, MonitorIndexExt.class, DataProxyMetricItemSet.class, ServiceDecoder.class);
-            LOG.info("Using channel processor:{}", getChannelProcessor().getClass().getName());
-            messageHandler = (MessageHandler) ctor
-                    .newInstance(getChannelProcessor(), monitorIndex, monitorIndexExt, metricItemSet, null);
-            messageHandler.configure(new Context(subProps));
-            srv = new Server(new QueuedThreadPool(threadPoolSize));
-            Connector[] connectors = new Connector[1];
-            if (sslEnabled) {
-                SslContextFactory sslContextFactory = new SslContextFactory(keyStorePath);
-                sslContextFactory.setKeyStorePassword(keyStorePassword);
-                SslConnectionFactory ssl = new SslConnectionFactory(sslContextFactory,
-                        "http/1.1");
-                HttpConfiguration httpsConfig = new HttpConfiguration();
-                httpsConfig.setSecureScheme("https");
-                httpsConfig.setSecurePort(port);
-                httpsConfig.addCustomizer(new SecureRequestCustomizer());
-                ServerConnector http2Connector = new ServerConnector(srv, ssl,
-                        new HttpConnectionFactory(httpsConfig));
-                srv.addConnector(http2Connector);
-                connectors[0] = http2Connector;
-                LOG.info("sslEnabled {}", sslEnabled);
-            } else {
-                ServerConnector connector = new ServerConnector(srv);
-                connector.setReuseAddress(true);
-                connector.setIdleTimeout(maxIdelTime);
-                connector.setAcceptedReceiveBufferSize(requestBufferSize);
-                connector.setAcceptQueueSize(backlog);
-                connector.setHost(host);
-                connector.setPort(port);
-                LOG.info("set config maxIdelTime {}, backlog {}", maxIdelTime, backlog);
-                connectors[0] = connector;
-            }
-
-            srv.setConnectors(connectors);
-
-            ServletContextHandler servletContext =
-                    new ServletContextHandler(srv, "/", ServletContextHandler.SESSIONS);
-            servletContext.setMaxFormContentSize(maxMsgLength);
-            servletContext
-                    .addFilter(new FilterHolder(new MessageFilter(maxMsgLength)), "/dataproxy/*",
-                            EnumSet.of(DispatcherType.REQUEST));
-            servletContext.addServlet(new ServletHolder(new MessageProcessServlet(messageHandler)),
-                    "/dataproxy/*");
-            servletContext.addServlet(new ServletHolder(new ConfigMessageServlet()),
-                    "/dataproxy/config/*");
-            srv.start();
-            ConfigManager.getInstance().addSourceReportInfo(
-                    host, String.valueOf(port), "HTTP");
-            Preconditions.checkArgument(srv.getHandler().equals(servletContext));
-        } catch (ClassNotFoundException ex) {
-            LOG.error("Error while configuring HTTPSource. Exception follows.", ex);
-            Throwables.propagate(ex);
-        } catch (ClassCastException ex) {
-            LOG.error("Deserializer is not an instance of HTTPSourceHandler."
-                    + "Deserializer must implement HTTPSourceHandler.");
-            Throwables.propagate(ex);
-        } catch (Exception ex) {
-            LOG.error("Error while starting HTTPSource. Exception follows.", ex);
-            Throwables.propagate(ex);
-        }
-        Preconditions.checkArgument(srv.isRunning());
-    }
-
-    private void checkPort() {
-        Preconditions.checkNotNull(port, "HTTPSource requires a port number to be"
-                + "specified");
-    }
-}
diff --git a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/http/SimpleMessageHandler.java b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/http/SimpleMessageHandler.java
deleted file mode 100644
index 2a37012d0d..0000000000
--- a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/http/SimpleMessageHandler.java
+++ /dev/null
@@ -1,236 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.inlong.dataproxy.http;
-
-import org.apache.inlong.common.enums.DataProxyMsgEncType;
-import org.apache.inlong.common.monitor.MonitorIndex;
-import org.apache.inlong.common.monitor.MonitorIndexExt;
-import org.apache.inlong.common.msg.AttributeConstants;
-import org.apache.inlong.common.msg.InLongMsg;
-import org.apache.inlong.common.util.NetworkUtils;
-import org.apache.inlong.dataproxy.config.ConfigManager;
-import org.apache.inlong.dataproxy.consts.AttrConstants;
-import org.apache.inlong.dataproxy.consts.ConfigConstants;
-import org.apache.inlong.dataproxy.http.exception.MessageProcessException;
-import org.apache.inlong.dataproxy.metrics.DataProxyMetricItemSet;
-import org.apache.inlong.dataproxy.metrics.audit.AuditUtils;
-import org.apache.inlong.dataproxy.source.ServiceDecoder;
-import org.apache.inlong.dataproxy.utils.DateTimeUtils;
-import org.apache.inlong.dataproxy.utils.MessageUtils;
-import org.apache.inlong.sdk.commons.protocol.EventConstants;
-
-import org.apache.commons.lang3.StringUtils;
-import org.apache.commons.lang3.math.NumberUtils;
-import org.apache.commons.lang3.tuple.Pair;
-import org.apache.flume.ChannelException;
-import org.apache.flume.Event;
-import org.apache.flume.channel.ChannelProcessor;
-import org.apache.flume.event.EventBuilder;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import javax.servlet.http.HttpServletRequest;
-
-import java.io.UnsupportedEncodingException;
-import java.util.HashMap;
-import java.util.Map;
-
-import static org.apache.inlong.dataproxy.consts.AttrConstants.SEP_HASHTAG;
-
-public class SimpleMessageHandler implements MessageHandler {
-
-    private static final Logger LOG = LoggerFactory.getLogger(SimpleMessageHandler.class);
-    private static final ConfigManager configManager = ConfigManager.getInstance();
-
-    private static final String DEFAULT_REMOTE_IDC_VALUE = "0";
-    private final MonitorIndex monitorIndex;
-    private final MonitorIndexExt monitorIndexExt;
-    private final DataProxyMetricItemSet metricItemSet;
-    private final ChannelProcessor processor;
-
-    @SuppressWarnings("unused")
-    private int maxMsgLength;
-    private long logCounter = 0L;
-    private long channelTrace = 0L;
-
-    public SimpleMessageHandler(ChannelProcessor processor, MonitorIndex monitorIndex,
-            MonitorIndexExt monitorIndexExt, DataProxyMetricItemSet metricItemSet,
-            ServiceDecoder decoder) {
-        this.processor = processor;
-        this.monitorIndex = monitorIndex;
-        this.monitorIndexExt = monitorIndexExt;
-        this.metricItemSet = metricItemSet;
-        init();
-    }
-
-    @Override
-    public void init() {
-    }
-
-    @Override
-    public void destroy() {
-    }
-
-    @Override
-    public void processMessage(Context context) throws MessageProcessException {
-        StringBuilder strBuff = new StringBuilder(512);
-        // get groupId and streamId
-        String groupId = (String) context.get(AttributeConstants.GROUP_ID);
-        String streamId = (String) context.get(AttributeConstants.STREAM_ID);
-        if (StringUtils.isBlank(groupId) || StringUtils.isBlank(streamId)) {
-            throw new MessageProcessException(strBuff.append("Field ")
-                    .append(AttributeConstants.GROUP_ID).append(" or ")
-                    .append(AttributeConstants.STREAM_ID)
-                    .append(" must exist and not blank!").toString());
-        }
-        groupId = groupId.trim();
-        streamId = streamId.trim();
-        // get topicName
-        String topicName = configManager.getTopicName(groupId, streamId);
-        if (StringUtils.isBlank(topicName)) {
-            throw new MessageProcessException(strBuff
-                    .append("Topic for message is null, inlongGroupId = ")
-                    .append(groupId).append(", inlongStreamId = ").append(streamId).toString());
-        }
-        // get message data time
-        final long msgRcvTime = System.currentTimeMillis();
-        String strDataTime = (String) context.get(AttributeConstants.DATA_TIME);
-        long longDataTime = NumberUtils.toLong(strDataTime, msgRcvTime);
-        strDataTime = String.valueOf(longDataTime);
-        // get char set
-        String charset = (String) context.get(AttrConstants.CHARSET);
-        if (StringUtils.isBlank(charset)) {
-            charset = AttrConstants.CHARSET;
-        }
-        String body = (String) context.get(AttrConstants.BODY);
-        if (StringUtils.isEmpty(body)) {
-            throw new MessageProcessException(strBuff.append("Field ")
-                    .append(AttrConstants.BODY)
-                    .append(" must exist and not empty!").toString());
-        }
-        // convert context to http request
-        HttpServletRequest request =
-                (HttpServletRequest) context.get(AttrConstants.HTTP_REQUEST);
-        // get report node ip
-        String strRemoteIP = request.getRemoteAddr();
-        // get message count
-        String strMsgCount = request.getParameter(AttributeConstants.MESSAGE_COUNT);
-        int intMsgCnt = NumberUtils.toInt(strMsgCount, 1);
-        strMsgCount = String.valueOf(intMsgCnt);
-        // build message attributes
-        InLongMsg inLongMsg = InLongMsg.newInLongMsg(true);
-        strBuff.append("groupId=").append(groupId)
-                .append("&streamId=").append(streamId)
-                .append("&dt=").append(strDataTime)
-                .append("&NodeIP=").append(strRemoteIP)
-                .append("&cnt=").append(strMsgCount)
-                .append("&rt=").append(msgRcvTime)
-                .append(AttributeConstants.SEPARATOR).append(AttributeConstants.MSG_RPT_TIME)
-                .append(AttributeConstants.KEY_VALUE_SEPARATOR).append(msgRcvTime);
-        try {
-            inLongMsg.addMsg(strBuff.toString(), body.getBytes(charset));
-            strBuff.delete(0, strBuff.length());
-        } catch (UnsupportedEncodingException e) {
-            throw new MessageProcessException(e);
-        }
-        // build flume event
-        Map<String, String> headers = new HashMap<>();
-        headers.put(AttributeConstants.GROUP_ID, groupId);
-        headers.put(AttributeConstants.STREAM_ID, streamId);
-        headers.put(ConfigConstants.TOPIC_KEY, topicName);
-        headers.put(AttributeConstants.DATA_TIME, strDataTime);
-        headers.put(ConfigConstants.REMOTE_IP_KEY, strRemoteIP);
-        headers.put(ConfigConstants.REMOTE_IDC_KEY, DEFAULT_REMOTE_IDC_VALUE);
-        headers.put(ConfigConstants.MSG_COUNTER_KEY, strMsgCount);
-        headers.put(ConfigConstants.MSG_ENCODE_VER,
-                DataProxyMsgEncType.MSG_ENCODE_TYPE_INLONGMSG.getStrId());
-        headers.put(EventConstants.HEADER_KEY_VERSION,
-                DataProxyMsgEncType.MSG_ENCODE_TYPE_INLONGMSG.getStrId());
-        byte[] data = inLongMsg.buildArray();
-        headers.put(AttributeConstants.RCV_TIME, String.valueOf(msgRcvTime));
-        Event event = EventBuilder.withBody(data, headers);
-        inLongMsg.reset();
-        Pair<Boolean, String> evenProcType =
-                MessageUtils.getEventProcType("", "");
-        // build metric data item
-        longDataTime = longDataTime / 1000 / 60 / 10;
-        longDataTime = longDataTime * 1000 * 60 * 10;
-        strBuff.append("http").append(SEP_HASHTAG).append(groupId).append(SEP_HASHTAG)
-                .append(streamId).append(SEP_HASHTAG).append(strRemoteIP).append(SEP_HASHTAG)
-                .append(NetworkUtils.getLocalIp()).append(SEP_HASHTAG)
-                .append(evenProcType.getRight()).append(SEP_HASHTAG)
-                .append(DateTimeUtils.ms2yyyyMMddHHmm(longDataTime)).append(SEP_HASHTAG)
-                .append(DateTimeUtils.ms2yyyyMMddHHmm(msgRcvTime));
-        long beginTime = System.currentTimeMillis();
-        try {
-            processor.processEvent(event);
-            if (monitorIndex != null) {
-                monitorIndex.addAndGet(strBuff.toString(),
-                        intMsgCnt, 1, data.length, 0);
-                monitorIndexExt.incrementAndGet("EVENT_SUCCESS");
-            }
-            addStatistics(true, data.length, event);
-        } catch (ChannelException ex) {
-            if (monitorIndex != null) {
-                monitorIndex.addAndGet(strBuff.toString(),
-                        0, 0, 0, intMsgCnt);
-                monitorIndexExt.incrementAndGet("EVENT_DROPPED");
-            }
-            addStatistics(false, data.length, event);
-            logCounter++;
-            if (logCounter == 1 || logCounter % 1000 == 0) {
-                LOG.error("Error writing to channel, and will retry after 1s, ex={},"
-                        + "logCounter={}, spend time={} ms", ex, logCounter, System.currentTimeMillis() - beginTime);
-                if (logCounter > Long.MAX_VALUE - 10) {
-                    logCounter = 0;
-                    LOG.info("logCounter will reverse");
-                }
-            }
-            throw ex;
-        }
-        channelTrace++;
-        if (channelTrace % 600000 == 0) {
-            LOG.info("processor.processEvent spend time={} ms", System.currentTimeMillis() - beginTime);
-        }
-        if (channelTrace > Long.MAX_VALUE - 10) {
-            channelTrace = 0;
-            LOG.info("channelTrace will reverse");
-        }
-    }
-
-    @Override
-    public void configure(org.apache.flume.Context context) {
-    }
-
-    /**
-     * add statistics information
-     *
-     * @param isSuccess  success or failure
-     * @param size    message size
-     * @param event   message event
-     */
-    private void addStatistics(boolean isSuccess, long size, Event event) {
-        if (event == null) {
-            return;
-        }
-        metricItemSet.fillSrcMetricItemsByEvent(event, isSuccess, size);
-        if (isSuccess) {
-            AuditUtils.add(AuditUtils.AUDIT_ID_DATAPROXY_READ_SUCCESS, event);
-        }
-    }
-}
diff --git a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/http/exception/MessageProcessException.java b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/http/exception/MessageProcessException.java
deleted file mode 100644
index 1848c325dc..0000000000
--- a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/http/exception/MessageProcessException.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.inlong.dataproxy.http.exception;
-
-public class MessageProcessException
-        extends
-            Exception {
-
-    public MessageProcessException() {
-    }
-
-    public MessageProcessException(String message) {
-        super(message);
-    }
-
-    public MessageProcessException(String message, Throwable cause) {
-        super(message, cause);
-    }
-
-    public MessageProcessException(Throwable cause) {
-        super(cause);
-    }
-}
diff --git a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/sink/mq/SimplePackProfile.java b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/sink/mq/SimplePackProfile.java
index fa292893e0..c131a4172a 100644
--- a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/sink/mq/SimplePackProfile.java
+++ b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/sink/mq/SimplePackProfile.java
@@ -24,7 +24,7 @@ import org.apache.inlong.common.msg.MsgType;
 import org.apache.inlong.common.util.NetworkUtils;
 import org.apache.inlong.dataproxy.base.SinkRspEvent;
 import org.apache.inlong.dataproxy.consts.ConfigConstants;
-import org.apache.inlong.dataproxy.source2.InLongMessageHandler;
+import org.apache.inlong.dataproxy.source.ServerMessageHandler;
 import org.apache.inlong.sdk.commons.protocol.EventConstants;
 import org.apache.inlong.sdk.commons.protocol.InlongId;
 
@@ -203,9 +203,9 @@ public class SimplePackProfile extends PackProfile {
             // build and send response message
             ByteBuf retData;
             if (MsgType.MSG_BIN_MULTI_BODY.equals(msgType)) {
-                retData = InLongMessageHandler.buildBinMsgRspPackage(strBuff.toString(), Long.parseLong(uid));
+                retData = ServerMessageHandler.buildBinMsgRspPackage(strBuff.toString(), Long.parseLong(uid));
             } else {
-                retData = InLongMessageHandler.buildTxtMsgRspPackage(msgType, strBuff.toString());
+                retData = ServerMessageHandler.buildTxtMsgRspPackage(msgType, strBuff.toString());
             }
             strBuff.delete(0, strBuff.length());
             if (channel == null || !channel.isWritable()) {
diff --git a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source/BaseSource.java b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source/BaseSource.java
index bcebe0d879..3917d22b83 100644
--- a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source/BaseSource.java
+++ b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source/BaseSource.java
@@ -18,147 +18,222 @@
 package org.apache.inlong.dataproxy.source;
 
 import org.apache.inlong.common.metric.MetricRegister;
-import org.apache.inlong.common.monitor.MonitorIndex;
-import org.apache.inlong.common.monitor.MonitorIndexExt;
+import org.apache.inlong.dataproxy.admin.ProxyServiceMBean;
 import org.apache.inlong.dataproxy.channel.FailoverChannelProcessor;
 import org.apache.inlong.dataproxy.config.CommonConfigHolder;
-import org.apache.inlong.dataproxy.consts.ConfigConstants;
+import org.apache.inlong.dataproxy.config.ConfigManager;
+import org.apache.inlong.dataproxy.config.holder.ConfigUpdateCallback;
+import org.apache.inlong.dataproxy.consts.AttrConstants;
+import org.apache.inlong.dataproxy.metrics.DataProxyMetricItem;
 import org.apache.inlong.dataproxy.metrics.DataProxyMetricItemSet;
+import org.apache.inlong.dataproxy.metrics.audit.AuditUtils;
+import org.apache.inlong.dataproxy.metrics.stats.MonitorIndex;
+import org.apache.inlong.dataproxy.metrics.stats.MonitorStats;
+import org.apache.inlong.dataproxy.source.httpMsg.HttpMessageHandler;
+import org.apache.inlong.dataproxy.utils.AddressUtils;
 import org.apache.inlong.dataproxy.utils.ConfStringUtils;
 import org.apache.inlong.dataproxy.utils.FailoverChannelProcessorHolder;
+import org.apache.inlong.sdk.commons.admin.AdminServiceRegister;
 
 import com.google.common.base.Preconditions;
+import io.netty.channel.Channel;
 import io.netty.channel.ChannelFuture;
 import io.netty.channel.ChannelInitializer;
 import io.netty.channel.EventLoopGroup;
 import io.netty.channel.group.ChannelGroup;
 import io.netty.channel.group.DefaultChannelGroup;
-import io.netty.util.concurrent.DefaultThreadFactory;
 import io.netty.util.concurrent.GlobalEventExecutor;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.flume.ChannelSelector;
 import org.apache.flume.Context;
+import org.apache.flume.Event;
 import org.apache.flume.EventDrivenSource;
 import org.apache.flume.FlumeException;
 import org.apache.flume.conf.Configurable;
-import org.apache.flume.conf.Configurables;
 import org.apache.flume.source.AbstractSource;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.lang.reflect.Constructor;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
 
 /**
- * source base clase
- *
+ * source base class
  */
 public abstract class BaseSource
         extends
             AbstractSource
         implements
+            ConfigUpdateCallback,
+            ProxyServiceMBean,
             EventDrivenSource,
             Configurable {
 
     private static final Logger logger = LoggerFactory.getLogger(BaseSource.class);
 
     protected Context context;
-
-    protected int port;
-
-    protected String host = null;
-
+    // whether source reject service
+    protected volatile boolean isRejectService = false;
+    // source service host
+    protected String srcHost;
+    // source serviced port
+    protected int srcPort;
+    protected String strPort;
+    // message factory name
     protected String msgFactoryName;
-
-    protected String serviceDecoderName;
-
+    // message handler name
     protected String messageHandlerName;
-
+    // source default append attribute
+    protected String defAttr = "";
+    // allowed max message length
     protected int maxMsgLength;
-
+    // whether compress message
     protected boolean isCompressed;
-
-    protected String topic;
-
-    protected String attr;
-
+    // whether filter empty message
     protected boolean filterEmptyMsg;
-
-    private int statIntervalSec;
-
-    protected int pkgTimeoutSec;
-
-    protected int maxConnections = Integer.MAX_VALUE;
-
-    private static final String CONNECTIONS = "connections";
-
-    protected boolean customProcessor = false;
-
-    private DataProxyMetricItemSet metricItemSet;
-
-    /*
-     * monitor
-     */
-    private MonitorIndex monitorIndex;
-
-    private MonitorIndexExt monitorIndexExt;
-
-    /*
-     * netty server
-     */
-
+    // whether custom channel processor
+    protected boolean customProcessor;
+    // max netty worker threads
+    protected int maxWorkerThreads;
+    // max netty accept threads
+    protected int maxAcceptThreads;
+    // max read idle time
+    protected long maxReadIdleTimeMs;
+    // max connection count
+    protected int maxConnections;
+    // reuse address
+    protected boolean reuseAddress;
+    // connect backlog
+    protected int conBacklog;
+    // connect linger
+    protected int conLinger = -1;
+    // netty parameters
     protected EventLoopGroup acceptorGroup;
-
     protected EventLoopGroup workerGroup;
-
-    protected DefaultThreadFactory acceptorThreadFactory;
-
-    protected boolean enableBusyWait = false;
-
     protected ChannelGroup allChannels;
-
     protected ChannelFuture channelFuture;
-
-    private static String HOST_DEFAULT_VALUE = "0.0.0.0";
-
-    private static int maxMonitorCnt = ConfigConstants.DEF_MONITOR_STAT_CNT;
-
-    private static int DEFAULT_MAX_CONNECTIONS = 5000;
-
-    private static int STAT_INTERVAL_MUST_THAN = 0;
-
-    private static int PKG_TIMEOUT_DEFAULT_SEC = 3;
-
-    private static int MSG_MIN_LENGTH = 4;
-
-    private static int MAX_MSG_DEFAULT_LENGTH = 1024 * 64;
-
-    private static int INTERVAL_SEC = 60;
-
-    protected static int BUFFER_SIZE_MUST_THAN = 0;
-
-    protected static int DEFAULT_MAX_THREADS = 32;
-
-    protected static int RECEIVE_BUFFER_DEFAULT_SIZE = 64 * 1024;
-
-    protected static int SEND_BUFFER_DEFAULT_SIZE = 64 * 1024;
-
-    protected static int RECEIVE_BUFFER_MAX_SIZE = 16 * 1024 * 1024;
-
-    protected static int SEND_BUFFER_MAX_SIZE = 16 * 1024 * 1024;
-
-    protected int receiveBufferSize;
-
-    protected int sendBufferSize;
-
-    protected int maxThreads = 32;
-
-    protected int acceptorThreads = 1;
+    // receive buffer size
+    protected int maxRcvBufferSize;
+    // send buffer size
+    protected int maxSendBufferSize;
+    // file metric statistic
+    private MonitorIndex monitorIndex = null;
+    private MonitorStats monitorStats = null;
+    // metric set
+    private DataProxyMetricItemSet metricItemSet;
 
     public BaseSource() {
         super();
         allChannels = new DefaultChannelGroup("DefaultChannelGroup", GlobalEventExecutor.INSTANCE);
     }
 
+    @Override
+    public void configure(Context context) {
+        logger.info("{} start to configure context:{}.", this.getName(), context.toString());
+        this.context = context;
+        this.srcHost = getHostIp(context);
+        this.srcPort = getHostPort(context);
+        this.strPort = String.valueOf(this.srcPort);
+        // get message factory
+        String tmpVal = context.getString(SourceConstants.SRCCXT_MSG_FACTORY_NAME,
+                ServerMessageFactory.class.getName()).trim();
+        Preconditions.checkArgument(StringUtils.isNotBlank(tmpVal),
+                SourceConstants.SRCCXT_MSG_FACTORY_NAME + " config is blank");
+        this.msgFactoryName = tmpVal.trim();
+        // get message handler
+        tmpVal = context.getString(SourceConstants.SRCCXT_MESSAGE_HANDLER_NAME);
+        if (StringUtils.isBlank(tmpVal)) {
+            tmpVal = SourceConstants.SRC_PROTOCOL_TYPE_HTTP.equalsIgnoreCase(getProtocolName())
+                    ? HttpMessageHandler.class.getName()
+                    : ServerMessageHandler.class.getName();
+        }
+        Preconditions.checkArgument(StringUtils.isNotBlank(tmpVal),
+                SourceConstants.SRCCXT_MESSAGE_HANDLER_NAME + " config is blank");
+        this.messageHandlerName = tmpVal;
+        // get default attributes
+        tmpVal = context.getString(SourceConstants.SRCCXT_DEF_ATTR);
+        if (StringUtils.isNotBlank(tmpVal)) {
+            this.defAttr = tmpVal.trim();
+        }
+        // get allowed max message length
+        this.maxMsgLength = ConfStringUtils.getIntValue(context,
+                SourceConstants.SRCCXT_MAX_MSG_LENGTH, SourceConstants.VAL_DEF_MAX_MSG_LENGTH);
+        Preconditions.checkArgument((this.maxMsgLength >= SourceConstants.VAL_MIN_MAX_MSG_LENGTH
+                && this.maxMsgLength <= SourceConstants.VAL_MAX_MAX_MSG_LENGTH),
+                SourceConstants.SRCCXT_MAX_MSG_LENGTH + " must be in ["
+                        + SourceConstants.VAL_MIN_MAX_MSG_LENGTH + ", "
+                        + SourceConstants.VAL_MAX_MAX_MSG_LENGTH + "]");
+        // get whether compress message
+        this.isCompressed = context.getBoolean(SourceConstants.SRCCXT_MSG_COMPRESSED,
+                SourceConstants.VAL_DEF_MSG_COMPRESSED);
+        // get whether filter empty message
+        this.filterEmptyMsg = context.getBoolean(SourceConstants.SRCCXT_FILTER_EMPTY_MSG,
+                SourceConstants.VAL_DEF_FILTER_EMPTY_MSG);
+        // get whether custom channel processor
+        this.customProcessor = context.getBoolean(SourceConstants.SRCCXT_CUSTOM_CHANNEL_PROCESSOR,
+                SourceConstants.VAL_DEF_CUSTOM_CH_PROCESSOR);
+        // get max accept threads
+        this.maxAcceptThreads = ConfStringUtils.getIntValue(context,
+                SourceConstants.SRCCXT_MAX_ACCEPT_THREADS, SourceConstants.VAL_DEF_NET_ACCEPT_THREADS);
+        Preconditions.checkArgument((this.maxAcceptThreads >= SourceConstants.VAL_MIN_ACCEPT_THREADS
+                && this.maxAcceptThreads <= SourceConstants.VAL_MAX_ACCEPT_THREADS),
+                SourceConstants.SRCCXT_MAX_ACCEPT_THREADS + " must be in ["
+                        + SourceConstants.VAL_MIN_ACCEPT_THREADS + ", "
+                        + SourceConstants.VAL_MAX_ACCEPT_THREADS + "]");
+        // get max worker threads
+        this.maxWorkerThreads = ConfStringUtils.getIntValue(context,
+                SourceConstants.SRCCXT_MAX_WORKER_THREADS, SourceConstants.VAL_DEF_WORKER_THREADS);
+        Preconditions.checkArgument((this.maxWorkerThreads >= SourceConstants.VAL_MIN_WORKER_THREADS),
+                SourceConstants.SRCCXT_MAX_WORKER_THREADS + " must be >= "
+                        + SourceConstants.VAL_MIN_WORKER_THREADS);
+        // get max read idle time
+        this.maxReadIdleTimeMs = ConfStringUtils.getLongValue(context,
+                SourceConstants.SRCCXT_MAX_READ_IDLE_TIME_MS, SourceConstants.VAL_DEF_READ_IDLE_TIME_MS);
+        Preconditions.checkArgument((this.maxReadIdleTimeMs >= SourceConstants.VAL_MIN_READ_IDLE_TIME_MS
+                && this.maxReadIdleTimeMs <= SourceConstants.VAL_MAX_READ_IDLE_TIME_MS),
+                SourceConstants.SRCCXT_MAX_READ_IDLE_TIME_MS + " must be in ["
+                        + SourceConstants.VAL_MIN_READ_IDLE_TIME_MS + ", "
+                        + SourceConstants.VAL_MAX_READ_IDLE_TIME_MS + "]");
+        // get max connect count
+        this.maxConnections = ConfStringUtils.getIntValue(context,
+                SourceConstants.SRCCXT_MAX_CONNECTION_CNT, SourceConstants.VAL_DEF_MAX_CONNECTION_CNT);
+        Preconditions.checkArgument(this.maxConnections >= SourceConstants.VAL_MIN_CONNECTION_CNT,
+                SourceConstants.SRCCXT_MAX_CONNECTION_CNT + " must be >= "
+                        + SourceConstants.VAL_MIN_CONNECTION_CNT);
+        // get connect backlog
+        this.conBacklog = ConfStringUtils.getIntValue(context,
+                SourceConstants.SRCCXT_CONN_BACKLOG, SourceConstants.VAL_DEF_CONN_BACKLOG);
+        Preconditions.checkArgument(this.conBacklog >= SourceConstants.VAL_MIN_CONN_BACKLOG,
+                SourceConstants.SRCCXT_CONN_BACKLOG + " must be >= "
+                        + SourceConstants.VAL_MIN_CONN_BACKLOG);
+        // get connect linger
+        Integer tmpValue = context.getInteger(SourceConstants.SRCCXT_CONN_LINGER);
+        if (tmpValue != null && tmpValue >= 0) {
+            this.conLinger = tmpValue;
+        }
+        // get whether reuse address
+        this.reuseAddress = context.getBoolean(SourceConstants.SRCCXT_REUSE_ADDRESS,
+                SourceConstants.VAL_DEF_REUSE_ADDRESS);
+
+        // get whether custom channel processor
+        this.customProcessor = context.getBoolean(SourceConstants.SRCCXT_CUSTOM_CHANNEL_PROCESSOR,
+                SourceConstants.VAL_DEF_CUSTOM_CH_PROCESSOR);
+        // get max receive buffer size
+        this.maxRcvBufferSize = ConfStringUtils.getIntValue(context,
+                SourceConstants.SRCCXT_RECEIVE_BUFFER_SIZE, SourceConstants.VAL_DEF_RECEIVE_BUFFER_SIZE);
+        Preconditions.checkArgument(this.maxRcvBufferSize >= SourceConstants.VAL_MIN_RECEIVE_BUFFER_SIZE,
+                SourceConstants.SRCCXT_RECEIVE_BUFFER_SIZE + " must be >= "
+                        + SourceConstants.VAL_MIN_RECEIVE_BUFFER_SIZE);
+        // get max send buffer size
+        this.maxSendBufferSize = ConfStringUtils.getIntValue(context,
+                SourceConstants.SRCCXT_SEND_BUFFER_SIZE, SourceConstants.VAL_DEF_SEND_BUFFER_SIZE);
+        Preconditions.checkArgument(this.maxSendBufferSize >= SourceConstants.VAL_MIN_SEND_BUFFER_SIZE,
+                SourceConstants.SRCCXT_SEND_BUFFER_SIZE + " must be >= "
+                        + SourceConstants.VAL_MIN_SEND_BUFFER_SIZE);
+    }
+
     @Override
     public synchronized void start() {
         if (customProcessor) {
@@ -170,196 +245,336 @@ public abstract class BaseSource
         }
         super.start();
         // initial metric item set
-        String clusterId = CommonConfigHolder.getInstance().getClusterName();
-        this.metricItemSet =
-                new DataProxyMetricItemSet(clusterId, this.getName(), String.valueOf(port));
+        this.metricItemSet = new DataProxyMetricItemSet(
+                CommonConfigHolder.getInstance().getClusterName(), getName(), String.valueOf(srcPort));
         MetricRegister.register(metricItemSet);
-        /*
-         * init monitor logic
-         */
-        monitorIndex = new MonitorIndex("Source", INTERVAL_SEC, maxMonitorCnt);
-        monitorIndexExt = new MonitorIndexExt("DataProxy_monitors#"
-                + this.getProtocolName(), INTERVAL_SEC, maxMonitorCnt);
+        // init monitor logic
+        if (CommonConfigHolder.getInstance().isEnableFileMetric()) {
+            this.monitorIndex = new MonitorIndex(CommonConfigHolder.getInstance().getFileMetricSourceOutName(),
+                    CommonConfigHolder.getInstance().getFileMetricStatInvlSec() * 1000L,
+                    CommonConfigHolder.getInstance().getFileMetricStatCacheCnt());
+            this.monitorIndex.start();
+            this.monitorStats = new MonitorStats(
+                    CommonConfigHolder.getInstance().getFileMetricEventOutName()
+                            + AttrConstants.SEP_HASHTAG + this.getProtocolName(),
+                    CommonConfigHolder.getInstance().getFileMetricStatInvlSec() * 1000L,
+                    CommonConfigHolder.getInstance().getFileMetricStatCacheCnt());
+            this.monitorStats.start();
+        }
         startSource();
+        // register
+        AdminServiceRegister.register(ProxyServiceMBean.MBEAN_TYPE, this.getName(), this);
     }
 
     @Override
     public synchronized void stop() {
         logger.info("[STOP {} SOURCE]{} stopping...", this.getProtocolName(), this.getName());
+        // close channels
         if (!allChannels.isEmpty()) {
             try {
                 allChannels.close().awaitUninterruptibly();
             } catch (Exception e) {
-                logger.warn("Simple Source netty server stop ex, {}", e);
+                logger.warn("Close {} netty channels throw exception", this.getName(), e);
             } finally {
                 allChannels.clear();
             }
         }
-
-        super.stop();
-        if (monitorIndex != null) {
-            monitorIndex.shutDown();
-        }
-        if (monitorIndexExt != null) {
-            monitorIndexExt.shutDown();
-        }
-
+        // close channel future
         if (channelFuture != null) {
             try {
                 channelFuture.channel().closeFuture().sync();
             } catch (InterruptedException e) {
-                logger.warn("Simple Source netty server stop ex, {}", e);
+                logger.warn("Close {} channel future throw exception", this.getName(), e);
+            }
+        }
+        // stop super class
+        super.stop();
+        // stop workers
+        if (this.acceptorGroup != null) {
+            this.acceptorGroup.shutdownGracefully();
+        }
+        if (this.workerGroup != null) {
+            this.workerGroup.shutdownGracefully();
+        }
+        // stop file statistic index
+        if (CommonConfigHolder.getInstance().isEnableFileMetric()) {
+            if (monitorIndex != null) {
+                monitorIndex.stop();
+            }
+            if (monitorStats != null) {
+                monitorStats.stop();
             }
         }
         logger.info("[STOP {} SOURCE]{} stopped", this.getProtocolName(), this.getName());
     }
 
     @Override
-    public void configure(Context context) {
+    public void update() {
+        // check current all links
+        if (ConfigManager.getInstance().needChkIllegalIP()) {
+            int cnt = 0;
+            Channel channel;
+            String strRemoteIP;
+            long startTime = System.currentTimeMillis();
+            Iterator<Channel> iterator = allChannels.iterator();
+            while (iterator.hasNext()) {
+                channel = iterator.next();
+                strRemoteIP = AddressUtils.getChannelRemoteIP(channel);
+                if (strRemoteIP == null) {
+                    continue;
+                }
+                if (ConfigManager.getInstance().isIllegalIP(strRemoteIP)) {
+                    channel.disconnect();
+                    channel.close();
+                    allChannels.remove(channel);
+                    cnt++;
+                    logger.error(strRemoteIP + " is Illegal IP, so disconnect it !");
+                }
+            }
+            logger.info("Source {} channel check, disconnects {} Illegal channels, waist {} ms",
+                    getName(), cnt, (System.currentTimeMillis() - startTime));
+        }
+    }
 
-        this.context = context;
+    /**
+     * get metricItemSet
+     *
+     * @return the metricItemSet
+     */
+    public DataProxyMetricItemSet getMetricItemSet() {
+        return metricItemSet;
+    }
 
-        port = context.getInteger(ConfigConstants.CONFIG_PORT);
+    public Context getContext() {
+        return context;
+    }
 
-        host = context.getString(ConfigConstants.CONFIG_HOST, HOST_DEFAULT_VALUE);
+    public String getSrcHost() {
+        return srcHost;
+    }
 
-        Configurables.ensureRequiredNonNull(context, ConfigConstants.CONFIG_PORT);
+    public int getSrcPort() {
+        return srcPort;
+    }
 
-        Preconditions.checkArgument(ConfStringUtils.isValidIp(host), "ip config not valid");
-        Preconditions.checkArgument(ConfStringUtils.isValidPort(port), "port config not valid");
+    public String getStrPort() {
+        return strPort;
+    }
 
-        msgFactoryName =
-                context.getString(ConfigConstants.MSG_FACTORY_NAME,
-                        "org.apache.inlong.dataproxy.source.ServerMessageFactory");
-        msgFactoryName = msgFactoryName.trim();
-        Preconditions.checkArgument(StringUtils.isNotBlank(msgFactoryName),
-                "msgFactoryName is empty");
+    public String getDefAttr() {
+        return defAttr;
+    }
 
-        serviceDecoderName =
-                context.getString(ConfigConstants.SERVICE_PROCESSOR_NAME,
-                        "org.apache.inlong.dataproxy.source.DefaultServiceDecoder");
-        serviceDecoderName = serviceDecoderName.trim();
-        Preconditions.checkArgument(StringUtils.isNotBlank(serviceDecoderName),
-                "serviceProcessorName is empty");
+    public int getMaxMsgLength() {
+        return maxMsgLength;
+    }
 
-        messageHandlerName =
-                context.getString(ConfigConstants.MESSAGE_HANDLER_NAME,
-                        "org.apache.inlong.dataproxy.source.ServerMessageHandler");
-        messageHandlerName = messageHandlerName.trim();
-        Preconditions.checkArgument(StringUtils.isNotBlank(messageHandlerName),
-                "messageHandlerName is empty");
+    public boolean isCompressed() {
+        return isCompressed;
+    }
 
-        maxMsgLength = context.getInteger(ConfigConstants.MAX_MSG_LENGTH, MAX_MSG_DEFAULT_LENGTH);
-        Preconditions.checkArgument(
-                (maxMsgLength >= MSG_MIN_LENGTH && maxMsgLength <= ConfigConstants.MSG_MAX_LENGTH_BYTES),
-                "maxMsgLength must be >= 4 and <= " + ConfigConstants.MSG_MAX_LENGTH_BYTES);
-        isCompressed = context.getBoolean(ConfigConstants.MSG_COMPRESSED, true);
+    public boolean isFilterEmptyMsg() {
+        return filterEmptyMsg;
+    }
 
-        filterEmptyMsg = context.getBoolean(ConfigConstants.FILTER_EMPTY_MSG, false);
+    public boolean isCustomProcessor() {
+        return customProcessor;
+    }
 
-        topic = context.getString(ConfigConstants.TOPIC, "");
-        attr = context.getString(ConfigConstants.ATTR);
-        Configurables.ensureRequiredNonNull(context, ConfigConstants.ATTR);
+    public int getMaxConnections() {
+        return maxConnections;
+    }
 
-        topic = topic.trim();
-        attr = attr.trim();
-        Preconditions.checkArgument(!attr.isEmpty(), "attr is empty");
+    public ChannelGroup getAllChannels() {
+        return allChannels;
+    }
 
-        statIntervalSec = context.getInteger(ConfigConstants.STAT_INTERVAL_SEC, INTERVAL_SEC);
-        Preconditions.checkArgument((statIntervalSec >= STAT_INTERVAL_MUST_THAN), "statIntervalSec must be >= 0");
+    public long getMaxReadIdleTimeMs() {
+        return maxReadIdleTimeMs;
+    }
 
-        pkgTimeoutSec = context.getInteger(ConfigConstants.PACKAGE_TIMEOUT_SEC, PKG_TIMEOUT_DEFAULT_SEC);
+    public String getMessageHandlerName() {
+        return messageHandlerName;
+    }
 
-        try {
-            maxConnections = context.getInteger(CONNECTIONS, DEFAULT_MAX_CONNECTIONS);
-        } catch (NumberFormatException e) {
-            logger.warn("BaseSource\'s \"connections\" property must specify an integer value.",
-                    context.getString(CONNECTIONS));
-        }
+    public int getMaxWorkerThreads() {
+        return maxWorkerThreads;
+    }
 
-        try {
-            maxThreads = context.getInteger(ConfigConstants.MAX_THREADS, DEFAULT_MAX_THREADS);
-        } catch (NumberFormatException e) {
-            logger.warn("Simple TCP Source max-threads property must specify an integer value. {}",
-                    context.getString(ConfigConstants.MAX_THREADS));
+    public void fileMetricIncSumStats(String eventKey) {
+        if (CommonConfigHolder.getInstance().isEnableFileMetric()) {
+            monitorStats.incSumStats(eventKey);
         }
-        // get maxMonitorCnt's configure value
-        try {
-            maxMonitorCnt = context.getInteger(
-                    ConfigConstants.MAX_MONITOR_CNT, ConfigConstants.DEF_MONITOR_STAT_CNT);
-        } catch (NumberFormatException e) {
-            logger.warn("Property {} must specify an integer value: {}",
-                    ConfigConstants.MAX_MONITOR_CNT, context.getString(ConfigConstants.MAX_MONITOR_CNT));
-        }
-        Preconditions.checkArgument(maxMonitorCnt >= 0, "maxMonitorCnt must be >= 0");
+    }
 
-        receiveBufferSize = context.getInteger(ConfigConstants.RECEIVE_BUFFER_SIZE, RECEIVE_BUFFER_DEFAULT_SIZE);
-        if (receiveBufferSize > RECEIVE_BUFFER_MAX_SIZE) {
-            receiveBufferSize = RECEIVE_BUFFER_MAX_SIZE;
+    public void fileMetricIncDetailStats(String eventKey) {
+        if (CommonConfigHolder.getInstance().isEnableFileMetric()) {
+            monitorStats.incDetailStats(eventKey);
         }
-        Preconditions.checkArgument(receiveBufferSize > BUFFER_SIZE_MUST_THAN,
-                "receiveBufferSize must be > 0");
+    }
 
-        sendBufferSize = context.getInteger(ConfigConstants.SEND_BUFFER_SIZE, SEND_BUFFER_DEFAULT_SIZE);
-        if (sendBufferSize > SEND_BUFFER_MAX_SIZE) {
-            sendBufferSize = SEND_BUFFER_MAX_SIZE;
+    public void fileMetricAddSuccCnt(String key, int cnt, int packCnt, long packSize) {
+        if (CommonConfigHolder.getInstance().isEnableFileMetric()) {
+            monitorIndex.addSuccStats(key, cnt, packCnt, packSize);
         }
-        Preconditions.checkArgument(sendBufferSize > BUFFER_SIZE_MUST_THAN,
-                "sendBufferSize must be > 0");
-
-        enableBusyWait = context.getBoolean(ConfigConstants.ENABLE_BUSY_WAIT, false);
+    }
 
-        this.customProcessor = context.getBoolean(ConfigConstants.CUSTOM_CHANNEL_PROCESSOR, false);
+    public void fileMetricAddFailCnt(String key, int failCnt) {
+        if (CommonConfigHolder.getInstance().isEnableFileMetric()) {
+            monitorIndex.addFailStats(key, failCnt);
+        }
+    }
+    /**
+     * addMetric
+     *
+     * @param result
+     * @param size
+     * @param event
+     */
+    public void addMetric(boolean result, long size, Event event) {
+        Map<String, String> dimensions = new HashMap<>();
+        dimensions.put(DataProxyMetricItem.KEY_CLUSTER_ID, CommonConfigHolder.getInstance().getClusterName());
+        dimensions.put(DataProxyMetricItem.KEY_SOURCE_ID, getName());
+        dimensions.put(DataProxyMetricItem.KEY_SOURCE_DATA_ID, getStrPort());
+        DataProxyMetricItem.fillInlongId(event, dimensions);
+        DataProxyMetricItem.fillAuditFormatTime(event, dimensions);
+        DataProxyMetricItem metricItem = metricItemSet.findMetricItem(dimensions);
+        if (result) {
+            metricItem.readSuccessCount.incrementAndGet();
+            metricItem.readSuccessSize.addAndGet(size);
+            AuditUtils.add(AuditUtils.AUDIT_ID_DATAPROXY_READ_SUCCESS, event);
+        } else {
+            metricItem.readFailCount.incrementAndGet();
+            metricItem.readFailSize.addAndGet(size);
+        }
     }
 
     /**
      * channel factory
+     *
      * @return
      */
     public ChannelInitializer getChannelInitializerFactory() {
-        logger.info(new StringBuffer("load msgFactory=").append(msgFactoryName)
-                .append(" and serviceDecoderName=").append(serviceDecoderName).toString());
         ChannelInitializer fac = null;
+        logger.info(this.getName() + " load msgFactory=" + msgFactoryName);
         try {
-            ServiceDecoder serviceDecoder = (ServiceDecoder) Class.forName(serviceDecoderName).newInstance();
             Class<? extends ChannelInitializer> clazz =
                     (Class<? extends ChannelInitializer>) Class.forName(msgFactoryName);
-            Constructor ctor = clazz.getConstructor(BaseSource.class, ChannelGroup.class,
-                    String.class, ServiceDecoder.class, String.class, Integer.class,
-                    String.class, String.class, Boolean.class,
-                    Integer.class, Boolean.class, MonitorIndex.class,
-                    MonitorIndexExt.class, String.class);
+            Constructor ctor = clazz.getConstructor(BaseSource.class);
             logger.info("Using channel processor:{}", getChannelProcessor().getClass().getName());
-            fac = (ChannelInitializer) ctor.newInstance(this, allChannels,
-                    this.getProtocolName(), serviceDecoder, messageHandlerName, maxMsgLength,
-                    topic, attr, filterEmptyMsg,
-                    maxConnections, isCompressed, monitorIndex,
-                    monitorIndexExt, this.getProtocolName());
+            fac = (ChannelInitializer) ctor.newInstance(this);
         } catch (Exception e) {
-            logger.error(
-                    "Simple {} Source start error, fail to construct ChannelPipelineFactory with name "
-                            + "{}, ex {}",
-                    this.getProtocolName(), msgFactoryName, e);
+            logger.error("{} start error, fail to construct ChannelPipelineFactory with name {}",
+                    this.getName(), msgFactoryName, e);
             stop();
             throw new FlumeException(e.getMessage());
         }
         return fac;
     }
 
+    public abstract String getProtocolName();
+
+    public abstract void startSource();
+
+    /**
+     * stopService
+     */
+    @Override
+    public void stopService() {
+        this.isRejectService = true;
+    }
+
     /**
-     * get metricItemSet
-     * @return the metricItemSet
+     * recoverService
      */
-    public DataProxyMetricItemSet getMetricItemSet() {
-        return metricItemSet;
+    @Override
+    public void recoverService() {
+        this.isRejectService = false;
     }
 
-    public Context getContext() {
-        return context;
+    /**
+     * isRejectService
+     *
+     * @return
+     */
+    public boolean isRejectService() {
+        return isRejectService;
     }
 
-    public abstract String getProtocolName();
+    /**
+     * getHostIp
+     *
+     * @param context
+     * @return
+     */
+    private String getHostIp(Context context) {
+        String result = null;
+        // first get host ip from dataProxy.conf
+        String tmpVal = context.getString(SourceConstants.SRCCXT_CONFIG_HOST);
+        if (StringUtils.isNotBlank(tmpVal)) {
+            tmpVal = tmpVal.trim();
+            Preconditions.checkArgument(ConfStringUtils.isValidIp(tmpVal),
+                    SourceConstants.SRCCXT_CONFIG_HOST + "(" + tmpVal + ") config in conf not valid");
+            result = tmpVal;
+        }
+        // second get host ip from system env
+        Map<String, String> envMap = System.getenv();
+        if (envMap.containsKey(SourceConstants.SYSENV_HOST_IP)) {
+            tmpVal = envMap.get(SourceConstants.SYSENV_HOST_IP);
+            Preconditions.checkArgument(ConfStringUtils.isValidIp(tmpVal),
+                    SourceConstants.SYSENV_HOST_IP + "(" + tmpVal + ") config in system env not valid");
+            result = tmpVal.trim();
+        }
+        if (StringUtils.isBlank(result)) {
+            result = SourceConstants.VAL_DEF_HOST_VALUE;
+        }
+        return result;
+    }
 
-    public abstract void startSource();
+    /**
+     * getHostPort
+     *
+     * @param context
+     * @return
+     */
+    private int getHostPort(Context context) {
+        Integer result = null;
+        // first get host port from dataProxy.conf
+        String tmpVal = context.getString(SourceConstants.SRCCXT_CONFIG_PORT);
+        if (StringUtils.isNotBlank(tmpVal)) {
+            tmpVal = tmpVal.trim();
+            try {
+                result = Integer.parseInt(tmpVal);
+            } catch (Throwable e) {
+                throw new IllegalArgumentException(
+                        SourceConstants.SYSENV_HOST_PORT + "(" + tmpVal + ") config in conf not integer");
+            }
+        }
+        if (result != null) {
+            Preconditions.checkArgument(ConfStringUtils.isValidPort(result),
+                    SourceConstants.SRCCXT_CONFIG_PORT + "(" + result + ") config in conf not valid");
+        }
+        // second get host port from system env
+        Map<String, String> envMap = System.getenv();
+        if (envMap.containsKey(SourceConstants.SYSENV_HOST_PORT)) {
+            tmpVal = envMap.get(SourceConstants.SYSENV_HOST_PORT);
+            if (StringUtils.isNotBlank(tmpVal)) {
+                tmpVal = tmpVal.trim();
+                try {
+                    result = Integer.parseInt(tmpVal);
+                } catch (Throwable e) {
+                    throw new IllegalArgumentException(
+                            SourceConstants.SYSENV_HOST_PORT + "(" + tmpVal + ") config in system env not integer");
+                }
+                Preconditions.checkArgument(ConfStringUtils.isValidPort(result),
+                        SourceConstants.SYSENV_HOST_PORT + "(" + tmpVal + ") config in system env not valid");
+            }
+        }
+        if (result == null) {
+            throw new IllegalArgumentException("Required parameter " +
+                    SourceConstants.SRCCXT_CONFIG_PORT + " must exist and may not be null");
+        }
+        return result;
+    }
 
 }
diff --git a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source/DefaultServiceDecoder.java b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source/DefaultServiceDecoder.java
deleted file mode 100644
index c17345d6d3..0000000000
--- a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source/DefaultServiceDecoder.java
+++ /dev/null
@@ -1,513 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.inlong.dataproxy.source;
-
-import org.apache.inlong.common.msg.AttributeConstants;
-import org.apache.inlong.common.msg.MsgType;
-import org.apache.inlong.dataproxy.base.ProxyMessage;
-import org.apache.inlong.dataproxy.consts.AttrConstants;
-import org.apache.inlong.dataproxy.consts.ConfigConstants;
-import org.apache.inlong.dataproxy.exception.ErrorCode;
-import org.apache.inlong.dataproxy.exception.MessageIDException;
-
-import com.google.common.base.Splitter;
-import io.netty.buffer.ByteBuf;
-import io.netty.channel.Channel;
-import org.apache.commons.lang3.StringUtils;
-import org.apache.commons.lang3.math.NumberUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.xerial.snappy.Snappy;
-
-import java.io.IOException;
-import java.net.SocketAddress;
-import java.nio.ByteBuffer;
-import java.nio.charset.StandardCharsets;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-public class DefaultServiceDecoder implements ServiceDecoder {
-
-    private static final int BIN_MSG_TOTALLEN_OFFSET = 0;
-    private static final int BIN_MSG_TOTALLEN_SIZE = 4;
-    private static final int BIN_MSG_MSGTYPE_OFFSET = 4;
-    private static final int BIN_MSG_EXTEND_OFFSET = 9;
-    private static final int BIN_MSG_EXTEND_SIZE = 2;
-    private static final int BIN_MSG_SET_SNAPPY = (1 << 5);
-    private static final int BIN_MSG_BODYLEN_SIZE = 4;
-    private static final int BIN_MSG_BODYLEN_OFFSET = 21;
-    private static final int BIN_MSG_BODY_OFFSET = BIN_MSG_BODYLEN_SIZE + BIN_MSG_BODYLEN_OFFSET;
-    private static final int BIN_MSG_ATTRLEN_SIZE = 2;
-
-    private static final int BIN_MSG_FORMAT_SIZE = 29;
-    private static final int BIN_MSG_MAGIC_SIZE = 2;
-    private static final int BIN_MSG_MAGIC = 0xEE01;
-
-    private static final int BIN_HB_TOTALLEN_SIZE = 4;
-    private static final int BIN_HB_BODYLEN_OFFSET = 10;
-    private static final int BIN_HB_BODYLEN_SIZE = 4;
-    private static final int BIN_HB_BODY_OFFSET = BIN_HB_BODYLEN_SIZE + BIN_HB_BODYLEN_OFFSET;
-    private static final int BIN_HB_ATTRLEN_SIZE = 2;
-    private static final int BIN_HB_FORMAT_SIZE = 17;
-
-    private static final Logger LOG = LoggerFactory
-            .getLogger(DefaultServiceDecoder.class);
-
-    private static final Splitter.MapSplitter mapSplitter = Splitter
-            .on(AttributeConstants.SEPARATOR).trimResults()
-            .withKeyValueSeparator(AttributeConstants.KEY_VALUE_SEPARATOR);
-
-    /**
-     * extract bin heart beat data, message type is 8
-     */
-    private Map<String, Object> extractNewBinHB(Map<String, Object> resultMap,
-            ByteBuf cb, Channel channel, int totalDataLen) throws Exception {
-        int msgHeadPos = cb.readerIndex() - 5;
-
-        // check validation
-        int bodyLen = cb.getInt(msgHeadPos + BIN_HB_BODYLEN_OFFSET);
-        int attrLen = cb.getShort(msgHeadPos + BIN_HB_BODY_OFFSET + bodyLen);
-        int msgMagic = cb.getUnsignedShort(msgHeadPos + BIN_HB_BODY_OFFSET
-                + bodyLen + BIN_HB_ATTRLEN_SIZE + attrLen);
-
-        if ((totalDataLen + BIN_HB_TOTALLEN_SIZE < (bodyLen
-                + attrLen + BIN_HB_FORMAT_SIZE)) || (msgMagic != BIN_MSG_MAGIC)) {
-
-            LOG.error("err msg, bodyLen + attrLen > totalDataLen, "
-                    + "and bodyLen={},attrLen={},totalDataLen={},magic={};Connection "
-                    + "info:{}",
-                    bodyLen, attrLen, totalDataLen, Integer.toHexString(msgMagic), channel.toString());
-
-            return resultMap;
-        }
-        cb.skipBytes(9 + bodyLen + BIN_HB_ATTRLEN_SIZE);
-        // extract common attr
-        String strAttr = null;
-        if (attrLen != 0) {
-            byte[] attrData = new byte[attrLen];
-            cb.readBytes(attrData, 0, attrLen);
-            strAttr = new String(attrData, StandardCharsets.UTF_8);
-            resultMap.put(ConfigConstants.DECODER_ATTRS, strAttr);
-        }
-        byte version = cb.getByte(msgHeadPos + 9);
-        resultMap.put(ConfigConstants.VERSION_TYPE, version);
-
-        return resultMap;
-    }
-
-    private void handleDateTime(Map<String, String> commonAttrMap, long uniq,
-            long dataTime, int msgCount, String strRemoteIP,
-            long msgRcvTime) {
-        commonAttrMap.put(AttributeConstants.UNIQ_ID, String.valueOf(uniq));
-        String time = String.valueOf(dataTime);
-        commonAttrMap.put(AttributeConstants.SEQUENCE_ID,
-                new StringBuilder(256).append(strRemoteIP)
-                        .append("#").append(time).append("#").append(uniq).toString());
-        // dt from sdk
-        commonAttrMap.put(AttributeConstants.DATA_TIME, String.valueOf(dataTime));
-        commonAttrMap
-                .put(AttributeConstants.RCV_TIME, String.valueOf(msgRcvTime));
-        commonAttrMap.put(AttributeConstants.MESSAGE_COUNT, String.valueOf(msgCount));
-    }
-
-    private boolean handleExtMap(Map<String, String> commonAttrMap, ByteBuf cb,
-            Map<String, Object> resultMap, int extendField, int msgHeadPos) {
-        boolean index = false;
-        if ((extendField & 0x8) == 0x8) {
-            index = true;
-            // if message is file metric
-            int dataLen = cb.getInt(msgHeadPos + BIN_MSG_BODYLEN_OFFSET + 4);
-            byte[] data = new byte[dataLen];
-            cb.getBytes(msgHeadPos + BIN_MSG_BODY_OFFSET + 4, data, 0,
-                    dataLen);
-            resultMap.put(ConfigConstants.FILE_BODY, data);
-            commonAttrMap.put(ConfigConstants.FILE_CHECK_DATA, "true");
-        } else if ((extendField & 0x10) == 0x10) {
-            index = true;
-            // if message is verification metric message
-            int dataLen = cb.getInt(msgHeadPos + BIN_MSG_BODYLEN_OFFSET + 4);
-            byte[] data = new byte[dataLen];
-            // remove body len
-            cb.getBytes(msgHeadPos + BIN_MSG_BODY_OFFSET + 4, data, 0,
-                    dataLen);
-            resultMap.put(ConfigConstants.FILE_BODY, data);
-            commonAttrMap.put(ConfigConstants.MINUTE_CHECK_DATA, "true");
-        }
-        return index;
-    }
-
-    private ByteBuffer handleExtraAppendAttrInfo(Map<String, String> commonAttrMap,
-            Channel channel, ByteBuf cb, int extendField,
-            int msgHeadPos, int totalDataLen, int attrLen,
-            String strAttr, int bodyLen, long msgRcvTime) {
-        // get and check report time from report node
-        boolean needRebuild = false;
-        String rtMs = "";
-        if (StringUtils.isBlank(commonAttrMap.get(AttributeConstants.MSG_RPT_TIME))) {
-            needRebuild = true;
-            rtMs = AttributeConstants.MSG_RPT_TIME
-                    + AttributeConstants.KEY_VALUE_SEPARATOR
-                    + System.currentTimeMillis();
-        }
-        // get trace requirement
-        String traceInfo = "";
-        boolean enableTrace = (((extendField & 0x2) >> 1) == 0x1);
-        if (enableTrace) {
-            needRebuild = true;
-            // get local address
-            String strNode2Ip = null;
-            SocketAddress loacalSockAddr = channel.localAddress();
-            if (null != loacalSockAddr) {
-                strNode2Ip = loacalSockAddr.toString();
-                try {
-                    strNode2Ip = strNode2Ip.substring(1, strNode2Ip.indexOf(':'));
-                } catch (Exception ee) {
-                    LOG.warn("fail to get the local IP, and strIP={},localSocketAddress={}",
-                            strNode2Ip, loacalSockAddr);
-                }
-            }
-            traceInfo = AttributeConstants.DATAPROXY_NODE_IP
-                    + AttributeConstants.KEY_VALUE_SEPARATOR + strNode2Ip
-                    + AttributeConstants.SEPARATOR
-                    + AttributeConstants.DATAPROXY_RCVTIME
-                    + AttributeConstants.KEY_VALUE_SEPARATOR + msgRcvTime;
-        }
-        // rebuild msg attribute
-        ByteBuffer dataBuf;
-        if (needRebuild) {
-            int newTotalLen = totalDataLen;
-            // add rtms attribute
-            if (StringUtils.isNotEmpty(rtMs)) {
-                if (StringUtils.isEmpty(strAttr)) {
-                    newTotalLen += rtMs.length();
-                    strAttr = rtMs;
-                } else {
-                    newTotalLen += AttributeConstants.SEPARATOR.length() + rtMs.length();
-                    strAttr = strAttr + AttributeConstants.SEPARATOR + rtMs;
-                }
-            }
-            // add trace attribute
-            if (StringUtils.isNotEmpty(traceInfo)) {
-                if (StringUtils.isEmpty(strAttr)) {
-                    newTotalLen += traceInfo.length();
-                    strAttr = traceInfo;
-                } else {
-                    newTotalLen += AttributeConstants.SEPARATOR.length() + traceInfo.length();
-                    strAttr = strAttr + AttributeConstants.SEPARATOR + traceInfo;
-                }
-            }
-            // build message buffer
-            dataBuf = ByteBuffer.allocate(newTotalLen + BIN_MSG_TOTALLEN_SIZE);
-            cb.getBytes(msgHeadPos, dataBuf.array(), 0,
-                    bodyLen + (BIN_MSG_FORMAT_SIZE - BIN_MSG_ATTRLEN_SIZE
-                            - BIN_MSG_MAGIC_SIZE));
-            dataBuf.putShort(
-                    bodyLen + (BIN_MSG_FORMAT_SIZE - BIN_MSG_ATTRLEN_SIZE - BIN_MSG_MAGIC_SIZE),
-                    (short) strAttr.length());
-            // copy all attributes
-            System.arraycopy(strAttr.getBytes(StandardCharsets.UTF_8), 0, dataBuf.array(),
-                    bodyLen + (BIN_MSG_FORMAT_SIZE - BIN_MSG_MAGIC_SIZE),
-                    strAttr.length());
-            dataBuf.putInt(0, newTotalLen);
-            dataBuf.putShort(newTotalLen + BIN_MSG_TOTALLEN_SIZE - BIN_MSG_MAGIC_SIZE,
-                    (short) 0xee01);
-        } else {
-            dataBuf = ByteBuffer.allocate(totalDataLen + BIN_MSG_TOTALLEN_SIZE);
-            cb.getBytes(msgHeadPos, dataBuf.array(), 0, totalDataLen + BIN_MSG_TOTALLEN_SIZE);
-        }
-        return dataBuf;
-    }
-
-    /**
-     * extract bin data, message type is 7
-     */
-    private Map<String, Object> extractNewBinData(Map<String, Object> resultMap,
-            ByteBuf cb, Channel channel,
-            int totalDataLen, MsgType msgType,
-            String strRemoteIP,
-            long msgRcvTime) throws Exception {
-        int msgHeadPos = cb.readerIndex() - 5;
-        // get body length
-        int bodyLen = cb.getInt(msgHeadPos + BIN_MSG_BODYLEN_OFFSET);
-        if (bodyLen == 0) {
-            throw new Exception(
-                    "Error msg,  bodyLen is empty; connection info:" + strRemoteIP);
-        }
-        // get attribute length
-        int attrLen = cb.getShort(msgHeadPos + BIN_MSG_BODY_OFFSET + bodyLen);
-        // get msg magic
-        int msgMagic = cb.getUnsignedShort(msgHeadPos + BIN_MSG_BODY_OFFSET
-                + bodyLen + BIN_MSG_ATTRLEN_SIZE + attrLen);
-        if ((msgMagic != BIN_MSG_MAGIC)
-                || (totalDataLen + BIN_MSG_TOTALLEN_SIZE < (bodyLen + attrLen + BIN_MSG_FORMAT_SIZE))) {
-            throw new Exception(
-                    "Error msg, bodyLen + attrLen > totalDataLen,or msgMagic is valid! and bodyLen="
-                            + bodyLen + ",attrLen=" + attrLen + ",totalDataLen=" + totalDataLen
-                            + ";magic=" + Integer.toHexString(msgMagic)
-                            + "; connection info:" + strRemoteIP);
-        }
-        // read data from ByteBuf
-        int groupIdNum = cb.readUnsignedShort();
-        int streamIdNum = cb.readUnsignedShort();
-        final int extendField = cb.readUnsignedShort();
-        long dataTime = cb.readUnsignedInt();
-        dataTime = dataTime * 1000;
-        int msgCount = cb.readUnsignedShort();
-        msgCount = (msgCount != 0) ? msgCount : 1;
-        long uniq = cb.readUnsignedInt();
-        cb.skipBytes(BIN_MSG_BODYLEN_SIZE + bodyLen + BIN_MSG_ATTRLEN_SIZE);
-        // read body data
-        byte[] bodyData = new byte[bodyLen];
-        cb.getBytes(msgHeadPos + BIN_MSG_BODY_OFFSET, bodyData, 0, bodyLen);
-        // read attr and write to map.
-        String strAttr = null;
-        Map<String, String> commonAttrMap = new HashMap<>();
-        if (attrLen != 0) {
-            byte[] attrData = new byte[attrLen];
-            cb.readBytes(attrData, 0, attrLen);
-            strAttr = new String(attrData, StandardCharsets.UTF_8);
-            resultMap.put(ConfigConstants.DECODER_ATTRS, strAttr);
-            try {
-                commonAttrMap.putAll(mapSplitter.split(strAttr));
-            } catch (Exception e) {
-                cb.clear();
-                throw new MessageIDException(uniq,
-                        ErrorCode.ATTR_ERROR,
-                        new Throwable("[Parse Error]new six segment protocol ,attr is "
-                                + strAttr + " , channel info:" + strRemoteIP));
-            }
-        }
-        // build attributes
-        resultMap.put(ConfigConstants.COMMON_ATTR_MAP, commonAttrMap);
-        resultMap.put(ConfigConstants.EXTRA_ATTR, ((extendField & 0x1) == 0x1) ? "true" : "false");
-        resultMap.put(ConfigConstants.DECODER_BODY, bodyData);
-        try {
-            // handle common attribute information
-            handleDateTime(commonAttrMap, uniq, dataTime, msgCount, strRemoteIP, msgRcvTime);
-            final boolean isIndexMsg =
-                    handleExtMap(commonAttrMap, cb, resultMap, extendField, msgHeadPos);
-            ByteBuffer dataBuf = handleExtraAppendAttrInfo(commonAttrMap, channel, cb,
-                    extendField, msgHeadPos, totalDataLen, attrLen, strAttr, bodyLen, msgRcvTime);
-            // Check if groupId and streamId are number-to-name
-            String groupId = commonAttrMap.get(AttributeConstants.GROUP_ID);
-            String streamId = commonAttrMap.get(AttributeConstants.STREAM_ID);
-            if ((groupId != null) && (streamId != null)) {
-                commonAttrMap.put(AttrConstants.NUM2NAME, "FALSE");
-                dataBuf.putShort(BIN_MSG_EXTEND_OFFSET, (short) (extendField | 0x4));
-            } else {
-                boolean hasNumGroupId = (((extendField & 0x4) >> 2) == 0x0);
-                if (hasNumGroupId && (0 != groupIdNum) && (0 != streamIdNum)) {
-                    commonAttrMap.put(AttrConstants.NUM2NAME, "TRUE");
-                    commonAttrMap.put(AttrConstants.GROUPID_NUM, String.valueOf(groupIdNum));
-                    commonAttrMap.put(AttrConstants.STREAMID_NUM, String.valueOf(streamIdNum));
-                }
-            }
-            // build ProxyMessage
-            if (MsgType.MSG_BIN_MULTI_BODY.equals(msgType)) {
-                List<ProxyMessage> msgList = new ArrayList<>(1);
-                if (isIndexMsg) {
-                    msgList.add(new ProxyMessage(groupId, streamId, commonAttrMap,
-                            (byte[]) resultMap.get(ConfigConstants.FILE_BODY)));
-                } else {
-                    msgList.add(new ProxyMessage(groupId,
-                            streamId, commonAttrMap, dataBuf.array()));
-                }
-                resultMap.put(ConfigConstants.MSG_LIST, msgList);
-            }
-        } catch (Exception ex) {
-            LOG.error("extractNewBinData has error: ", ex);
-            cb.clear();
-            throw new MessageIDException(uniq, ErrorCode.OTHER_ERROR, ex.getCause());
-        }
-        return resultMap;
-    }
-
-    /**
-     * extract bin data, message type less than 7
-     */
-    private Map<String, Object> extractDefaultData(Map<String, Object> resultMap,
-            ByteBuf cb, int totalDataLen,
-            MsgType msgType, String strRemoteIP,
-            long msgRcvTime) throws Exception {
-        int bodyLen = cb.readInt();
-        if (bodyLen == 0) {
-            throw new Exception("Error msg: bodyLen is empty, connection info:" + strRemoteIP);
-        }
-        // if body len is bigger than totalDataLen - 5(bodyLen bytes + message type bytes),
-        // that means an invalid message, reject it.
-        if (bodyLen > totalDataLen - 5) {
-            throw new Exception("Error msg, firstLen > totalDataLen, and bodyLen="
-                    + bodyLen + ",totalDataLen=" + totalDataLen
-                    + ", connection info:" + strRemoteIP);
-        }
-        // extract body bytes
-        byte[] bodyData = new byte[bodyLen];
-        cb.readBytes(bodyData, 0, bodyLen);
-        resultMap.put(ConfigConstants.DECODER_BODY, bodyData);
-        // extract attribute
-        int attrLen = cb.readInt();
-        // 9 means bodyLen bytes(4) + message type bytes(1) + attrLen bytes(4)
-        if (totalDataLen != 9 + attrLen + bodyLen) {
-            throw new Exception(
-                    "Error msg, totalDataLen != 9 + bodyLen + attrLen,and bodyLen=" + bodyLen
-                            + ",totalDataLen=" + totalDataLen + ",attrDataLen=" + attrLen
-                            + ", connection info:" + strRemoteIP);
-        }
-        // extract attr bytes
-        byte[] attrData = new byte[attrLen];
-        cb.readBytes(attrData, 0, attrLen);
-        // convert attr bytes to map
-        Map<String, String> commonAttrMap;
-        String strAttr = new String(attrData, StandardCharsets.UTF_8);
-        try {
-            commonAttrMap = new HashMap<>(mapSplitter.split(strAttr));
-        } catch (Exception e) {
-            throw new Exception("Parse commonAttrMap error.commonAttrString is: "
-                    + strAttr + " , connection info:" + strRemoteIP);
-        }
-        resultMap.put(ConfigConstants.DECODER_ATTRS, strAttr);
-        resultMap.put(ConfigConstants.COMMON_ATTR_MAP, commonAttrMap);
-        // decompress body data if compress type exists.
-        String compressType = commonAttrMap.get(AttributeConstants.COMPRESS_TYPE);
-        if (StringUtils.isNotBlank(compressType)) {
-            resultMap.put(ConfigConstants.COMPRESS_TYPE, compressType);
-            byte[] unCompressedData = processUnCompress(bodyData, compressType);
-            if (unCompressedData == null || unCompressedData.length == 0) {
-                throw new Exception("Uncompressed data error! compress type:"
-                        + compressType + ";attr:" + strAttr
-                        + " , connection info:" + strRemoteIP);
-            }
-            bodyData = unCompressedData;
-        }
-        // fill up attr map with some keys.
-        String groupId = commonAttrMap.get(AttributeConstants.GROUP_ID);
-        String streamId = commonAttrMap.get(AttributeConstants.STREAM_ID);
-        String strDataTime = commonAttrMap.get(AttributeConstants.DATA_TIME);
-        long longDataTime = NumberUtils.toLong(strDataTime, msgRcvTime);
-        commonAttrMap.put(AttributeConstants.DATA_TIME, String.valueOf(longDataTime));
-        // add message report time field
-        if (StringUtils.isBlank(commonAttrMap.get(AttributeConstants.MSG_RPT_TIME))) {
-            commonAttrMap.put(AttributeConstants.MSG_RPT_TIME,
-                    String.valueOf(msgRcvTime));
-        }
-        commonAttrMap.put(AttributeConstants.RCV_TIME, String.valueOf(msgRcvTime));
-        // check message count attr
-        String strMsgCnt = commonAttrMap.get(AttributeConstants.MESSAGE_COUNT);
-        int intMsgCnt = NumberUtils.toInt(strMsgCnt, 1);
-        commonAttrMap.put(AttributeConstants.MESSAGE_COUNT, String.valueOf(intMsgCnt));
-        // extract data from bodyData and if message type is 5, convert data into list.
-        int calCnt = 0;
-        List<ProxyMessage> msgList = null;
-        ByteBuffer bodyBuffer = ByteBuffer.wrap(bodyData);
-        if (MsgType.MSG_MULTI_BODY.equals(msgType)) {
-            msgList = new ArrayList<>(intMsgCnt);
-            while (bodyBuffer.remaining() > 0) {
-                int singleMsgLen = bodyBuffer.getInt();
-                if (singleMsgLen <= 0 || singleMsgLen > bodyBuffer.remaining()) {
-                    throw new Exception(
-                            "[Malformed Data]Invalid data len! channel is " + strRemoteIP);
-                }
-                byte[] record = new byte[singleMsgLen];
-                bodyBuffer.get(record);
-                ProxyMessage message = new ProxyMessage(groupId, streamId, commonAttrMap, record);
-                msgList.add(message);
-                calCnt++;
-            }
-        } else {
-            msgList = new ArrayList<>(1);
-            msgList.add(new ProxyMessage(groupId, streamId, commonAttrMap, bodyData));
-            calCnt++;
-        }
-        if (calCnt != intMsgCnt) {
-            commonAttrMap.put(AttributeConstants.MESSAGE_COUNT, String.valueOf(calCnt));
-        }
-        resultMap.put(ConfigConstants.MSG_LIST, msgList);
-        return resultMap;
-    }
-
-    private byte[] processUnCompress(byte[] input, String compressType) {
-        byte[] result;
-        try {
-            int uncompressedLen = Snappy.uncompressedLength(input, 0, input.length);
-            result = new byte[uncompressedLen];
-            Snappy.uncompress(input, 0, input.length, result, 0);
-        } catch (IOException e) {
-            LOG.error("Uncompressed data error: ", e);
-            return null;
-        }
-        return result;
-    }
-
-    /**
-     * BEFORE                                                                   AFTER
-     * +--------+--------+--------+----------------+--------+----------------+------------------------+
-     * | Length | Msgtype| Length | Actual Content1| Length | Actual Content2|----- >| Actual
-     * Content1| Time    postfix              | | 0x000C | 0x02   | 0x000C | "HELLO, WORLD" | 0x000C
-     * | "view video? " |       | "HELLO, WORLD" | ",recvtimestamp:12345674321" |
-     * +--------+--------+--------+----------------+--------+----------------+------------------------+
-     */
-    @Override
-    public Map<String, Object> extractData(ByteBuf cb, String strRemoteIP,
-            long msgRcvTime, Channel channel) throws Exception {
-        Map<String, Object> resultMap = new HashMap<>();
-        if (null == cb) {
-            LOG.error("cb == null");
-            return resultMap;
-        }
-        int totalLen = cb.readableBytes();
-        if (ConfigConstants.MSG_MAX_LENGTH_BYTES < totalLen) {
-            throw new Exception("Error msg, ConfigConstants.MSG_MAX_LENGTH_BYTES "
-                    + "< totalLen, and  totalLen=" + totalLen);
-        }
-        // save index, reset it if buffer is not satisfied.
-        cb.markReaderIndex();
-        int totalDataLen = cb.readInt();
-        if (totalDataLen + HEAD_LENGTH <= totalLen) {
-            int msgTypeInt = cb.readByte();
-            int compressType = ((msgTypeInt & 0xE0) >> 5);
-            MsgType msgType = MsgType.valueOf(msgTypeInt);
-            resultMap.put(ConfigConstants.MSG_TYPE, msgType);
-
-            // if it's heart beat or unknown message, just return without handling it.
-            if (MsgType.MSG_HEARTBEAT.equals(msgType)
-                    || MsgType.MSG_UNKNOWN.equals(msgType)) {
-                return resultMap;
-            }
-            // if it's bin heart beat.
-            if (MsgType.MSG_BIN_HEARTBEAT.equals(msgType)) {
-                return extractNewBinHB(resultMap, cb, channel, totalDataLen);
-            }
-            // process data message
-            if (msgType.getValue() >= MsgType.MSG_BIN_MULTI_BODY.getValue()) {
-                resultMap.put(ConfigConstants.COMPRESS_TYPE, (compressType != 0) ? "snappy" : "");
-                return extractNewBinData(resultMap, cb,
-                        channel, totalDataLen, msgType,
-                        strRemoteIP, msgRcvTime);
-            } else {
-                return extractDefaultData(resultMap, cb,
-                        totalDataLen, msgType, strRemoteIP, msgRcvTime);
-            }
-        } else {
-            // reset index.
-            cb.resetReaderIndex();
-            return null;
-        }
-    }
-}
diff --git a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source/ServerMessageFactory.java b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source/ServerMessageFactory.java
index 9cb371b65b..7b35c27ed4 100644
--- a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source/ServerMessageFactory.java
+++ b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source/ServerMessageFactory.java
@@ -17,139 +17,68 @@
 
 package org.apache.inlong.dataproxy.source;
 
-import org.apache.inlong.common.monitor.MonitorIndex;
-import org.apache.inlong.common.monitor.MonitorIndexExt;
-import org.apache.inlong.dataproxy.consts.ConfigConstants;
-
 import io.netty.channel.ChannelInboundHandlerAdapter;
 import io.netty.channel.ChannelInitializer;
-import io.netty.channel.group.ChannelGroup;
 import io.netty.channel.socket.SocketChannel;
 import io.netty.handler.codec.LengthFieldBasedFrameDecoder;
+import io.netty.handler.codec.http.HttpObjectAggregator;
+import io.netty.handler.codec.http.HttpServerCodec;
 import io.netty.handler.timeout.ReadTimeoutHandler;
-import org.apache.flume.channel.ChannelProcessor;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.lang.reflect.Constructor;
 import java.util.concurrent.TimeUnit;
 
-public class ServerMessageFactory
-        extends
-            ChannelInitializer<SocketChannel> {
+public class ServerMessageFactory extends ChannelInitializer<SocketChannel> {
 
+    public static final int INLONG_LENGTH_FIELD_OFFSET = 0;
+    public static final int INLONG_LENGTH_FIELD_LENGTH = 4;
+    public static final int INLONG_LENGTH_ADJUSTMENT = 0;
+    public static final int INLONG_INITIAL_BYTES_TO_STRIP = 0;
+    public static final boolean DEFAULT_FAIL_FAST = true;
     private static final Logger LOG = LoggerFactory.getLogger(ServerMessageFactory.class);
-
-    private static final int DEFAULT_READ_IDLE_TIME = 70 * 60 * 1000;
-
-    private static long MAX_CHANNEL_MEMORY_SIZE = 1024 * 1024;
-
-    private static long MAX_TOTAL_MEMORY_SIZE = 1024 * 1024;
-
-    private static int MSG_LENGTH_LEN = 4;
-
-    private BaseSource source;
-
-    private ChannelProcessor processor;
-
-    private ChannelGroup allChannels;
-
-    private String protocolType;
-
-    private ServiceDecoder serviceDecoder;
-
-    private String messageHandlerName;
-
-    private int maxConnections = Integer.MAX_VALUE;
-
-    private int maxMsgLength;
-
-    private boolean isCompressed;
-
-    private String name;
-
-    private String topic;
-
-    private String attr;
-
-    private boolean filterEmptyMsg;
-
-    private MonitorIndex monitorIndex;
-
-    private MonitorIndexExt monitorIndexExt;
+    private final BaseSource source;
 
     /**
      * get server factory
      *
      * @param source
-     * @param allChannels
-     * @param protocol
-     * @param serviceDecoder
-     * @param messageHandlerName
-     * @param topic
-     * @param attr
-     * @param filterEmptyMsg
-     * @param maxCons
-     * @param isCompressed
-     * @param monitorIndex
-     * @param monitorIndexExt
-     * @param name
      */
-    public ServerMessageFactory(BaseSource source, ChannelGroup allChannels, String protocol,
-            ServiceDecoder serviceDecoder, String messageHandlerName, Integer maxMsgLength,
-            String topic, String attr, Boolean filterEmptyMsg, Integer maxCons,
-            Boolean isCompressed, MonitorIndex monitorIndex, MonitorIndexExt monitorIndexExt,
-            String name) {
+    public ServerMessageFactory(BaseSource source) {
         this.source = source;
-        this.processor = source.getChannelProcessor();
-        this.allChannels = allChannels;
-        this.topic = topic;
-        this.attr = attr;
-        this.filterEmptyMsg = filterEmptyMsg;
-        int cores = Runtime.getRuntime().availableProcessors();
-        this.protocolType = protocol;
-        this.serviceDecoder = serviceDecoder;
-        this.messageHandlerName = messageHandlerName;
-        this.name = name;
-        this.maxConnections = maxCons;
-        this.maxMsgLength = maxMsgLength;
-        this.isCompressed = isCompressed;
-        this.monitorIndex = monitorIndex;
-        this.monitorIndexExt = monitorIndexExt;
     }
 
     @Override
     protected void initChannel(SocketChannel ch) throws Exception {
 
-        if (this.protocolType
-                .equalsIgnoreCase(ConfigConstants.TCP_PROTOCOL)) {
+        if (source.getProtocolName()
+                .equalsIgnoreCase(SourceConstants.SRC_PROTOCOL_TYPE_TCP)) {
             ch.pipeline().addLast("messageDecoder", new LengthFieldBasedFrameDecoder(
-                    this.maxMsgLength, 0,
-                    MSG_LENGTH_LEN, 0, 0, true));
+                    source.getMaxMsgLength(), INLONG_LENGTH_FIELD_OFFSET, INLONG_LENGTH_FIELD_LENGTH,
+                    INLONG_LENGTH_ADJUSTMENT, INLONG_INITIAL_BYTES_TO_STRIP, DEFAULT_FAIL_FAST));
             ch.pipeline().addLast("readTimeoutHandler",
-                    new ReadTimeoutHandler(DEFAULT_READ_IDLE_TIME, TimeUnit.MILLISECONDS));
-        }
+                    new ReadTimeoutHandler(source.getMaxReadIdleTimeMs(), TimeUnit.MILLISECONDS));
+        } else if (source.getProtocolName().equalsIgnoreCase(SourceConstants.SRC_PROTOCOL_TYPE_HTTP)) {
+            // add http message codec
+            ch.pipeline().addLast("msgCodec", new HttpServerCodec());
+            ch.pipeline().addLast("msgAggregator", new HttpObjectAggregator(source.getMaxMsgLength()));
+            ch.pipeline().addLast("readTimeoutHandler",
+                    new ReadTimeoutHandler(source.getMaxReadIdleTimeMs(), TimeUnit.MILLISECONDS));
 
-        if (processor != null) {
+        }
+        // build message handler
+        if (source.getChannelProcessor() != null) {
             try {
                 Class<? extends ChannelInboundHandlerAdapter> clazz =
-                        (Class<? extends ChannelInboundHandlerAdapter>) Class
-                                .forName(messageHandlerName);
-
-                Constructor<?> ctor = clazz.getConstructor(
-                        BaseSource.class, ServiceDecoder.class, ChannelGroup.class,
-                        String.class, String.class, Boolean.class,
-                        Integer.class, Boolean.class, MonitorIndex.class,
-                        MonitorIndexExt.class, String.class);
-
-                ChannelInboundHandlerAdapter messageHandler = (ChannelInboundHandlerAdapter) ctor
-                        .newInstance(source, serviceDecoder, allChannels, topic, attr,
-                                filterEmptyMsg, maxConnections,
-                                isCompressed, monitorIndex, monitorIndexExt, protocolType);
-
+                        (Class<? extends ChannelInboundHandlerAdapter>) Class.forName(source.getMessageHandlerName());
+                Constructor<?> ctor = clazz.getConstructor(BaseSource.class);
+                ChannelInboundHandlerAdapter messageHandler =
+                        (ChannelInboundHandlerAdapter) ctor.newInstance(source);
                 ch.pipeline().addLast("messageHandler", messageHandler);
             } catch (Exception e) {
-                LOG.info("SimpleChannelHandler.newInstance  has error:" + name, e);
+                LOG.error("{} newInstance {} failure!", source.getName(),
+                        source.getMessageHandlerName(), e);
             }
         }
     }
diff --git a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source/ServerMessageHandler.java b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source/ServerMessageHandler.java
index bfb5a3fa55..929407a778 100644
--- a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source/ServerMessageHandler.java
+++ b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source/ServerMessageHandler.java
@@ -18,51 +18,51 @@
 package org.apache.inlong.dataproxy.source;
 
 import org.apache.inlong.common.enums.DataProxyErrCode;
-import org.apache.inlong.common.enums.DataProxyMsgEncType;
-import org.apache.inlong.common.monitor.MonitorIndex;
-import org.apache.inlong.common.monitor.MonitorIndexExt;
+import org.apache.inlong.common.monitor.LogCounter;
 import org.apache.inlong.common.msg.AttributeConstants;
-import org.apache.inlong.common.msg.InLongMsg;
 import org.apache.inlong.common.msg.MsgType;
-import org.apache.inlong.dataproxy.base.ProxyMessage;
-import org.apache.inlong.dataproxy.base.SinkRspEvent;
 import org.apache.inlong.dataproxy.config.CommonConfigHolder;
 import org.apache.inlong.dataproxy.config.ConfigManager;
 import org.apache.inlong.dataproxy.consts.AttrConstants;
 import org.apache.inlong.dataproxy.consts.ConfigConstants;
-import org.apache.inlong.dataproxy.exception.MessageIDException;
-import org.apache.inlong.dataproxy.metrics.DataProxyMetricItemSet;
-import org.apache.inlong.dataproxy.metrics.audit.AuditUtils;
+import org.apache.inlong.dataproxy.consts.StatConstants;
+import org.apache.inlong.dataproxy.source.v0msg.AbsV0MsgCodec;
+import org.apache.inlong.dataproxy.source.v0msg.CodecBinMsg;
+import org.apache.inlong.dataproxy.source.v0msg.CodecTextMsg;
+import org.apache.inlong.dataproxy.source.v1msg.InlongTcpSourceCallback;
 import org.apache.inlong.dataproxy.utils.AddressUtils;
 import org.apache.inlong.dataproxy.utils.DateTimeUtils;
-import org.apache.inlong.dataproxy.utils.MessageUtils;
-import org.apache.inlong.sdk.commons.protocol.EventConstants;
+import org.apache.inlong.sdk.commons.protocol.EventUtils;
+import org.apache.inlong.sdk.commons.protocol.ProxyEvent;
+import org.apache.inlong.sdk.commons.protocol.ProxyPackEvent;
+import org.apache.inlong.sdk.commons.protocol.ProxySdk;
 
-import com.google.common.base.Joiner;
-import com.google.common.base.Splitter;
 import io.netty.buffer.ByteBuf;
+import io.netty.buffer.ByteBufAllocator;
+import io.netty.buffer.Unpooled;
 import io.netty.channel.Channel;
 import io.netty.channel.ChannelHandlerContext;
 import io.netty.channel.ChannelInboundHandlerAdapter;
-import io.netty.channel.group.ChannelGroup;
 import org.apache.commons.lang3.StringUtils;
-import org.apache.commons.lang3.tuple.Pair;
-import org.apache.flume.ChannelException;
 import org.apache.flume.Event;
-import org.apache.flume.channel.ChannelProcessor;
-import org.apache.flume.event.EventBuilder;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.io.IOException;
-import java.nio.ByteBuffer;
 import java.nio.charset.StandardCharsets;
-import java.util.ArrayList;
-import java.util.HashMap;
 import java.util.List;
-import java.util.Map;
-
-import static org.apache.inlong.common.util.NetworkUtils.getLocalIp;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.inlong.dataproxy.source.ServerMessageFactory.INLONG_LENGTH_FIELD_LENGTH;
+import static org.apache.inlong.dataproxy.source.v0msg.MsgFieldConsts.BIN_HB_ATTRLEN_SIZE;
+import static org.apache.inlong.dataproxy.source.v0msg.MsgFieldConsts.BIN_HB_BODYLEN_OFFSET;
+import static org.apache.inlong.dataproxy.source.v0msg.MsgFieldConsts.BIN_HB_BODY_OFFSET;
+import static org.apache.inlong.dataproxy.source.v0msg.MsgFieldConsts.BIN_HB_FIXED_CONTENT_SIZE;
+import static org.apache.inlong.dataproxy.source.v0msg.MsgFieldConsts.BIN_HB_FORMAT_SIZE;
+import static org.apache.inlong.dataproxy.source.v0msg.MsgFieldConsts.BIN_HB_TOTALLEN_SIZE;
+import static org.apache.inlong.dataproxy.source.v0msg.MsgFieldConsts.BIN_HB_VERSION_OFFSET;
+import static org.apache.inlong.dataproxy.source.v0msg.MsgFieldConsts.BIN_MSG_FIXED_CONTENT_SIZE;
+import static org.apache.inlong.dataproxy.source.v0msg.MsgFieldConsts.BIN_MSG_MAGIC;
+import static org.apache.inlong.dataproxy.source.v0msg.MsgFieldConsts.TXT_MSG_FIXED_CONTENT_SIZE;
 
 /**
  * Server message handler
@@ -71,497 +71,630 @@ import static org.apache.inlong.common.util.NetworkUtils.getLocalIp;
 public class ServerMessageHandler extends ChannelInboundHandlerAdapter {
 
     private static final Logger logger = LoggerFactory.getLogger(ServerMessageHandler.class);
+    // log print count
+    private static final LogCounter logCounter = new LogCounter(10, 100000, 30 * 1000);
 
-    private static final String DEFAULT_REMOTE_IP_VALUE = "0.0.0.0";
-
-    private static final String DEFAULT_REMOTE_IDC_VALUE = "0";
+    private static final int INLONG_MSG_V1 = 1;
 
     private static final ConfigManager configManager = ConfigManager.getInstance();
-
-    private static final Joiner.MapJoiner mapJoiner = Joiner.on(AttributeConstants.SEPARATOR)
-            .withKeyValueSeparator(AttributeConstants.KEY_VALUE_SEPARATOR);
-
-    private static final Splitter.MapSplitter mapSplitter = Splitter
-            .on(AttributeConstants.SEPARATOR)
-            .trimResults().withKeyValueSeparator(AttributeConstants.KEY_VALUE_SEPARATOR);
-
-    private BaseSource source;
-
-    private final ChannelGroup allChannels;
-
-    private int maxConnections = Integer.MAX_VALUE;
-
-    private boolean filterEmptyMsg = false;
-
-    private final boolean isCompressed;
-
-    private final ChannelProcessor processor;
-
-    private final ServiceDecoder serviceDecoder;
-
-    private final String defaultTopic;
-
-    private String defaultMXAttr = "m=3";
-
-    private final String protocolType;
-
-    private MonitorIndex monitorIndex;
-
-    private MonitorIndexExt monitorIndexExt;
-
-    //
-    private final DataProxyMetricItemSet metricItemSet;
+    private final BaseSource source;
 
     /**
      * Constructor
      *
      * @param source AbstractSource
-     * @param serviceDecoder ServiceDecoder
-     * @param allChannels ChannelGroup
-     * @param topic Topic
-     * @param attr String
-     * @param filterEmptyMsg Boolean
-     * @param maxCons maxCons
-     * @param isCompressed Is compressed
-     * @param monitorIndex MonitorIndex
-     * @param monitorIndexExt MonitorIndexExt
-     * @param protocolType protocolType
      */
-    public ServerMessageHandler(BaseSource source, ServiceDecoder serviceDecoder,
-            ChannelGroup allChannels,
-            String topic, String attr, Boolean filterEmptyMsg,
-            Integer maxCons, Boolean isCompressed, MonitorIndex monitorIndex,
-            MonitorIndexExt monitorIndexExt, String protocolType) {
+    public ServerMessageHandler(BaseSource source) {
         this.source = source;
-        this.processor = source.getChannelProcessor();
-        this.serviceDecoder = serviceDecoder;
-        this.allChannels = allChannels;
-        this.defaultTopic = topic;
-        if (null != attr) {
-            this.defaultMXAttr = attr;
-        }
-        this.filterEmptyMsg = filterEmptyMsg;
-        this.isCompressed = isCompressed;
-        this.maxConnections = maxCons;
-        this.protocolType = protocolType;
-        this.metricItemSet = source.getMetricItemSet();
-        this.monitorIndex = monitorIndex;
-        this.monitorIndexExt = monitorIndexExt;
     }
 
-    private byte[] newBinMsg(byte[] orgBinMsg, String extraAttr) {
-        final int BIN_MSG_TOTALLEN_OFFSET = 0;
-        final int BIN_MSG_TOTALLEN_SIZE = 4;
-        final int BIN_MSG_BODYLEN_SIZE = 4;
-        final int BIN_MSG_EXTEND_OFFSET = 9;
-        final int BIN_MSG_BODYLEN_OFFSET = 21;
-        final int BIN_MSG_BODY_OFFSET = BIN_MSG_BODYLEN_SIZE + BIN_MSG_BODYLEN_OFFSET;
-        final int BIN_MSG_ATTRLEN_SIZE = 2;
-        final int BIN_MSG_FORMAT_SIZE = 29;
-        final int BIN_MSG_MAGIC_SIZE = 2;
-        final int BIN_MSG_MAGIC = 0xEE01;
-
-        ByteBuffer orgBuf = ByteBuffer.wrap(orgBinMsg);
-        int totalLen = orgBuf.getInt(BIN_MSG_TOTALLEN_OFFSET);
-        int dataLen = orgBuf.getInt(BIN_MSG_BODYLEN_OFFSET);
-        int attrLen = orgBuf.getShort(BIN_MSG_BODY_OFFSET + dataLen);
-
-        int newTotalLen = 0;
-        String strAttr;
-        if (attrLen != 0) {
-            newTotalLen = totalLen + extraAttr.length() + "&".length();
-            strAttr = "&" + extraAttr;
-        } else {
-            newTotalLen = totalLen + extraAttr.length();
-            strAttr = extraAttr;
+    @Override
+    public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception {
+        if (msg == null) {
+            source.fileMetricIncSumStats(StatConstants.EVENT_PKG_READABLE_EMPTY);
+            return;
+        }
+        ByteBuf cb = (ByteBuf) msg;
+        try {
+            int readableLength = cb.readableBytes();
+            if (readableLength == 0 && source.isFilterEmptyMsg()) {
+                cb.clear();
+                source.fileMetricIncSumStats(StatConstants.EVENT_PKG_READABLE_EMPTY);
+                return;
+            }
+            if (readableLength > source.getMaxMsgLength()) {
+                source.fileMetricIncSumStats(StatConstants.EVENT_PKG_READABLE_OVERMAX);
+                throw new Exception("Error msg, readableLength(" + readableLength +
+                        ") > max allowed message length (" + source.getMaxMsgLength() + ")");
+            }
+            // save index
+            cb.markReaderIndex();
+            // read total data length
+            int totalDataLen = cb.readInt();
+            if (readableLength < totalDataLen + INLONG_LENGTH_FIELD_LENGTH) {
+                // reset index when buffer is not satisfied.
+                cb.resetReaderIndex();
+                source.fileMetricIncSumStats(StatConstants.EVENT_PKG_READABLE_UNFILLED);
+                throw new Exception("Error msg, buffer is unfilled, readableLength="
+                        + readableLength + ", totalPackLength=" + totalDataLen + " + 4");
+            }
+            // read type
+            int msgTypeValue = cb.readByte();
+            if (msgTypeValue == 0x0) {
+                // process v1 messsages
+                msgTypeValue = cb.readByte();
+                if (msgTypeValue == INLONG_MSG_V1) {
+                    // decode version 1
+                    int bodyLength = totalDataLen - 2;
+                    processV1Msg(ctx, cb, bodyLength);
+                } else {
+                    // unknown message type
+                    source.fileMetricIncSumStats(StatConstants.EVENT_PKG_MSGTYPE_V1_INVALID);
+                    throw new Exception("Unknown V1 message version, version = " + msgTypeValue);
+                }
+            } else {
+                // process v0 messages
+                Channel channel = ctx.channel();
+                MsgType msgType = MsgType.valueOf(msgTypeValue);
+                final long msgRcvTime = System.currentTimeMillis();
+                if (MsgType.MSG_UNKNOWN == msgType) {
+                    source.fileMetricIncSumStats(StatConstants.EVENT_PKG_MSGTYPE_V0_INVALID);
+                    if (logger.isDebugEnabled()) {
+                        logger.debug("Received unknown message, channel {}", channel);
+                    }
+                    throw new Exception("Unknown V0 message type, type = " + msgTypeValue);
+                } else if (MsgType.MSG_HEARTBEAT == msgType) {
+                    // send response message
+                    flushV0MsgPackage(source, channel,
+                            buildHeartBeatMsgRspPackage(), MsgType.MSG_HEARTBEAT.name());
+                    return;
+                } else if (MsgType.MSG_BIN_HEARTBEAT == msgType) {
+                    procBinHeartbeatMsg(source, channel, cb, totalDataLen);
+                    return;
+                }
+                // process msgType in {2,3,4,5,6,7}
+                AbsV0MsgCodec msgCodec;
+                String strRemoteIP = AddressUtils.getChannelRemoteIP(channel);
+                // check whether totalDataLen is valid.
+                if (MsgType.MSG_BIN_MULTI_BODY == msgType) {
+                    if (totalDataLen < BIN_MSG_FIXED_CONTENT_SIZE) {
+                        source.fileMetricIncSumStats(StatConstants.EVENT_MSG_BIN_TOTALLEN_BELOWMIN);
+                        String errMsg = String.format("Malformed msg, totalDataLen(%d) < min bin7-msg length(%d)",
+                                totalDataLen, BIN_MSG_FIXED_CONTENT_SIZE);
+                        if (logger.isDebugEnabled()) {
+                            logger.debug(errMsg + ", channel {}", channel);
+                        }
+                        throw new Exception(errMsg);
+                    }
+                    msgCodec = new CodecBinMsg(totalDataLen, msgTypeValue, msgRcvTime, strRemoteIP);
+                } else {
+                    if (totalDataLen < TXT_MSG_FIXED_CONTENT_SIZE) {
+                        source.fileMetricIncSumStats(StatConstants.EVENT_MSG_TXT_TOTALLEN_BELOWMIN);
+                        String errMsg = String.format("Malformed msg, totalDataLen(%d) < min txt-msg length(%d)",
+                                totalDataLen, TXT_MSG_FIXED_CONTENT_SIZE);
+                        if (logger.isDebugEnabled()) {
+                            logger.debug(errMsg + ", channel {}", channel);
+                        }
+                        throw new Exception(errMsg);
+                    }
+                    msgCodec = new CodecTextMsg(totalDataLen, msgTypeValue, msgRcvTime, strRemoteIP);
+                }
+                // process request
+                processV0Msg(channel, cb, msgCodec);
+            }
+        } finally {
+            cb.release();
         }
-
-        ByteBuffer dataBuf = ByteBuffer.allocate(newTotalLen + BIN_MSG_TOTALLEN_SIZE);
-        dataBuf
-                .put(orgBuf.array(), 0, dataLen + (BIN_MSG_FORMAT_SIZE - BIN_MSG_MAGIC_SIZE) + attrLen);
-        dataBuf
-                .putShort(dataLen + (BIN_MSG_FORMAT_SIZE - BIN_MSG_ATTRLEN_SIZE - BIN_MSG_MAGIC_SIZE),
-                        (short) (strAttr.length() + attrLen));
-
-        System.arraycopy(strAttr.getBytes(StandardCharsets.UTF_8), 0, dataBuf.array(),
-                dataLen + (BIN_MSG_FORMAT_SIZE - BIN_MSG_MAGIC_SIZE) + attrLen,
-                strAttr.length());
-        int extendField = orgBuf.getShort(BIN_MSG_EXTEND_OFFSET);
-        dataBuf.putShort(BIN_MSG_EXTEND_OFFSET, (short) (extendField | 0x4));
-        dataBuf.putInt(0, newTotalLen);
-        dataBuf.putShort(newTotalLen + BIN_MSG_TOTALLEN_SIZE - BIN_MSG_MAGIC_SIZE,
-                (short) BIN_MSG_MAGIC);
-        return dataBuf.array();
     }
 
     @Override
     public void channelActive(ChannelHandlerContext ctx) throws Exception {
-        // check max allowed connection count
-        if (allChannels.size() - 1 >= maxConnections) {
-            ctx.channel().disconnect();
-            ctx.channel().close();
-            logger.warn("{} refuse to connect = {} , connections = {}, maxConnections = {}",
-                    source.getName(), ctx.channel(), (allChannels.size() - 1), maxConnections);
-            return;
-        }
         // check illegal ip
         if (ConfigManager.getInstance().needChkIllegalIP()) {
             String strRemoteIp = AddressUtils.getChannelRemoteIP(ctx.channel());
             if (strRemoteIp != null
                     && ConfigManager.getInstance().isIllegalIP(strRemoteIp)) {
+                source.fileMetricIncSumStats(StatConstants.EVENT_VISIT_ILLEGAL);
                 ctx.channel().disconnect();
                 ctx.channel().close();
                 logger.error(strRemoteIp + " is Illegal IP, so refuse it !");
                 return;
             }
         }
+        // check max allowed connection count
+        if (source.getAllChannels().size() >= source.getMaxConnections()) {
+            source.fileMetricIncSumStats(StatConstants.EVENT_VISIT_OVERMAX);
+            ctx.channel().disconnect();
+            ctx.channel().close();
+            logger.warn("{} refuse to connect = {} , connections = {}, maxConnections = {}",
+                    source.getName(), ctx.channel(), source.getAllChannels().size(), source.getMaxConnections());
+            return;
+        }
         // add legal channel
-        allChannels.add(ctx.channel());
+        source.getAllChannels().add(ctx.channel());
         ctx.fireChannelActive();
-        logger.info("{} added new channel, current connections = {}, maxConnections = {}",
-                source.getName(), (allChannels.size() - 1), maxConnections);
+        source.fileMetricIncSumStats(StatConstants.EVENT_VISIT_LINKIN);
+        logger.info("{} added new channel {}, current connections = {}, maxConnections = {}",
+                source.getName(), ctx.channel(), source.getAllChannels().size(), source.getMaxConnections());
     }
 
     @Override
     public void channelInactive(ChannelHandlerContext ctx) {
-        logger.error("{} channel inactive {}",
-                source.getName(), ctx.channel());
+        source.fileMetricIncSumStats(StatConstants.EVENT_VISIT_LINKOUT);
         ctx.fireChannelInactive();
-        allChannels.remove(ctx.channel());
+        source.getAllChannels().remove(ctx.channel());
     }
 
     @Override
-    public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception {
-        if (msg == null) {
-            logger.debug("Get null msg, just skip!");
+    public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception {
+        source.fileMetricIncSumStats(StatConstants.EVENT_VISIT_EXCEPTION);
+        if (logCounter.shouldPrint()) {
+            logger.warn("{} received an exception from channel {}",
+                    source.getName(), ctx.channel(), cause);
+        }
+        if (ctx.channel() != null) {
+            source.getAllChannels().remove(ctx.channel());
+            try {
+                ctx.channel().disconnect();
+                ctx.channel().close();
+            } catch (Exception ex) {
+                //
+            }
+        }
+        ctx.close();
+    }
+
+    private void processV0Msg(Channel channel, ByteBuf cb, AbsV0MsgCodec msgCodec) throws Exception {
+        final StringBuilder strBuff = new StringBuilder(512);
+        // decode the request message
+        if (!msgCodec.descMsg(source, cb)) {
+            responseV0Msg(channel, msgCodec, strBuff);
             return;
         }
-        ByteBuf cb = (ByteBuf) msg;
+        // check service status.
+        if (source.isRejectService()) {
+            source.fileMetricIncSumStats(StatConstants.EVENT_SERVICE_CLOSED);
+            msgCodec.setFailureInfo(DataProxyErrCode.SERVICE_CLOSED);
+            responseV0Msg(channel, msgCodec, strBuff);
+            return;
+        }
+        // check if the node is linked to the Manager.
+        if (!ConfigManager.getInstance().isMqClusterReady()) {
+            source.fileMetricIncSumStats(StatConstants.EVENT_SERVICE_SINK_UNREADY);
+            msgCodec.setFailureInfo(DataProxyErrCode.SINK_SERVICE_UNREADY);
+            responseV0Msg(channel, msgCodec, strBuff);
+            return;
+        }
+        // valid and fill extra fields.
+        if (!msgCodec.validAndFillFields(source, strBuff)) {
+            responseV0Msg(channel, msgCodec, strBuff);
+            return;
+        }
+        // build InLong event.
+        Event event = msgCodec.encEventPackage(source, channel);
+        // build metric data item
+        long longDataTime = msgCodec.getDataTimeMs() / 1000 / 60 / 10;
+        longDataTime = longDataTime * 1000 * 60 * 10;
+        String statsKey = strBuff.append(source.getProtocolName()).append(AttrConstants.SEP_HASHTAG)
+                .append(msgCodec.getGroupId()).append(AttrConstants.SEP_HASHTAG)
+                .append(msgCodec.getStreamId()).append(AttrConstants.SEP_HASHTAG)
+                .append(msgCodec.getStrRemoteIP()).append(AttrConstants.SEP_HASHTAG)
+                .append(source.getSrcHost()).append(AttrConstants.SEP_HASHTAG)
+                .append(msgCodec.getMsgProcType()).append(AttrConstants.SEP_HASHTAG)
+                .append(DateTimeUtils.ms2yyyyMMddHHmm(longDataTime)).append(AttrConstants.SEP_HASHTAG)
+                .append(DateTimeUtils.ms2yyyyMMddHHmm(msgCodec.getMsgRcvTime())).toString();
+        strBuff.delete(0, strBuff.length());
         try {
-            Channel remoteChannel = ctx.channel();
-            String strRemoteIP = AddressUtils.getChannelRemoteIP(remoteChannel);
-            int len = cb.readableBytes();
-            if (len == 0 && this.filterEmptyMsg) {
-                logger.debug("Get empty msg from {}, just skip!", strRemoteIP);
-                return;
+            source.getChannelProcessor().processEvent(event);
+            source.fileMetricIncSumStats(StatConstants.EVENT_MSG_V0_POST_SUCCESS);
+            source.fileMetricAddSuccCnt(statsKey, msgCodec.getMsgCount(), 1, msgCodec.getBodyLength());
+            source.addMetric(true, event.getBody().length, event);
+            if (msgCodec.isNeedResp() && !msgCodec.isOrderOrProxy()) {
+                msgCodec.setSuccessInfo();
+                responseV0Msg(channel, msgCodec, strBuff);
             }
-            // parse message
-            Map<String, Object> resultMap = null;
-            final long msgRcvTime = System.currentTimeMillis();
-            try {
-                resultMap = serviceDecoder.extractData(cb,
-                        strRemoteIP, msgRcvTime, remoteChannel);
-                if (resultMap == null || resultMap.isEmpty()) {
-                    logger.debug("Parse message result is null, from {}", strRemoteIP);
-                    return;
+        } catch (Throwable ex) {
+            source.fileMetricIncSumStats(StatConstants.EVENT_MSG_V0_POST_FAILURE);
+            source.fileMetricAddFailCnt(statsKey, 1);
+            source.addMetric(false, event.getBody().length, event);
+            if (msgCodec.isNeedResp() && !msgCodec.isOrderOrProxy()) {
+                msgCodec.setFailureInfo(DataProxyErrCode.PUT_EVENT_TO_CHANNEL_FAILURE,
+                        strBuff.append("Put event to channel failure: ").append(ex.getMessage()).toString());
+                strBuff.delete(0, strBuff.length());
+                responseV0Msg(channel, msgCodec, strBuff);
+            }
+            if (logCounter.shouldPrint()) {
+                logger.error("Error writing event to channel failure.", ex);
+            }
+        }
+    }
+
+    private void processV1Msg(ChannelHandlerContext ctx, ByteBuf cb, int bodyLength) throws Exception {
+        // read bytes
+        byte[] msgBytes = new byte[bodyLength];
+        cb.readBytes(msgBytes);
+        // decode
+        ProxySdk.MessagePack packObject = ProxySdk.MessagePack.parseFrom(msgBytes);
+        // reject service
+        if (source.isRejectService()) {
+            source.addMetric(false, 0, null);
+            source.fileMetricIncSumStats(StatConstants.EVENT_SERVICE_CLOSED);
+            this.responsePackage(ctx, ProxySdk.ResultCode.ERR_REJECT, packObject);
+            return;
+        }
+        // uncompress
+        List<ProxyEvent> events = EventUtils.decodeSdkPack(packObject);
+        // response success if event size is zero
+        if (events.size() == 0) {
+            this.responsePackage(ctx, ProxySdk.ResultCode.SUCCUSS, packObject);
+        }
+        // process
+        if (CommonConfigHolder.getInstance().isResponseAfterSave()) {
+            this.processAndWaitingSave(ctx, packObject, events);
+        } else {
+            this.processAndResponse(ctx, packObject, events);
+        }
+    }
+
+    /**
+     * responsePackage
+     *
+     * @param  ctx
+     * @param  code
+     * @throws Exception
+     */
+    private void responsePackage(ChannelHandlerContext ctx,
+            ProxySdk.ResultCode code,
+            ProxySdk.MessagePack packObject) throws Exception {
+        ProxySdk.ResponseInfo.Builder builder = ProxySdk.ResponseInfo.newBuilder();
+        builder.setResult(code);
+        ProxySdk.MessagePackHeader header = packObject.getHeader();
+        builder.setPackId(header.getPackId());
+
+        // encode
+        byte[] responseBytes = builder.build().toByteArray();
+        //
+        ByteBuf buffer = Unpooled.wrappedBuffer(responseBytes);
+        Channel remoteChannel = ctx.channel();
+        if (remoteChannel.isWritable()) {
+            remoteChannel.write(buffer);
+        } else {
+            buffer.release();
+            logger.warn("Send buffer2 is not writable, disconnect {}", remoteChannel);
+            throw new Exception("Send buffer2 is not writable, disconnect " + remoteChannel);
+        }
+    }
+
+    /**
+     * processAndWaitingSave
+     * @param ctx
+     * @param packObject
+     * @param events
+     * @throws Exception
+     */
+    private void processAndWaitingSave(ChannelHandlerContext ctx,
+            ProxySdk.MessagePack packObject,
+            List<ProxyEvent> events) throws Exception {
+        ProxySdk.MessagePackHeader header = packObject.getHeader();
+        InlongTcpSourceCallback callback = new InlongTcpSourceCallback(ctx, header);
+        String inlongGroupId = header.getInlongGroupId();
+        String inlongStreamId = header.getInlongStreamId();
+        ProxyPackEvent packEvent = new ProxyPackEvent(inlongGroupId, inlongStreamId, events, callback);
+        // put to channel
+        try {
+            source.getChannelProcessor().processEvent(packEvent);
+            events.forEach(event -> {
+                source.addMetric(true, event.getBody().length, event);
+                source.fileMetricIncSumStats(StatConstants.EVENT_MSG_V1_POST_SUCCESS);
+            });
+            boolean awaitResult = callback.getLatch().await(
+                    CommonConfigHolder.getInstance().getMaxResAfterSaveTimeout(), TimeUnit.MILLISECONDS);
+            if (!awaitResult) {
+                if (!callback.getHasResponsed().getAndSet(true)) {
+                    this.responsePackage(ctx, ProxySdk.ResultCode.ERR_REJECT, packObject);
                 }
-            } catch (MessageIDException ex) {
-                logger.error("MessageIDException ex = {}", ex);
-                throw new IOException(ex.getCause());
             }
-            // get msgType from parsed result
-            MsgType msgType = (MsgType) resultMap.get(ConfigConstants.MSG_TYPE);
-            // get attribute data from parsed result
-            Map<String, String> commonAttrMap =
-                    (Map<String, String>) resultMap.get(ConfigConstants.COMMON_ATTR_MAP);
-            if (commonAttrMap == null) {
-                commonAttrMap = new HashMap<>();
+        } catch (Throwable ex) {
+            logger.error("Process Controller Event error can't write event to channel.", ex);
+            events.forEach(event -> {
+                source.addMetric(false, event.getBody().length, event);
+                source.fileMetricIncSumStats(StatConstants.EVENT_MSG_V1_POST_DROPPED);
+            });
+            if (!callback.getHasResponsed().getAndSet(true)) {
+                this.responsePackage(ctx, ProxySdk.ResultCode.ERR_REJECT, packObject);
             }
-            // check whether extract data failure
-            String errCode = commonAttrMap.get(AttributeConstants.MESSAGE_PROCESS_ERRCODE);
-            if (!StringUtils.isEmpty(errCode)
-                    && !DataProxyErrCode.SUCCESS.getErrCodeStr().equals(errCode)) {
-                MessageUtils.sourceReturnRspPackage(
-                        commonAttrMap, resultMap, remoteChannel, msgType);
+        }
+    }
+
+    /**
+     * processAndResponse
+     * @param ctx
+     * @param packObject
+     * @param events
+     * @throws Exception
+     */
+    private void processAndResponse(ChannelHandlerContext ctx,
+            ProxySdk.MessagePack packObject,
+            List<ProxyEvent> events) throws Exception {
+        for (ProxyEvent event : events) {
+            // get configured topic name
+            String topic = configManager.getTopicName(event.getInlongGroupId(), event.getInlongStreamId());
+            if (StringUtils.isBlank(topic)) {
+                source.fileMetricIncSumStats(StatConstants.EVENT_CONFIG_TOPIC_MISSING);
+                source.addMetric(false, event.getBody().length, event);
+                this.responsePackage(ctx, ProxySdk.ResultCode.ERR_ID_ERROR, packObject);
                 return;
             }
-            // process heartbeat message
-            if (MsgType.MSG_HEARTBEAT.equals(msgType)
-                    || MsgType.MSG_BIN_HEARTBEAT.equals(msgType)) {
-                MessageUtils.sourceReturnRspPackage(
-                        commonAttrMap, resultMap, remoteChannel, msgType);
+            event.setTopic(topic);
+            // put to channel
+            try {
+                source.getChannelProcessor().processEvent(event);
+                source.addMetric(true, event.getBody().length, event);
+                source.fileMetricIncSumStats(StatConstants.EVENT_MSG_V1_POST_SUCCESS);
+            } catch (Throwable ex) {
+                logger.error("Process Controller Event error can't write event to channel.", ex);
+                source.addMetric(false, event.getBody().length, event);
+                this.responsePackage(ctx, ProxySdk.ResultCode.ERR_REJECT, packObject);
+                source.fileMetricIncSumStats(StatConstants.EVENT_MSG_V1_POST_DROPPED);
                 return;
             }
-            // reject unsupported messages
-            if (commonAttrMap.containsKey(ConfigConstants.FILE_CHECK_DATA)
-                    || commonAttrMap.containsKey(ConfigConstants.MINUTE_CHECK_DATA)) {
-                commonAttrMap.put(AttributeConstants.MESSAGE_PROCESS_ERRCODE,
-                        DataProxyErrCode.UNSUPPORTED_EXTEND_FIELD_VALUE.getErrCodeStr());
-                MessageUtils.sourceReturnRspPackage(
-                        commonAttrMap, resultMap, remoteChannel, msgType);
-                return;
+        }
+        this.responsePackage(ctx, ProxySdk.ResultCode.SUCCUSS, packObject);
+    }
+
+    /**
+     *  Return response to client in source
+     */
+    private void responseV0Msg(Channel channel, AbsV0MsgCodec msgObj, StringBuilder strBuff) throws Exception {
+        // check channel status
+        if (channel == null || !channel.isWritable()) {
+            source.fileMetricIncSumStats(StatConstants.EVENT_REMOTE_UNWRITABLE);
+            if (logCounter.shouldPrint()) {
+                logger.warn("Prepare send msg but channel full, msgType={}, attr={}, channel={}",
+                        msgObj.getMsgType(), msgObj.getAttr(), channel);
             }
-            // check message's groupId, streamId, topic
-            List<ProxyMessage> msgList =
-                    (List<ProxyMessage>) resultMap.get(ConfigConstants.MSG_LIST);
-            if (msgList == null) {
-                commonAttrMap.put(AttributeConstants.MESSAGE_PROCESS_ERRCODE,
-                        DataProxyErrCode.EMPTY_MSG.getErrCodeStr());
-                MessageUtils.sourceReturnRspPackage(
-                        commonAttrMap, resultMap, remoteChannel, msgType);
-                return;
+            throw new Exception("Prepare send msg but channel full");
+        }
+        // check whether return response message
+        if (!msgObj.isNeedResp()) {
+            return;
+        }
+        // build return attribute string
+        strBuff.append(ConfigConstants.DATAPROXY_IP_KEY)
+                .append(AttributeConstants.KEY_VALUE_SEPARATOR).append(source.getSrcHost());
+        if (msgObj.getErrCode() != DataProxyErrCode.SUCCESS) {
+            strBuff.append(AttributeConstants.SEPARATOR).append(AttributeConstants.MESSAGE_PROCESS_ERRCODE)
+                    .append(AttributeConstants.KEY_VALUE_SEPARATOR).append(msgObj.getErrCode().getErrCodeStr());
+            if (StringUtils.isNotEmpty(msgObj.getErrMsg())) {
+                strBuff.append(AttributeConstants.SEPARATOR).append(AttributeConstants.MESSAGE_PROCESS_ERRMSG)
+                        .append(AttributeConstants.KEY_VALUE_SEPARATOR).append(msgObj.getErrMsg());
             }
-            // check sink service status
-            if (!ConfigManager.getInstance().isMqClusterReady()) {
-                commonAttrMap.put(AttributeConstants.MESSAGE_PROCESS_ERRCODE,
-                        DataProxyErrCode.SINK_SERVICE_UNREADY.getErrCodeStr());
-                MessageUtils.sourceReturnRspPackage(
-                        commonAttrMap, resultMap, remoteChannel, msgType);
-                return;
+        }
+        if (StringUtils.isNotEmpty(msgObj.getAttr())) {
+            strBuff.append(AttributeConstants.SEPARATOR).append(msgObj.getAttr());
+        }
+        // build and send response message
+        ByteBuf retData;
+        MsgType msgType = MsgType.valueOf(msgObj.getMsgType());
+        if (MsgType.MSG_BIN_MULTI_BODY.equals(msgType)) {
+            retData = buildBinMsgRspPackage(strBuff.toString(), msgObj.getUniq());
+        } else {
+            retData = buildTxtMsgRspPackage(msgType, strBuff.toString(), msgObj);
+        }
+        strBuff.delete(0, strBuff.length());
+        flushV0MsgPackage(source, channel, retData, msgObj.getAttr());
+    }
+
+    /**
+     * extract and process bin heart beat msg, message type is 8
+     */
+    private void procBinHeartbeatMsg(BaseSource source, Channel channel,
+            ByteBuf cb, int totalDataLen) throws Exception {
+        // Check if the message is complete and legal
+        if (totalDataLen < BIN_HB_FIXED_CONTENT_SIZE) {
+            source.fileMetricIncSumStats(StatConstants.EVENT_MSG_HB_TOTALLEN_BELOWMIN);
+            String errMsg = String.format("Malformed msg, totalDataLen(%d) < min hb-msg length(%d)",
+                    totalDataLen, BIN_HB_FIXED_CONTENT_SIZE);
+            if (logger.isDebugEnabled()) {
+                logger.debug(errMsg + ", channel {}", channel);
             }
-            // convert message data
-            Map<String, HashMap<String, List<ProxyMessage>>> messageMap =
-                    new HashMap<>(msgList.size());
-            if (!convertMsgList(msgList, commonAttrMap, messageMap, strRemoteIP)) {
-                MessageUtils.sourceReturnRspPackage(
-                        commonAttrMap, resultMap, remoteChannel, msgType);
-                return;
+            throw new Exception(errMsg);
+        }
+        // check validation
+        int msgHeadPos = cb.readerIndex() - 5;
+        int bodyLen = cb.getInt(msgHeadPos + BIN_HB_BODYLEN_OFFSET);
+        int attrLen = cb.getShort(msgHeadPos + BIN_HB_BODY_OFFSET + bodyLen);
+        int msgMagic = cb.getUnsignedShort(msgHeadPos
+                + BIN_HB_BODY_OFFSET + bodyLen + BIN_HB_ATTRLEN_SIZE + attrLen);
+        if (msgMagic != BIN_MSG_MAGIC) {
+            source.fileMetricIncSumStats(StatConstants.EVENT_MSG_HB_MAGIC_UNEQUAL);
+            String errMsg = String.format(
+                    "Malformed msg, msgMagic(%d) != %d", msgMagic, BIN_MSG_MAGIC);
+            if (logger.isDebugEnabled()) {
+                logger.debug(errMsg + ", channel {}", channel);
             }
-            // send messages to channel
-            formatMessagesAndSend(ctx, commonAttrMap, resultMap,
-                    messageMap, strRemoteIP, msgType, msgRcvTime);
-            // return response
-            if (!MessageUtils.isSinkRspType(commonAttrMap)) {
-                MessageUtils.sourceReturnRspPackage(
-                        commonAttrMap, resultMap, remoteChannel, msgType);
+            throw new Exception(errMsg);
+        }
+        if (totalDataLen + BIN_HB_TOTALLEN_SIZE < (bodyLen + attrLen + BIN_HB_FORMAT_SIZE)) {
+            source.fileMetricIncSumStats(StatConstants.EVENT_MSG_HB_LEN_MALFORMED);
+            String errMsg = String.format(
+                    "Malformed msg, bodyLen(%d) + attrLen(%d) > totalDataLen(%d)",
+                    bodyLen, attrLen, totalDataLen);
+            if (logger.isDebugEnabled()) {
+                logger.debug(errMsg + ", channel {}", channel);
             }
-        } finally {
-            cb.release();
+            throw new Exception(errMsg);
+        }
+        // read message content
+        byte version = cb.getByte(msgHeadPos + BIN_HB_VERSION_OFFSET);
+        byte[] attrData = null;
+        if (attrLen > 0) {
+            attrData = new byte[attrLen];
+            cb.getBytes(msgHeadPos + BIN_HB_BODY_OFFSET
+                    + bodyLen + BIN_HB_ATTRLEN_SIZE, attrData, 0, attrLen);
         }
+        // build and send response message
+        flushV0MsgPackage(source, channel,
+                buildHBRspPackage(attrData, version, 0), MsgType.MSG_BIN_HEARTBEAT.name());
     }
 
-    @Override
-    public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception {
-        monitorIndexExt.incrementAndGet("EVENT_OTHEREXP");
-        ctx.close();
-        logger.error("{} exception caught cause = {}", source.getName(), cause);
+    /**
+     * Build bin-msg response message ByteBuf
+     *
+     * @param attrs   the return attribute
+     * @param uniqVal sequence Id
+     * @return ByteBuf
+     */
+    public static ByteBuf buildBinMsgRspPackage(String attrs, long uniqVal) {
+        // calculate total length
+        // binTotalLen = mstType + uniq + attrsLen + attrs + magic
+        int binTotalLen = 1 + 4 + 2 + 2;
+        if (null != attrs) {
+            binTotalLen += attrs.length();
+        }
+        // allocate buffer and write fields
+        ByteBuf binBuffer = ByteBufAllocator.DEFAULT.buffer(4 + binTotalLen);
+        binBuffer.writeInt(binTotalLen);
+        binBuffer.writeByte(MsgType.MSG_BIN_MULTI_BODY.getValue());
+        byte[] uniq = new byte[4];
+        uniq[0] = (byte) ((uniqVal >> 24) & 0xFF);
+        uniq[1] = (byte) ((uniqVal >> 16) & 0xFF);
+        uniq[2] = (byte) ((uniqVal >> 8) & 0xFF);
+        uniq[3] = (byte) (uniqVal & 0xFF);
+        binBuffer.writeBytes(uniq);
+        if (null != attrs) {
+            binBuffer.writeShort(attrs.length());
+            binBuffer.writeBytes(attrs.getBytes(StandardCharsets.UTF_8));
+        } else {
+            binBuffer.writeShort(0x0);
+        }
+        binBuffer.writeShort(0xee01);
+        return binBuffer;
     }
 
     /**
-     * Complete the message content and covert proxy message to map
-     *
-     * @param msgList  the message list
-     * @param commonAttrMap common attribute map
-     * @param messageMap    message list
-     * @param strRemoteIP   remote ip
+     * Build default-msg response message ByteBuf
      *
-     * @return  convert result
+     * @param msgType  the message type
+     * @param attrs    the return attribute
+     * @return ByteBuf
      */
-    private boolean convertMsgList(List<ProxyMessage> msgList, Map<String, String> commonAttrMap,
-            Map<String, HashMap<String, List<ProxyMessage>>> messageMap,
-            String strRemoteIP) {
-        for (ProxyMessage message : msgList) {
-            String configTopic = null;
-            String groupId = message.getGroupId();
-            String streamId = message.getStreamId();
-            // get topic by groupId and streamId
-            if (null == groupId) {
-                String num2name = commonAttrMap.get(AttrConstants.NUM2NAME);
-                String groupIdNum = commonAttrMap.get(AttrConstants.GROUPID_NUM);
-                String streamIdNum = commonAttrMap.get(AttrConstants.STREAMID_NUM);
-                // get configured groupId and steamId by numbers
-                if (!configManager.isGroupIdNumConfigEmpty()
-                        && !configManager.isStreamIdNumConfigEmpty()) {
-                    groupId = configManager.getGroupIdNameByNum(groupIdNum);
-                    streamId = configManager.getStreamIdNameByIdNum(groupIdNum, streamIdNum);
-                    if (groupId != null && streamId != null) {
-                        if ((configManager.isEnableNum2NameTrans(groupIdNum) && "TRUE".equalsIgnoreCase(num2name))) {
-                            String extraAttr = "groupId=" + groupId + "&" + "streamId=" + streamId;
-                            message.setData(newBinMsg(message.getData(), extraAttr));
-                        }
-                        // reset groupId and streamId to message and attrMap
-                        message.setGroupId(groupId);
-                        message.setStreamId(streamId);
-                        // get configured topic name
-                        configTopic = configManager.getTopicName(groupId, streamId);
-                    }
-                }
-            } else {
-                // get configured topic name
-                configTopic = configManager.getTopicName(groupId, streamId);
-            }
-            // check topic configure
-            if (StringUtils.isEmpty(configTopic)) {
-                if (CommonConfigHolder.getInstance().isEnableUnConfigTopicAccept()) {
-                    configTopic = this.defaultTopic;
-                } else {
-                    commonAttrMap.put(AttributeConstants.MESSAGE_PROCESS_ERRCODE,
-                            DataProxyErrCode.UNCONFIGURED_GROUPID_OR_STREAMID.getErrCodeStr());
-                    logger.error("Topic for message is null , inlongGroupId = {}, inlongStreamId = {}",
-                            groupId, streamId);
-                    return false;
-                }
-            }
-            if (streamId == null) {
-                streamId = "";
-                message.setStreamId(streamId);
-            }
-            // append topic
-            message.setTopic(configTopic);
-            commonAttrMap.put(AttributeConstants.NODE_IP, strRemoteIP);
-            // add ProxyMessage
-            HashMap<String, List<ProxyMessage>> streamIdMsgMap = messageMap
-                    .computeIfAbsent(configTopic, k -> new HashMap<>());
-            List<ProxyMessage> streamIdMsgList = streamIdMsgMap
-                    .computeIfAbsent(streamId, k -> new ArrayList<>());
-            streamIdMsgList.add(message);
-        }
-        return true;
+    public static ByteBuf buildTxtMsgRspPackage(MsgType msgType, String attrs) {
+        int attrsLen = 0;
+        int bodyLen = 0;
+        if (attrs != null) {
+            attrsLen = attrs.length();
+        }
+        // backTotalLen = mstType + bodyLen + body + attrsLen + attrs
+        int backTotalLen = 1 + 4 + bodyLen + 4 + attrsLen;
+        ByteBuf buffer = ByteBufAllocator.DEFAULT.buffer(4 + backTotalLen);
+        buffer.writeInt(backTotalLen);
+        buffer.writeByte(msgType.getValue());
+        buffer.writeInt(bodyLen);
+        buffer.writeInt(attrsLen);
+        if (attrsLen > 0) {
+            buffer.writeBytes(attrs.getBytes(StandardCharsets.UTF_8));
+        }
+        return buffer;
     }
 
     /**
-     * format message to event and send to channel
+     * Build default-msg response message ByteBuf
      *
-     * @param ctx  client connect
-     * @param commonAttrMap common attribute map
-     * @param resultMap    the result map
-     * @param messageMap    message list
-     * @param strRemoteIP   remote ip
-     * @param msgType    the message type
-     * @param msgRcvTime  the received time
+     * @param msgType  the message type
+     * @param attrs    the return attribute
+     * @param msgObj   the request message object
+     * @return ByteBuf
      */
-    private void formatMessagesAndSend(ChannelHandlerContext ctx, Map<String, String> commonAttrMap,
-            Map<String, Object> resultMap,
-            Map<String, HashMap<String, List<ProxyMessage>>> messageMap,
-            String strRemoteIP, MsgType msgType, long msgRcvTime) throws MessageIDException {
-
-        int inLongMsgVer = 1;
-        if (MsgType.MSG_MULTI_BODY_ATTR.equals(msgType)) {
-            inLongMsgVer = 3;
-        } else if (MsgType.MSG_BIN_MULTI_BODY.equals(msgType)) {
-            inLongMsgVer = 4;
-        }
-        StringBuilder strBuff = new StringBuilder(512);
-        int recordMsgCnt = Integer.parseInt(commonAttrMap.get(AttributeConstants.MESSAGE_COUNT));
-        // process each ProxyMessage
-        for (Map.Entry<String, HashMap<String, List<ProxyMessage>>> topicEntry : messageMap.entrySet()) {
-            for (Map.Entry<String, List<ProxyMessage>> streamIdEntry : topicEntry.getValue().entrySet()) {
-                // build InLongMsg
-                String groupId = null;
-                InLongMsg inLongMsg = InLongMsg.newInLongMsg(this.isCompressed, inLongMsgVer);
-                if (MsgType.MSG_MULTI_BODY_ATTR.equals(msgType) || MsgType.MSG_MULTI_BODY.equals(msgType)) {
-                    for (ProxyMessage message : streamIdEntry.getValue()) {
-                        if (StringUtils.isEmpty(groupId)) {
-                            groupId = message.getGroupId();
-                        }
-                        message.getAttributeMap().put(AttributeConstants.MESSAGE_COUNT, String.valueOf(1));
-                        inLongMsg.addMsg(mapJoiner.join(message.getAttributeMap()), message.getData());
-                    }
-                } else if (MsgType.MSG_BIN_MULTI_BODY.equals(msgType)) {
-                    for (ProxyMessage message : streamIdEntry.getValue()) {
-                        if (StringUtils.isEmpty(groupId)) {
-                            groupId = message.getGroupId();
-                        }
-                        inLongMsg.addMsg(message.getData());
-                    }
-                } else {
-                    for (ProxyMessage message : streamIdEntry.getValue()) {
-                        if (StringUtils.isEmpty(groupId)) {
-                            groupId = message.getGroupId();
-                        }
-                        inLongMsg.addMsg(mapJoiner.join(message.getAttributeMap()), message.getData());
-                    }
-                }
-                commonAttrMap.put(AttributeConstants.MESSAGE_COUNT, String.valueOf(recordMsgCnt));
-                // build headers
-                Map<String, String> headers = new HashMap<>();
-                headers.put(AttributeConstants.GROUP_ID, groupId);
-                headers.put(AttributeConstants.STREAM_ID, streamIdEntry.getKey());
-                headers.put(ConfigConstants.TOPIC_KEY, topicEntry.getKey());
-                String strDataTime = commonAttrMap.get(AttributeConstants.DATA_TIME);
-                headers.put(AttributeConstants.DATA_TIME, strDataTime);
-                headers.put(ConfigConstants.REMOTE_IP_KEY, strRemoteIP);
-                headers.put(ConfigConstants.REMOTE_IDC_KEY, DEFAULT_REMOTE_IDC_VALUE);
-                headers.put(ConfigConstants.MSG_COUNTER_KEY,
-                        commonAttrMap.get(AttributeConstants.MESSAGE_COUNT));
-                headers.put(ConfigConstants.MSG_ENCODE_VER,
-                        DataProxyMsgEncType.MSG_ENCODE_TYPE_INLONGMSG.getStrId());
-                headers.put(EventConstants.HEADER_KEY_VERSION,
-                        DataProxyMsgEncType.MSG_ENCODE_TYPE_INLONGMSG.getStrId());
-                headers.put(AttributeConstants.RCV_TIME,
-                        commonAttrMap.get(AttributeConstants.RCV_TIME));
-                headers.put(ConfigConstants.DECODER_ATTRS,
-                        (String) resultMap.get(ConfigConstants.DECODER_ATTRS));
-                // add extra key-value information
-                headers.put(AttributeConstants.UNIQ_ID,
-                        commonAttrMap.get(AttributeConstants.UNIQ_ID));
-                if ("false".equals(commonAttrMap.get(AttributeConstants.MESSAGE_IS_ACK))) {
-                    headers.put(AttributeConstants.MESSAGE_IS_ACK, "false");
-                }
-                String syncSend = commonAttrMap.get(AttributeConstants.MESSAGE_SYNC_SEND);
-                if (StringUtils.isNotEmpty(syncSend)) {
-                    headers.put(AttributeConstants.MESSAGE_SYNC_SEND, syncSend);
-                }
-                String proxySend = commonAttrMap.get(AttributeConstants.MESSAGE_PROXY_SEND);
-                if (StringUtils.isNotEmpty(proxySend)) {
-                    headers.put(AttributeConstants.MESSAGE_PROXY_SEND, proxySend);
-                }
-                String partitionKey = commonAttrMap.get(AttributeConstants.MESSAGE_PARTITION_KEY);
-                if (StringUtils.isNotEmpty(partitionKey)) {
-                    headers.put(AttributeConstants.MESSAGE_PARTITION_KEY, partitionKey);
-                }
-                String sequenceId = commonAttrMap.get(AttributeConstants.SEQUENCE_ID);
-                if (StringUtils.isNotEmpty(sequenceId)) {
-                    strBuff.append(topicEntry.getKey()).append(AttributeConstants.SEPARATOR)
-                            .append(streamIdEntry.getKey())
-                            .append(AttributeConstants.SEPARATOR).append(sequenceId);
-                    headers.put(ConfigConstants.SEQUENCE_ID, strBuff.toString());
-                    strBuff.delete(0, strBuff.length());
-                }
-                final byte[] data = inLongMsg.buildArray();
-                Event event = EventBuilder.withBody(data, headers);
-                event.getHeaders().putAll(headers);
-                inLongMsg.reset();
-                Pair<Boolean, String> evenProcType =
-                        MessageUtils.getEventProcType(syncSend, proxySend);
-                if (evenProcType.getLeft()) {
-                    event = new SinkRspEvent(event, msgType, ctx.channel());
-                }
-                // build metric data item
-                long longDataTime = Long.parseLong(strDataTime);
-                longDataTime = longDataTime / 1000 / 60 / 10;
-                longDataTime = longDataTime * 1000 * 60 * 10;
-                strBuff.append(protocolType).append(AttrConstants.SEPARATOR)
-                        .append(groupId).append(AttrConstants.SEPARATOR)
-                        .append(streamIdEntry.getKey()).append(AttrConstants.SEPARATOR)
-                        .append(strRemoteIP).append(AttrConstants.SEPARATOR)
-                        .append(getLocalIp()).append(AttrConstants.SEPARATOR)
-                        .append(evenProcType.getRight()).append(AttrConstants.SEPARATOR)
-                        .append(DateTimeUtils.ms2yyyyMMddHHmm(longDataTime)).append(AttrConstants.SEPARATOR)
-                        .append(DateTimeUtils.ms2yyyyMMddHHmm(msgRcvTime));
-                try {
-                    processor.processEvent(event);
-                    monitorIndexExt.incrementAndGet("EVENT_SUCCESS");
-                    this.addStatistics(true, data.length, event);
-                    monitorIndex.addAndGet(strBuff.toString(),
-                            recordMsgCnt, 1, data.length, 0);
-                    strBuff.delete(0, strBuff.length());
-                } catch (Throwable ex) {
-                    logger.error("Error writting to channel, data will discard.", ex);
-                    monitorIndexExt.incrementAndGet("EVENT_DROPPED");
-                    monitorIndex.addAndGet(strBuff.toString(), 0, 0, 0, recordMsgCnt);
-                    this.addStatistics(false, data.length, event);
-                    strBuff.delete(0, strBuff.length());
-                    throw new ChannelException("ProcessEvent error can't write event to channel.");
-                }
+    private ByteBuf buildTxtMsgRspPackage(MsgType msgType, String attrs, AbsV0MsgCodec msgObj) {
+        int attrsLen = 0;
+        int bodyLen = 0;
+        byte[] backBody = null;
+        if (attrs != null) {
+            attrsLen = attrs.length();
+        }
+        if (MsgType.MSG_ORIGINAL_RETURN.equals(msgType)) {
+            backBody = msgObj.getOrigBody();
+            if (backBody != null) {
+                bodyLen = backBody.length;
             }
         }
+        // backTotalLen = mstType + bodyLen + body + attrsLen + attrs
+        int backTotalLen = 1 + 4 + bodyLen + 4 + attrsLen;
+        ByteBuf buffer = ByteBufAllocator.DEFAULT.buffer(4 + backTotalLen);
+        buffer.writeInt(backTotalLen);
+        buffer.writeByte(msgType.getValue());
+        buffer.writeInt(bodyLen);
+        if (bodyLen > 0) {
+            buffer.writeBytes(backBody);
+        }
+        buffer.writeInt(attrsLen);
+        if (attrsLen > 0) {
+            buffer.writeBytes(attrs.getBytes(StandardCharsets.UTF_8));
+        }
+        return buffer;
     }
 
     /**
-     * add statistics information
+     * Build heartbeat response message ByteBuf
      *
-     * @param isSuccess  success or failure
-     * @param size    message size
-     * @param event   message event
+     * @param attrData  the attribute data
+     * @param version   the version
+     * @param loadValue the node load value
+     * @return ByteBuf
      */
-    private void addStatistics(boolean isSuccess, long size, Event event) {
-        if (event == null) {
-            return;
+    private ByteBuf buildHBRspPackage(byte[] attrData, byte version, int loadValue) {
+        // calculate total length
+        // binTotalLen = mstType + dataTime + version + bodyLen + body + attrsLen + attrs + magic
+        int binTotalLen = 1 + 4 + 1 + 4 + 2 + 2 + 2;
+        if (null != attrData) {
+            binTotalLen += attrData.length;
         }
-        this.metricItemSet.fillSrcMetricItemsByEvent(event, isSuccess, size);
-        if (isSuccess) {
-            AuditUtils.add(AuditUtils.AUDIT_ID_DATAPROXY_READ_SUCCESS, event);
+        // check load value
+        if (loadValue == 0 || loadValue == (-1)) {
+            loadValue = 0xffff;
+        }
+        // allocate buffer and write fields
+        ByteBuf binBuffer = ByteBufAllocator.DEFAULT.buffer(4 + binTotalLen);
+        binBuffer.writeInt(binTotalLen);
+        binBuffer.writeByte(MsgType.MSG_BIN_HEARTBEAT.getValue());
+        binBuffer.writeInt((int) (System.currentTimeMillis() / 1000));
+        binBuffer.writeByte(version);
+        binBuffer.writeInt(2);
+        binBuffer.writeShort(loadValue);
+        if (null != attrData) {
+            binBuffer.writeShort(attrData.length);
+            binBuffer.writeBytes(attrData);
+        } else {
+            binBuffer.writeShort(0x0);
+        }
+        binBuffer.writeShort(0xee01);
+        return binBuffer;
+    }
+
+    /**
+     * Build hearbeat(1)-msg response message ByteBuf
+     *
+     * @return ByteBuf
+     */
+    private ByteBuf buildHeartBeatMsgRspPackage() {
+        ByteBuf buffer = ByteBufAllocator.DEFAULT.buffer(5);
+        // magic data
+        buffer.writeBytes(new byte[]{0, 0, 0, 1, 1});
+        return buffer;
+    }
+
+    private void flushV0MsgPackage(BaseSource source, Channel channel,
+            ByteBuf binBuffer, String orgAttr) throws Exception {
+        if (channel == null || !channel.isWritable()) {
+            // release allocated ByteBuf
+            binBuffer.release();
+            source.fileMetricIncSumStats(StatConstants.EVENT_REMOTE_UNWRITABLE);
+            if (logCounter.shouldPrint()) {
+                logger.warn("Send msg but channel full, attr={}, channel={}", orgAttr, channel);
+            }
+            throw new Exception("Send response but channel full");
         }
+        channel.writeAndFlush(binBuffer);
     }
 }
diff --git a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source/ServiceDecoder.java b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source/ServiceDecoder.java
deleted file mode 100644
index 6c5048bc4e..0000000000
--- a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source/ServiceDecoder.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.inlong.dataproxy.source;
-
-import io.netty.buffer.ByteBuf;
-import io.netty.channel.Channel;
-
-import java.util.Map;
-
-public interface ServiceDecoder {
-
-    int HEAD_LENGTH = 4;
-
-    /**
-     * extract data from buffer and convert it into map.
-     * 
-     * @param cb           the message Byte buffer
-     * @param strRemoteIP  the remote ip message sent
-     * @param msgRcvTime   the received message time
-     * @param channel      the channel
-     * @return Map         the message map
-     * @throws Exception
-     */
-    Map<String, Object> extractData(ByteBuf cb, String strRemoteIP,
-            long msgRcvTime, Channel channel) throws Exception;
-}
diff --git a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source2/SimpleHttpSource.java b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source/SimpleHttpSource.java
similarity index 99%
rename from inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source2/SimpleHttpSource.java
rename to inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source/SimpleHttpSource.java
index 7bae8c2852..48962c2923 100644
--- a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source2/SimpleHttpSource.java
+++ b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source/SimpleHttpSource.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.inlong.dataproxy.source2;
+package org.apache.inlong.dataproxy.source;
 
 import org.apache.inlong.dataproxy.config.ConfigManager;
 import org.apache.inlong.dataproxy.utils.ConfStringUtils;
diff --git a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source/SimpleMessageHandler.java b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source/SimpleMessageHandler.java
deleted file mode 100644
index 2d0c8ebbd4..0000000000
--- a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source/SimpleMessageHandler.java
+++ /dev/null
@@ -1,673 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.inlong.dataproxy.source;
-
-import org.apache.inlong.common.enums.DataProxyMsgEncType;
-import org.apache.inlong.common.msg.AttributeConstants;
-import org.apache.inlong.common.msg.InLongMsg;
-import org.apache.inlong.common.msg.MsgType;
-import org.apache.inlong.dataproxy.base.ProxyMessage;
-import org.apache.inlong.dataproxy.config.ConfigManager;
-import org.apache.inlong.dataproxy.consts.AttrConstants;
-import org.apache.inlong.dataproxy.consts.ConfigConstants;
-import org.apache.inlong.dataproxy.exception.MessageIDException;
-import org.apache.inlong.dataproxy.metrics.DataProxyMetricItem;
-import org.apache.inlong.dataproxy.metrics.DataProxyMetricItemSet;
-import org.apache.inlong.dataproxy.metrics.audit.AuditUtils;
-import org.apache.inlong.dataproxy.utils.AddressUtils;
-import org.apache.inlong.dataproxy.utils.Constants;
-import org.apache.inlong.sdk.commons.protocol.EventConstants;
-
-import com.google.common.base.Joiner;
-import com.google.common.base.Splitter;
-import io.netty.buffer.ByteBuf;
-import io.netty.buffer.ByteBufAllocator;
-import io.netty.channel.Channel;
-import io.netty.channel.ChannelHandlerContext;
-import io.netty.channel.ChannelInboundHandlerAdapter;
-import io.netty.channel.group.ChannelGroup;
-import org.apache.commons.lang3.StringUtils;
-import org.apache.commons.lang3.math.NumberUtils;
-import org.apache.flume.ChannelException;
-import org.apache.flume.Event;
-import org.apache.flume.channel.ChannelProcessor;
-import org.apache.flume.event.EventBuilder;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.net.SocketAddress;
-import java.nio.ByteBuffer;
-import java.nio.charset.StandardCharsets;
-import java.text.SimpleDateFormat;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.atomic.AtomicReference;
-
-import static org.apache.inlong.dataproxy.consts.ConfigConstants.SLA_METRIC_DATA;
-import static org.apache.inlong.dataproxy.consts.ConfigConstants.SLA_METRIC_GROUPID;
-
-/**
- * Server message handler
- *
- */
-public class SimpleMessageHandler extends ChannelInboundHandlerAdapter {
-
-    private static final Logger logger = LoggerFactory.getLogger(SimpleMessageHandler.class);
-
-    private static final String DEFAULT_REMOTE_IP_VALUE = "0.0.0.0";
-    private static final String DEFAULT_REMOTE_IDC_VALUE = "0";
-    private static final ConfigManager configManager = ConfigManager.getInstance();
-    private static final Joiner.MapJoiner mapJoiner = Joiner.on(AttributeConstants.SEPARATOR)
-            .withKeyValueSeparator(AttributeConstants.KEY_VALUE_SEPARATOR);
-    private static final Splitter.MapSplitter mapSplitter = Splitter
-            .on(AttributeConstants.SEPARATOR)
-            .trimResults().withKeyValueSeparator(AttributeConstants.KEY_VALUE_SEPARATOR);
-
-    private static final ThreadLocal<SimpleDateFormat> dateFormator = new ThreadLocal<SimpleDateFormat>() {
-
-        @Override
-        protected SimpleDateFormat initialValue() {
-            return new SimpleDateFormat("yyyyMMddHHmm");
-        }
-    };
-    private static final ThreadLocal<SimpleDateFormat> dateFormator4Transfer = new ThreadLocal<SimpleDateFormat>() {
-
-        @Override
-        protected SimpleDateFormat initialValue() {
-            return new SimpleDateFormat("yyyyMMddHHmmss");
-        }
-    };
-    private BaseSource source;
-    private final ChannelGroup allChannels;
-    private int maxConnections = Integer.MAX_VALUE;
-    private boolean filterEmptyMsg = false;
-    private final boolean isCompressed;
-    private final ChannelProcessor processor;
-    private final ServiceDecoder serviceProcessor;
-    private final String defaultTopic;
-    private String defaultMXAttr = "m=3";
-    private final String protocolType;
-    private final DataProxyMetricItemSet metricItemSet;
-
-    /**
-     * SimpleMessageHandler
-     * @param source
-     * @param serProcessor
-     * @param allChannels
-     * @param topic
-     * @param attr
-     * @param filterEmptyMsg
-     * @param maxMsgLength
-     * @param maxCons
-     * @param isCompressed
-     * @param protocolType
-     */
-    public SimpleMessageHandler(BaseSource source, ServiceDecoder serProcessor,
-            ChannelGroup allChannels,
-            String topic, String attr, Boolean filterEmptyMsg, Integer maxMsgLength,
-            Integer maxCons,
-            Boolean isCompressed, String protocolType) {
-        this.source = source;
-        this.processor = source.getChannelProcessor();
-        this.serviceProcessor = serProcessor;
-        this.allChannels = allChannels;
-        this.defaultTopic = topic;
-        if (null != attr) {
-            this.defaultMXAttr = attr;
-        }
-
-        this.filterEmptyMsg = filterEmptyMsg;
-        this.isCompressed = isCompressed;
-        this.maxConnections = maxCons;
-        this.protocolType = protocolType;
-        this.metricItemSet = source.getMetricItemSet();
-    }
-
-    private byte[] newBinMsg(byte[] orgBinMsg, String extraAttr) {
-        final int BIN_MSG_TOTALLEN_OFFSET = 0;
-        final int BIN_MSG_TOTALLEN_SIZE = 4;
-        final int BIN_MSG_BODYLEN_SIZE = 4;
-        final int BIN_MSG_EXTEND_OFFSET = 9;
-        final int BIN_MSG_BODYLEN_OFFSET = 21;
-        final int BIN_MSG_BODY_OFFSET = BIN_MSG_BODYLEN_SIZE + BIN_MSG_BODYLEN_OFFSET;
-        final int BIN_MSG_ATTRLEN_SIZE = 2;
-        final int BIN_MSG_FORMAT_SIZE = 29;
-        final int BIN_MSG_MAGIC_SIZE = 2;
-        final int BIN_MSG_MAGIC = 0xEE01;
-
-        ByteBuffer orgBuf = ByteBuffer.wrap(orgBinMsg);
-        int totalLen = orgBuf.getInt(BIN_MSG_TOTALLEN_OFFSET);
-        int dataLen = orgBuf.getInt(BIN_MSG_BODYLEN_OFFSET);
-        int attrLen = orgBuf.getShort(BIN_MSG_BODY_OFFSET + dataLen);
-
-        int newTotalLen = 0;
-        String strAttr;
-        if (attrLen != 0) {
-            newTotalLen = totalLen + extraAttr.length() + "&".length();
-            strAttr = "&" + extraAttr;
-        } else {
-            newTotalLen = totalLen + extraAttr.length();
-            strAttr = extraAttr;
-        }
-
-        ByteBuffer dataBuf = ByteBuffer.allocate(newTotalLen + BIN_MSG_TOTALLEN_SIZE);
-        dataBuf
-                .put(orgBuf.array(), 0, dataLen + (BIN_MSG_FORMAT_SIZE - BIN_MSG_MAGIC_SIZE) + attrLen);
-        dataBuf
-                .putShort(dataLen + (BIN_MSG_FORMAT_SIZE - BIN_MSG_ATTRLEN_SIZE - BIN_MSG_MAGIC_SIZE),
-                        (short) (strAttr.length() + attrLen));
-
-        System.arraycopy(strAttr.getBytes(StandardCharsets.UTF_8), 0, dataBuf.array(),
-                dataLen + (BIN_MSG_FORMAT_SIZE - BIN_MSG_MAGIC_SIZE) + attrLen,
-                strAttr.length());
-        int extendField = orgBuf.getShort(BIN_MSG_EXTEND_OFFSET);
-        dataBuf.putShort(BIN_MSG_EXTEND_OFFSET, (short) (extendField | 0x4));
-        dataBuf.putInt(0, newTotalLen);
-        dataBuf.putShort(newTotalLen + BIN_MSG_TOTALLEN_SIZE - BIN_MSG_MAGIC_SIZE,
-                (short) BIN_MSG_MAGIC);
-        return dataBuf.array();
-    }
-
-    @Override
-    public void channelActive(ChannelHandlerContext ctx) throws Exception {
-        // check max allowed connection count
-        if (allChannels.size() - 1 >= maxConnections) {
-            ctx.channel().disconnect();
-            ctx.channel().close();
-            logger.warn("{} refuse to connect = {} , connections = {}, maxConnections = {}",
-                    source.getName(), ctx.channel(), (allChannels.size() - 1), maxConnections);
-            return;
-        }
-        // check illegal ip
-        if (ConfigManager.getInstance().needChkIllegalIP()) {
-            String strRemoteIp = AddressUtils.getChannelRemoteIP(ctx.channel());
-            if (strRemoteIp != null
-                    && ConfigManager.getInstance().isIllegalIP(strRemoteIp)) {
-                ctx.channel().disconnect();
-                ctx.channel().close();
-                logger.error(strRemoteIp + " is Illegal IP, so refuse it !");
-                return;
-            }
-        }
-        // add legal channel
-        allChannels.add(ctx.channel());
-        ctx.fireChannelActive();
-        logger.info("{} added new channel, current connections = {}, maxConnections = {}",
-                source.getName(), (allChannels.size() - 1), maxConnections);
-    }
-
-    private void checkGroupIdInfo(ProxyMessage message, Map<String, String> commonAttrMap,
-            Map<String, String> attrMap, AtomicReference<String> topicInfo) {
-        String groupId = message.getGroupId();
-        String streamId = message.getStreamId();
-        if (null != groupId) {
-
-            String value = configManager.getTopicName(groupId, streamId);
-            if (StringUtils.isNotEmpty(value)) {
-                topicInfo.set(value.trim());
-            }
-        } else {
-            String num2name = commonAttrMap.get(AttrConstants.NUM2NAME);
-            String groupIdNum = commonAttrMap.get(AttrConstants.GROUPID_NUM);
-            String streamIdNum = commonAttrMap.get(AttrConstants.STREAMID_NUM);
-
-            if (!configManager.isGroupIdNumConfigEmpty()
-                    && !configManager.isStreamIdNumConfigEmpty()) {
-                groupId = configManager.getGroupIdNameByNum(groupIdNum);
-                streamId = configManager.getStreamIdNameByIdNum(groupIdNum, streamIdNum);
-                if (groupId != null && streamId != null) {
-                    if ((configManager.isEnableNum2NameTrans(groupIdNum)
-                            && "TRUE".equalsIgnoreCase(num2name))) {
-                        String extraAttr = "groupId=" + groupId + "&" + "streamId=" + streamId;
-                        message.setData(newBinMsg(message.getData(), extraAttr));
-                    }
-
-                    attrMap.put(AttributeConstants.GROUP_ID, groupId);
-                    attrMap.put(AttributeConstants.STREAM_ID, streamId);
-                    message.setGroupId(groupId);
-                    message.setStreamId(streamId);
-
-                    String value = configManager.getTopicName(groupId, streamId);
-                    if (StringUtils.isNotEmpty(value)) {
-                        topicInfo.set(value.trim());
-                    }
-                }
-            }
-        }
-    }
-
-    private void updateMsgList(List<ProxyMessage> msgList, Map<String, String> commonAttrMap,
-            Map<String, HashMap<String, List<ProxyMessage>>> messageMap,
-            String strRemoteIP, MsgType msgType) {
-        for (ProxyMessage message : msgList) {
-            Map<String, String> attrMap = message.getAttributeMap();
-
-            String topic = this.defaultTopic;
-
-            AtomicReference<String> topicInfo = new AtomicReference<>(topic);
-            checkGroupIdInfo(message, commonAttrMap, attrMap, topicInfo);
-            topic = topicInfo.get();
-
-            // if(groupId==null)groupId="b_test";//default groupId
-
-            message.setTopic(topic);
-            commonAttrMap.put(AttributeConstants.NODE_IP, strRemoteIP);
-
-            String groupId = message.getGroupId();
-            String streamId = message.getStreamId();
-
-            // whether sla
-            if (SLA_METRIC_GROUPID.equals(groupId)) {
-                commonAttrMap.put(SLA_METRIC_DATA, "true");
-                message.setTopic(SLA_METRIC_DATA);
-            }
-
-            if (!"pb".equals(attrMap.get(AttributeConstants.MESSAGE_TYPE))
-                    && !MsgType.MSG_MULTI_BODY.equals(msgType)
-                    && !MsgType.MSG_MULTI_BODY_ATTR.equals(msgType)) {
-                byte[] data = message.getData();
-                if (data[data.length - 1] == '\n') {
-                    int tripDataLen = data.length - 1;
-                    if (data[data.length - 2] == '\r') {
-                        tripDataLen = data.length - 2;
-                    }
-                    byte[] tripData = new byte[tripDataLen];
-                    System.arraycopy(data, 0, tripData, 0, tripDataLen);
-                    message.setData(tripData);
-                }
-            }
-
-            if (streamId == null) {
-                streamId = "";
-            }
-            HashMap<String, List<ProxyMessage>> streamIdMsgMap = messageMap
-                    .computeIfAbsent(topic, k -> new HashMap<>());
-            List<ProxyMessage> streamIdMsgList = streamIdMsgMap
-                    .computeIfAbsent(streamId, k -> new ArrayList<>());
-            streamIdMsgList.add(message);
-        }
-    }
-
-    /**
-     * formatMessagesAndSend
-     * 
-     * @param  commonAttrMap
-     * @param  messageMap
-     * @param  strRemoteIP
-     * @param  msgType
-     * @throws MessageIDException
-     */
-    private void formatMessagesAndSend(Map<String, String> commonAttrMap,
-            Map<String, HashMap<String, List<ProxyMessage>>> messageMap,
-            String strRemoteIP, MsgType msgType) throws MessageIDException {
-
-        int inLongMsgVer = 1;
-        if (MsgType.MSG_MULTI_BODY_ATTR.equals(msgType)) {
-            inLongMsgVer = 3;
-        } else if (MsgType.MSG_BIN_MULTI_BODY.equals(msgType)) {
-            inLongMsgVer = 4;
-        }
-
-        for (Map.Entry<String, HashMap<String, List<ProxyMessage>>> topicEntry : messageMap.entrySet()) {
-            for (Map.Entry<String, List<ProxyMessage>> streamIdEntry : topicEntry.getValue().entrySet()) {
-
-                InLongMsg inLongMsg = InLongMsg.newInLongMsg(this.isCompressed, inLongMsgVer);
-                Map<String, String> headers = new HashMap<String, String>();
-                for (ProxyMessage message : streamIdEntry.getValue()) {
-                    if (MsgType.MSG_MULTI_BODY_ATTR.equals(msgType) || MsgType.MSG_MULTI_BODY.equals(msgType)) {
-                        message.getAttributeMap().put(AttributeConstants.MESSAGE_COUNT, String.valueOf(1));
-                        inLongMsg.addMsg(mapJoiner.join(message.getAttributeMap()), message.getData());
-                    } else if (MsgType.MSG_BIN_MULTI_BODY.equals(msgType)) {
-                        inLongMsg.addMsg(message.getData());
-                    } else {
-                        inLongMsg.addMsg(mapJoiner.join(message.getAttributeMap()), message.getData());
-                    }
-                }
-
-                long pkgTimeInMillis = inLongMsg.getCreatetime();
-                String pkgTimeStr = dateFormator.get().format(pkgTimeInMillis);
-
-                if (inLongMsgVer == 4) {
-                    if (commonAttrMap.containsKey(ConfigConstants.PKG_TIME_KEY)) {
-                        pkgTimeStr = commonAttrMap.get(ConfigConstants.PKG_TIME_KEY);
-                    } else {
-                        pkgTimeStr = dateFormator.get().format(System.currentTimeMillis());
-                    }
-                }
-
-                long dtTime = NumberUtils.toLong(commonAttrMap.get(AttributeConstants.DATA_TIME),
-                        System.currentTimeMillis());
-                headers.put(AttributeConstants.DATA_TIME, String.valueOf(dtTime));
-
-                headers.put(ConfigConstants.TOPIC_KEY, topicEntry.getKey());
-                headers.put(AttributeConstants.STREAM_ID, streamIdEntry.getKey());
-                headers.put(ConfigConstants.REMOTE_IP_KEY, strRemoteIP);
-                headers.put(ConfigConstants.REMOTE_IDC_KEY, DEFAULT_REMOTE_IDC_VALUE);
-                // every message share the same msg cnt? what if msgType = 5
-                String proxyMetricMsgCnt = commonAttrMap.get(AttributeConstants.MESSAGE_COUNT);
-                headers.put(ConfigConstants.MSG_COUNTER_KEY, proxyMetricMsgCnt);
-
-                byte[] data = inLongMsg.buildArray();
-                headers.put(ConfigConstants.TOTAL_LEN, String.valueOf(data.length));
-
-                String sequenceId = commonAttrMap.get(AttributeConstants.SEQUENCE_ID);
-                if (StringUtils.isNotEmpty(sequenceId)) {
-                    StringBuilder sidBuilder =
-                            new StringBuilder().append(topicEntry.getKey())
-                                    .append(AttributeConstants.SEPARATOR).append(streamIdEntry.getKey())
-                                    .append(AttributeConstants.SEPARATOR).append(sequenceId);
-                    headers.put(ConfigConstants.SEQUENCE_ID, sidBuilder.toString());
-                }
-
-                headers.put(ConfigConstants.PKG_TIME_KEY, pkgTimeStr);
-
-                // process proxy message list
-                this.processProxyMessageList(headers, streamIdEntry.getValue());
-            }
-        }
-    }
-
-    /**
-     * processProxyMessageList
-     * 
-     * @param commonHeaders
-     * @param proxyMessages
-     */
-    private void processProxyMessageList(Map<String, String> commonHeaders,
-            List<ProxyMessage> proxyMessages) {
-        for (ProxyMessage message : proxyMessages) {
-            Event event = this.parseProxyMessage2Event(commonHeaders, message);
-            try {
-                processor.processEvent(event);
-                this.addMetric(true, event.getBody().length, event);
-            } catch (Throwable ex) {
-                logger.error("Error writting to channel,data will discard.", ex);
-                this.addMetric(false, event.getBody().length, event);
-                throw new ChannelException("ProcessEvent error can't write event to channel.");
-            }
-        }
-    }
-
-    /**
-     * parseProxyMessage2Event
-     * 
-     * @param  commonHeaders
-     * @param  proxyMessage
-     * @return
-     */
-    private Event parseProxyMessage2Event(Map<String, String> commonHeaders, ProxyMessage proxyMessage) {
-        Map<String, String> headers = new HashMap<>();
-        if (proxyMessage.getAttributeMap() != null) {
-            headers.putAll(proxyMessage.getAttributeMap());
-        }
-        headers.putAll(commonHeaders);
-        headers.put(AttributeConstants.MESSAGE_COUNT, "1");
-        headers.put(Constants.INLONG_GROUP_ID, proxyMessage.getGroupId());
-        headers.put(Constants.INLONG_STREAM_ID, proxyMessage.getStreamId());
-        headers.put(Constants.TOPIC, proxyMessage.getTopic());
-        headers.put(Constants.HEADER_KEY_MSG_TIME,
-                commonHeaders.get(AttributeConstants.DATA_TIME));
-        headers.put(Constants.HEADER_KEY_SOURCE_IP,
-                commonHeaders.get(AttributeConstants.NODE_IP));
-        headers.put(ConfigConstants.MSG_ENCODE_VER, DataProxyMsgEncType.MSG_ENCODE_TYPE_PB.getStrId());
-        headers.put(EventConstants.HEADER_KEY_VERSION, DataProxyMsgEncType.MSG_ENCODE_TYPE_PB.getStrId());
-        return EventBuilder.withBody(proxyMessage.getData(), headers);
-    }
-
-    private void responsePackage(Map<String, String> commonAttrMap,
-            Map<String, Object> resultMap,
-            Channel remoteChannel,
-            SocketAddress remoteSocketAddress,
-            MsgType msgType) throws Exception {
-        if (!commonAttrMap.containsKey("isAck") || "true".equals(commonAttrMap.get("isAck"))) {
-            if (MsgType.MSG_ACK_SERVICE.equals(msgType) || MsgType.MSG_ORIGINAL_RETURN
-                    .equals(msgType)
-                    || MsgType.MSG_MULTI_BODY.equals(msgType) || MsgType.MSG_MULTI_BODY_ATTR
-                            .equals(msgType)) {
-                byte[] backAttr = mapJoiner.join(commonAttrMap).getBytes(StandardCharsets.UTF_8);
-                byte[] backBody = null;
-
-                if (backAttr != null && !new String(backAttr, StandardCharsets.UTF_8).isEmpty()) {
-                    if (MsgType.MSG_ORIGINAL_RETURN.equals(msgType)) {
-
-                        backBody = (byte[]) resultMap.get(ConfigConstants.DECODER_BODY);
-                    } else {
-
-                        backBody = new byte[]{50};
-                    }
-                    int backTotalLen = 1 + 4 + backBody.length + 4 + backAttr.length;
-                    ByteBuf buffer = ByteBufAllocator.DEFAULT.buffer(4 + backTotalLen);
-                    buffer.writeInt(backTotalLen);
-                    buffer.writeByte(msgType.getValue());
-                    buffer.writeInt(backBody.length);
-                    buffer.writeBytes(backBody);
-                    buffer.writeInt(backAttr.length);
-                    buffer.writeBytes(backAttr);
-                    if (remoteChannel.isWritable()) {
-                        remoteChannel.write(buffer);
-                    } else {
-                        String backAttrStr = new String(backAttr, StandardCharsets.UTF_8);
-                        logger.warn(
-                                "the send buffer1 is full, so disconnect it!please check remote client"
-                                        + "; Connection info:"
-                                        + remoteChannel + ";attr is " + backAttrStr);
-                        buffer.release();
-                        throw new Exception(new Throwable(
-                                "the send buffer1 is full, so disconnect it!please check remote client"
-                                        +
-                                        "; Connection info:" + remoteChannel + ";attr is "
-                                        + backAttrStr));
-                    }
-                }
-            } else if (MsgType.MSG_BIN_MULTI_BODY.equals(msgType)) {
-                String backattrs = null;
-                if (resultMap.containsKey(ConfigConstants.DECODER_ATTRS)) {
-                    backattrs = (String) resultMap.get(ConfigConstants.DECODER_ATTRS);
-                }
-
-                int binTotalLen = 1 + 4 + 2 + 2;
-                if (null != backattrs) {
-                    binTotalLen += backattrs.length();
-                }
-
-                ByteBuf binBuffer = ByteBufAllocator.DEFAULT.buffer(4 + binTotalLen);
-                binBuffer.writeInt(binTotalLen);
-                binBuffer.writeByte(msgType.getValue());
-
-                long uniqVal = Long.parseLong(commonAttrMap.get(AttributeConstants.UNIQ_ID));
-                byte[] uniq = new byte[4];
-                uniq[0] = (byte) ((uniqVal >> 24) & 0xFF);
-                uniq[1] = (byte) ((uniqVal >> 16) & 0xFF);
-                uniq[2] = (byte) ((uniqVal >> 8) & 0xFF);
-                uniq[3] = (byte) (uniqVal & 0xFF);
-                binBuffer.writeBytes(uniq);
-
-                if (null != backattrs) {
-                    binBuffer.writeShort(backattrs.length());
-                    binBuffer.writeBytes(backattrs.getBytes(StandardCharsets.UTF_8));
-                } else {
-                    binBuffer.writeShort(0x0);
-                }
-
-                binBuffer.writeShort(0xee01);
-                if (remoteChannel.isWritable()) {
-                    remoteChannel.write(binBuffer);
-                } else {
-                    logger.warn(
-                            "the send buffer2 is full, so disconnect it!please check remote client"
-                                    + "; Connection info:" + remoteChannel + ";attr is "
-                                    + backattrs);
-                    binBuffer.release();
-                    throw new Exception(new Throwable(
-                            "the send buffer2 is full,so disconnect it!please check remote client, Connection info:"
-                                    + remoteChannel + ";attr is " + backattrs));
-                }
-            }
-        }
-    }
-
-    @Override
-    public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception {
-        logger.info("message received");
-        if (msg == null) {
-            logger.error("get null messageevent, just skip");
-            this.addMetric(false, 0, null);
-            return;
-        }
-        Channel remoteChannel = ctx.channel();
-        String strRemoteIP = AddressUtils.getChannelRemoteIP(remoteChannel);
-        ByteBuf cb = (ByteBuf) msg;
-        try {
-            int len = cb.readableBytes();
-            if (len == 0 && this.filterEmptyMsg) {
-                logger.warn("skip empty msg.");
-                cb.clear();
-                this.addMetric(false, 0, null);
-                return;
-            }
-            Map<String, Object> resultMap = null;
-            final long msgRcvTime = System.currentTimeMillis();
-            try {
-                resultMap = serviceProcessor.extractData(cb,
-                        strRemoteIP, msgRcvTime, remoteChannel);
-            } catch (MessageIDException ex) {
-                this.addMetric(false, 0, null);
-                throw new IOException(ex.getCause());
-            }
-
-            if (resultMap == null) {
-                logger.info("result is null");
-                this.addMetric(false, 0, null);
-                return;
-            }
-
-            MsgType msgType = (MsgType) resultMap.get(ConfigConstants.MSG_TYPE);
-            if (MsgType.MSG_HEARTBEAT.equals(msgType)) {
-                ByteBuf heartbeatBuffer = ByteBufAllocator.DEFAULT.buffer(5);
-                heartbeatBuffer.writeBytes(new byte[]{0, 0, 0, 1, 1});
-                remoteChannel.write(heartbeatBuffer);
-                this.addMetric(false, 0, null);
-                return;
-            }
-
-            if (MsgType.MSG_BIN_HEARTBEAT.equals(msgType)) {
-                this.addMetric(false, 0, null);
-                return;
-            }
-
-            Map<String, String> commonAttrMap = (Map<String, String>) resultMap.get(ConfigConstants.COMMON_ATTR_MAP);
-            if (commonAttrMap == null) {
-                commonAttrMap = new HashMap<String, String>();
-            }
-
-            List<ProxyMessage> msgList = (List<ProxyMessage>) resultMap.get(ConfigConstants.MSG_LIST);
-            if (msgList != null
-                    && !commonAttrMap.containsKey(ConfigConstants.FILE_CHECK_DATA)
-                    && !commonAttrMap.containsKey(ConfigConstants.MINUTE_CHECK_DATA)) {
-                Map<String, HashMap<String, List<ProxyMessage>>> messageMap = new HashMap<>(msgList.size());
-                updateMsgList(msgList, commonAttrMap, messageMap, strRemoteIP, msgType);
-
-                formatMessagesAndSend(commonAttrMap, messageMap, strRemoteIP, msgType);
-
-            } else if (msgList != null && commonAttrMap.containsKey(ConfigConstants.FILE_CHECK_DATA)) {
-                Map<String, String> headers = new HashMap<String, String>();
-                headers.put("msgtype", "filestatus");
-                headers.put(ConfigConstants.FILE_CHECK_DATA,
-                        "true");
-                for (ProxyMessage message : msgList) {
-                    byte[] body = message.getData();
-                    Event event = EventBuilder.withBody(body, headers);
-                    try {
-                        processor.processEvent(event);
-                        this.addMetric(true, body.length, event);
-                    } catch (Throwable ex) {
-                        logger.error("Error writing to controller,data will discard.", ex);
-                        this.addMetric(false, body.length, event);
-                        throw new ChannelException(
-                                "Process Controller Event error can't write event to channel.");
-                    }
-                }
-            } else if (msgList != null && commonAttrMap
-                    .containsKey(ConfigConstants.MINUTE_CHECK_DATA)) {
-                logger.info("i am in MINUTE_CHECK_DATA");
-                Map<String, String> headers = new HashMap<String, String>();
-                headers.put("msgtype", "measure");
-                headers.put(ConfigConstants.FILE_CHECK_DATA,
-                        "true");
-                for (ProxyMessage message : msgList) {
-                    byte[] body = message.getData();
-                    Event event = EventBuilder.withBody(body, headers);
-                    try {
-                        processor.processEvent(event);
-                        this.addMetric(true, body.length, event);
-                    } catch (Throwable ex) {
-                        logger.error("Error writing to controller,data will discard.", ex);
-                        this.addMetric(false, body.length, event);
-                        throw new ChannelException(
-                                "Process Controller Event error can't write event to channel.");
-                    }
-                }
-            }
-            SocketAddress remoteSocketAddress = remoteChannel.remoteAddress();
-            responsePackage(commonAttrMap, resultMap, remoteChannel, remoteSocketAddress, msgType);
-        } finally {
-            cb.release();
-        }
-    }
-
-    @Override
-    public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception {
-        logger.error("exception caught cause = {}", cause);
-        ctx.fireExceptionCaught(cause);
-        ctx.close();
-    }
-
-    @Override
-    public void channelInactive(ChannelHandlerContext ctx) throws Exception {
-        logger.error("channel inactive {}", ctx.channel());
-        ctx.fireChannelInactive();
-    }
-
-    /**
-     * addMetric
-     * 
-     * @param result
-     * @param size
-     * @param event
-     */
-    private void addMetric(boolean result, long size, Event event) {
-        Map<String, String> dimensions = new HashMap<>();
-        dimensions.put(DataProxyMetricItem.KEY_CLUSTER_ID, "DataProxy");
-        dimensions.put(DataProxyMetricItem.KEY_SOURCE_ID, source.getName());
-        dimensions.put(DataProxyMetricItem.KEY_SOURCE_DATA_ID, source.getName());
-        DataProxyMetricItem.fillInlongId(event, dimensions);
-        DataProxyMetricItem.fillAuditFormatTime(event, dimensions);
-        DataProxyMetricItem metricItem = this.metricItemSet.findMetricItem(dimensions);
-        if (result) {
-            metricItem.readSuccessCount.incrementAndGet();
-            metricItem.readSuccessSize.addAndGet(size);
-            AuditUtils.add(AuditUtils.AUDIT_ID_DATAPROXY_READ_SUCCESS, event);
-        } else {
-            metricItem.readFailCount.incrementAndGet();
-            metricItem.readFailSize.addAndGet(size);
-        }
-    }
-}
diff --git a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source/SimpleTcpSource.java b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source/SimpleTcpSource.java
index 9fea7c353e..2860bdf8e1 100644
--- a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source/SimpleTcpSource.java
+++ b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source/SimpleTcpSource.java
@@ -18,108 +18,100 @@
 package org.apache.inlong.dataproxy.source;
 
 import org.apache.inlong.dataproxy.config.ConfigManager;
-import org.apache.inlong.dataproxy.config.holder.ConfigUpdateCallback;
-import org.apache.inlong.dataproxy.consts.ConfigConstants;
-import org.apache.inlong.dataproxy.utils.AddressUtils;
+import org.apache.inlong.dataproxy.utils.ConfStringUtils;
 import org.apache.inlong.dataproxy.utils.EventLoopUtil;
 
 import com.google.common.base.Preconditions;
 import io.netty.bootstrap.ServerBootstrap;
 import io.netty.buffer.ByteBufAllocator;
-import io.netty.channel.Channel;
-import io.netty.channel.ChannelInitializer;
 import io.netty.channel.ChannelOption;
 import io.netty.util.concurrent.DefaultThreadFactory;
 import org.apache.flume.Context;
-import org.apache.flume.EventDrivenSource;
 import org.apache.flume.conf.Configurable;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.net.InetSocketAddress;
-import java.util.Iterator;
 
 /**
  * Simple tcp source
- *
  */
-public class SimpleTcpSource extends BaseSource
-        implements
-            Configurable,
-            ConfigUpdateCallback,
-            EventDrivenSource {
+public class SimpleTcpSource extends BaseSource implements Configurable {
 
     private static final Logger logger = LoggerFactory.getLogger(SimpleTcpSource.class);
 
-    private static int TRAFFIC_CLASS_TYPE_0 = 0;
-
-    private static int TRAFFIC_CLASS_TYPE_96 = 96;
-
-    private static int HIGH_WATER_MARK_DEFAULT_VALUE = 64 * 1024;
-
-    private boolean tcpNoDelay = true;
-
-    private boolean keepAlive = true;
-
-    private int highWaterMark;
-
-    private int trafficClass;
-
-    protected String topic;
-
     private ServerBootstrap bootstrap;
+    private boolean tcpNoDelay;
+    private boolean tcpKeepAlive;
+    private int highWaterMark;
+    private boolean enableBusyWait;
 
     public SimpleTcpSource() {
         super();
         ConfigManager.getInstance().regIPVisitConfigChgCallback(this);
     }
 
+    @Override
+    public void configure(Context context) {
+        logger.info("Source {} context is {}", getName(), context);
+        super.configure(context);
+        // get tcp no-delay parameter
+        this.tcpNoDelay = context.getBoolean(SourceConstants.SRCCXT_TCP_NO_DELAY,
+                SourceConstants.VAL_DEF_TCP_NO_DELAY);
+        // get tcp keep-alive parameter
+        this.tcpKeepAlive = context.getBoolean(SourceConstants.SRCCXT_TCP_KEEP_ALIVE,
+                SourceConstants.VAL_DEF_TCP_KEEP_ALIVE);
+        // get tcp enable busy-wait
+        this.enableBusyWait = context.getBoolean(SourceConstants.SRCCXT_TCP_ENABLE_BUSY_WAIT,
+                SourceConstants.VAL_DEF_TCP_ENABLE_BUSY_WAIT);
+        // get tcp high watermark
+        this.highWaterMark = ConfStringUtils.getIntValue(context,
+                SourceConstants.SRCCXT_TCP_HIGH_WATER_MARK, SourceConstants.VAL_DEF_TCP_HIGH_WATER_MARK);
+        Preconditions.checkArgument((this.highWaterMark >= SourceConstants.VAL_MIN_TCP_HIGH_WATER_MARK),
+                SourceConstants.VAL_DEF_TCP_HIGH_WATER_MARK + " must be >= "
+                        + SourceConstants.VAL_MIN_TCP_HIGH_WATER_MARK);
+    }
+
     @Override
     public synchronized void startSource() {
         logger.info("start " + this.getName());
-
-        logger.info("Set max workers : {} ;", maxThreads);
-
-        acceptorThreadFactory = new DefaultThreadFactory("tcpSource-nettyBoss-threadGroup");
-
-        this.acceptorGroup = EventLoopUtil.newEventLoopGroup(
-                acceptorThreads, false, acceptorThreadFactory);
-
-        this.workerGroup = EventLoopUtil
-                .newEventLoopGroup(maxThreads, enableBusyWait,
-                        new DefaultThreadFactory("tcpSource-nettyWorker-threadGroup"));
-
+        // build accept group
+        this.acceptorGroup = EventLoopUtil.newEventLoopGroup(maxAcceptThreads, enableBusyWait,
+                new DefaultThreadFactory(this.getName() + "-boss-group"));
+        // build worker group
+        this.workerGroup = EventLoopUtil.newEventLoopGroup(maxWorkerThreads, enableBusyWait,
+                new DefaultThreadFactory(this.getName() + "-worker-group"));
+        // init boostrap
         bootstrap = new ServerBootstrap();
-
+        if (conLinger >= 0) {
+            bootstrap.option(ChannelOption.SO_LINGER, conLinger);
+        }
+        bootstrap.option(ChannelOption.SO_BACKLOG, conBacklog);
+        bootstrap.option(ChannelOption.SO_REUSEADDR, reuseAddress);
         bootstrap.childOption(ChannelOption.ALLOCATOR, ByteBufAllocator.DEFAULT);
         bootstrap.childOption(ChannelOption.TCP_NODELAY, tcpNoDelay);
-        bootstrap.childOption(ChannelOption.SO_KEEPALIVE, keepAlive);
-        bootstrap.childOption(ChannelOption.SO_RCVBUF, receiveBufferSize);
-        bootstrap.childOption(ChannelOption.SO_SNDBUF, sendBufferSize);
-        // serverBootstrap.childOption("child.trafficClass", trafficClass);
+        bootstrap.childOption(ChannelOption.SO_KEEPALIVE, tcpKeepAlive);
+        bootstrap.childOption(ChannelOption.SO_RCVBUF, maxRcvBufferSize);
+        bootstrap.childOption(ChannelOption.SO_SNDBUF, maxSendBufferSize);
         bootstrap.childOption(ChannelOption.WRITE_BUFFER_HIGH_WATER_MARK, highWaterMark);
         bootstrap.channel(EventLoopUtil.getServerSocketChannelClass(workerGroup));
         EventLoopUtil.enableTriggeredMode(bootstrap);
         bootstrap.group(acceptorGroup, workerGroup);
-        logger.info("load msgFactory=" + msgFactoryName
-                + " and serviceDecoderName=" + serviceDecoderName);
-
-        ChannelInitializer fac = this.getChannelInitializerFactory();
-        bootstrap.childHandler(fac);
+        bootstrap.childHandler(this.getChannelInitializerFactory());
         try {
-            if (host == null) {
-                channelFuture = bootstrap.bind(new InetSocketAddress(port)).sync();
+            if (srcHost == null) {
+                channelFuture = bootstrap.bind(new InetSocketAddress(srcPort)).sync();
             } else {
-                channelFuture = bootstrap.bind(new InetSocketAddress(host, port)).sync();
+                channelFuture = bootstrap.bind(new InetSocketAddress(srcHost, srcPort)).sync();
             }
         } catch (Exception e) {
-            logger.error("Simple TCP Source error bind host {} port {},program will exit! e = {}",
-                    host, port, e);
+            logger.error("Source {} bind ({}:{}) error, program will exit! e = {}",
+                    this.getName(), srcHost, srcPort, e);
             System.exit(-1);
         }
         ConfigManager.getInstance().addSourceReportInfo(
-                host, String.valueOf(port), getProtocolName().toUpperCase());
-        logger.info("Simple TCP Source started at host {}, port {}", host, port);
+                srcHost, String.valueOf(srcPort), getProtocolName().toUpperCase());
+        logger.info("Source {} started at ({}:{})!", this.getName(), srcHost, srcPort);
     }
 
     @Override
@@ -127,71 +119,9 @@ public class SimpleTcpSource extends BaseSource
         super.stop();
     }
 
-    @Override
-    public void configure(Context context) {
-        logger.info("context is {}", context);
-        super.configure(context);
-        tcpNoDelay = context.getBoolean(ConfigConstants.TCP_NO_DELAY, true);
-        keepAlive = context.getBoolean(ConfigConstants.KEEP_ALIVE, true);
-        highWaterMark = context.getInteger(ConfigConstants.HIGH_WATER_MARK, HIGH_WATER_MARK_DEFAULT_VALUE);
-        receiveBufferSize = context.getInteger(ConfigConstants.RECEIVE_BUFFER_SIZE, RECEIVE_BUFFER_DEFAULT_SIZE);
-        if (receiveBufferSize > RECEIVE_BUFFER_MAX_SIZE) {
-            receiveBufferSize = RECEIVE_BUFFER_MAX_SIZE;
-        }
-        Preconditions.checkArgument(receiveBufferSize > BUFFER_SIZE_MUST_THAN,
-                "receiveBufferSize must be > 0");
-
-        sendBufferSize = context.getInteger(ConfigConstants.SEND_BUFFER_SIZE, SEND_BUFFER_DEFAULT_SIZE);
-        if (sendBufferSize > SEND_BUFFER_MAX_SIZE) {
-            sendBufferSize = SEND_BUFFER_MAX_SIZE;
-        }
-        Preconditions.checkArgument(sendBufferSize > BUFFER_SIZE_MUST_THAN,
-                "sendBufferSize must be > 0");
-
-        trafficClass = context.getInteger(ConfigConstants.TRAFFIC_CLASS, TRAFFIC_CLASS_TYPE_0);
-        Preconditions.checkArgument((trafficClass == TRAFFIC_CLASS_TYPE_0
-                || trafficClass == TRAFFIC_CLASS_TYPE_96),
-                "trafficClass must be == 0 or == 96");
-
-        try {
-            maxThreads = context.getInteger(ConfigConstants.MAX_THREADS, 32);
-        } catch (NumberFormatException e) {
-            logger.warn("Simple TCP Source max-threads property must specify an integer value. {}",
-                    context.getString(ConfigConstants.MAX_THREADS));
-        }
-    }
-
     @Override
     public String getProtocolName() {
-        return "tcp";
-    }
-
-    @Override
-    public void update() {
-        // check current all links
-        if (ConfigManager.getInstance().needChkIllegalIP()) {
-            int cnt = 0;
-            Channel channel;
-            String strRemoteIP;
-            long startTime = System.currentTimeMillis();
-            Iterator<Channel> iterator = allChannels.iterator();
-            while (iterator.hasNext()) {
-                channel = iterator.next();
-                strRemoteIP = AddressUtils.getChannelRemoteIP(channel);
-                if (strRemoteIP == null) {
-                    continue;
-                }
-                if (ConfigManager.getInstance().isIllegalIP(strRemoteIP)) {
-                    channel.disconnect();
-                    channel.close();
-                    allChannels.remove(channel);
-                    cnt++;
-                    logger.error(strRemoteIP + " is Illegal IP, so disconnect it !");
-                }
-            }
-            logger.info("Channel check, {} disconnects {} Illegal channels, waist {} ms",
-                    getName(), cnt, (System.currentTimeMillis() - startTime));
-        }
+        return SourceConstants.SRC_PROTOCOL_TYPE_TCP;
     }
 
 }
diff --git a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source/SimpleUdpSource.java b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source/SimpleUdpSource.java
index 67212660f4..eb9dd70220 100644
--- a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source/SimpleUdpSource.java
+++ b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source/SimpleUdpSource.java
@@ -18,66 +18,63 @@
 package org.apache.inlong.dataproxy.source;
 
 import org.apache.inlong.dataproxy.config.ConfigManager;
-import org.apache.inlong.dataproxy.consts.ConfigConstants;
 
 import io.netty.bootstrap.Bootstrap;
-import io.netty.channel.ChannelInitializer;
 import io.netty.channel.ChannelOption;
 import io.netty.channel.socket.nio.NioDatagramChannel;
 import org.apache.flume.Context;
-import org.apache.flume.EventDrivenSource;
 import org.apache.flume.conf.Configurable;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.net.InetSocketAddress;
 
-public class SimpleUdpSource
-        extends
-            BaseSource
-        implements
-            EventDrivenSource,
-            Configurable {
+public class SimpleUdpSource extends BaseSource implements Configurable {
 
     private static final Logger logger = LoggerFactory
             .getLogger(SimpleUdpSource.class);
 
-    private static int UPD_BUFFER_DEFAULT_SIZE = 8192;
-
     private Bootstrap bootstrap;
 
     public SimpleUdpSource() {
         super();
     }
 
+    @Override
+    public void configure(Context context) {
+        logger.info("Source {} context is {}", getName(), context);
+        super.configure(context);
+    }
+
     @SuppressWarnings({"unchecked", "rawtypes"})
     @Override
     public void startSource() {
         // setup Netty server
         logger.info("start " + this.getName());
         bootstrap = new Bootstrap();
-        logger.info("Set max workers : {} ;", maxThreads);
         bootstrap.channel(NioDatagramChannel.class);
-        bootstrap.option(ChannelOption.SO_RCVBUF, receiveBufferSize);
-        bootstrap.option(ChannelOption.SO_SNDBUF, sendBufferSize);
-        ChannelInitializer fac = this.getChannelInitializerFactory();
-        bootstrap.handler(fac);
+        if (conLinger >= 0) {
+            bootstrap.option(ChannelOption.SO_LINGER, conLinger);
+        }
+        bootstrap.option(ChannelOption.SO_BACKLOG, conBacklog);
+        bootstrap.option(ChannelOption.SO_REUSEADDR, reuseAddress);
+        bootstrap.option(ChannelOption.SO_RCVBUF, maxRcvBufferSize);
+        bootstrap.option(ChannelOption.SO_SNDBUF, maxSendBufferSize);
+        bootstrap.handler(this.getChannelInitializerFactory());
         try {
-            if (host == null) {
-                channelFuture = bootstrap.bind(new InetSocketAddress(port)).sync();
+            if (srcHost == null) {
+                channelFuture = bootstrap.bind(new InetSocketAddress(srcPort)).sync();
             } else {
-
-                channelFuture = bootstrap.bind(new InetSocketAddress(host, port)).sync();
+                channelFuture = bootstrap.bind(new InetSocketAddress(srcHost, srcPort)).sync();
             }
         } catch (Exception e) {
-            logger.error("Simple UDP Source error bind host {} port {}, program will exit!",
-                    new Object[]{host, port});
+            logger.error("Source {} bind ({}:{}) error, program will exit! e = {}",
+                    this.getName(), srcHost, srcPort, e);
             System.exit(-1);
-            // throw new FlumeException(e.getMessage());
         }
         ConfigManager.getInstance().addSourceReportInfo(
-                host, String.valueOf(port), getProtocolName().toUpperCase());
-        logger.info("Simple UDP Source started at host {}, port {}", host, port);
+                srcHost, String.valueOf(srcPort), getProtocolName().toUpperCase());
+        logger.info("Source {} started at ({}:{})!", this.getName(), srcHost, srcPort);
     }
 
     @Override
@@ -85,19 +82,8 @@ public class SimpleUdpSource
         super.stop();
     }
 
-    @Override
-    public void configure(Context context) {
-        super.configure(context);
-        try {
-            maxThreads = context.getInteger(ConfigConstants.MAX_THREADS, 32);
-        } catch (NumberFormatException e) {
-            logger.warn("Simple UDP Source max-threads property must specify an integer value.",
-                    context.getString(ConfigConstants.MAX_THREADS));
-        }
-    }
-
     @Override
     public String getProtocolName() {
-        return "udp";
+        return SourceConstants.SRC_PROTOCOL_TYPE_UDP;
     }
 }
diff --git a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source2/SourceConstants.java b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source/SourceConstants.java
similarity index 99%
rename from inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source2/SourceConstants.java
rename to inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source/SourceConstants.java
index c47e6d5cac..98cf98b677 100644
--- a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source2/SourceConstants.java
+++ b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source/SourceConstants.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.inlong.dataproxy.source2;
+package org.apache.inlong.dataproxy.source;
 
 public class SourceConstants {
 
diff --git a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source/SourceContext.java b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source/SourceContext.java
deleted file mode 100644
index 11f9556e62..0000000000
--- a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source/SourceContext.java
+++ /dev/null
@@ -1,270 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.inlong.dataproxy.source;
-
-import org.apache.inlong.common.metric.MetricRegister;
-import org.apache.inlong.dataproxy.config.CommonConfigHolder;
-import org.apache.inlong.dataproxy.consts.ConfigConstants;
-import org.apache.inlong.dataproxy.metrics.DataProxyMetricItemSet;
-
-import com.google.common.base.Preconditions;
-import io.netty.channel.group.ChannelGroup;
-import org.apache.commons.lang3.math.NumberUtils;
-import org.apache.flume.Context;
-import org.apache.flume.source.AbstractSource;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.Map;
-
-/**
- * 
- * SinkContext
- */
-public class SourceContext {
-
-    public static final Logger LOG = LoggerFactory.getLogger(SourceContext.class);
-
-    public static final String KEY_MAX_THREADS = "maxThreads";
-    public static final String KEY_PROCESSINTERVAL = "processInterval";
-    public static final String KEY_RELOADINTERVAL = "reloadInterval";
-    public static final String CONNECTIONS = "connections";
-    public static final String INLONG_HOST_IP = "inlongHostIp";
-    public static final String INLONG_HOST_PORT = "inlongHostPort";
-
-    protected AbstractSource source;
-    protected ChannelGroup allChannels;
-    protected String proxyClusterId;
-    protected String sourceId;
-    protected String sourceDataId;
-    // config
-    protected String hostIp;
-    protected int hostPort;
-    protected int maxThreads = 32;
-    protected int maxConnections = Integer.MAX_VALUE;
-    protected int maxMsgLength;
-    // metric
-    protected DataProxyMetricItemSet metricItemSet;
-    // reload
-    protected Context parentContext;
-    protected long reloadInterval;
-    // isRejectService
-    protected boolean isRejectService = false;
-
-    /**
-     * Constructor
-     * 
-     * @param source
-     * @param allChannels
-     * @param context
-     */
-    public SourceContext(AbstractSource source, ChannelGroup allChannels, Context context) {
-        this.source = source;
-        this.allChannels = allChannels;
-        this.proxyClusterId = CommonConfigHolder.getInstance().getClusterName();
-        this.sourceId = source.getName();
-        // metric
-        this.metricItemSet = new DataProxyMetricItemSet(sourceId);
-        MetricRegister.register(metricItemSet);
-        // config
-        this.maxConnections = context.getInteger(CONNECTIONS, 5000);
-        this.maxThreads = context.getInteger(KEY_MAX_THREADS, 32);
-        this.maxMsgLength = context.getInteger(ConfigConstants.MAX_MSG_LENGTH, 1024 * 64);
-        Preconditions.checkArgument(
-                (maxMsgLength >= 4 && maxMsgLength <= ConfigConstants.MSG_MAX_LENGTH_BYTES),
-                "maxMsgLength must be >= 4 and <= " + ConfigConstants.MSG_MAX_LENGTH_BYTES);
-        // port
-        this.hostIp = this.getHostIp(context);
-        this.hostPort = this.getHostPort(context);
-        this.sourceDataId = String.valueOf(hostPort);
-        //
-        this.parentContext = context;
-        this.reloadInterval = context.getLong(KEY_RELOADINTERVAL, 60000L);
-    }
-
-    /**
-     * getHostIp
-     * 
-     * @param  context
-     * @return
-     */
-    private String getHostIp(Context context) {
-        Map<String, String> envMap = System.getenv();
-        if (envMap.containsKey(INLONG_HOST_IP)) {
-            String hostIp = envMap.get(INLONG_HOST_IP);
-            return hostIp;
-        }
-        return context.getString(INLONG_HOST_IP);
-    }
-
-    /**
-     * getHostPort
-     * 
-     * @param  context
-     * @return
-     */
-    private int getHostPort(Context context) {
-        Map<String, String> envMap = System.getenv();
-        if (envMap.containsKey(INLONG_HOST_PORT)) {
-            String strPort = envMap.get(INLONG_HOST_PORT);
-            return NumberUtils.toInt(strPort, 0);
-        }
-        return context.getInteger(INLONG_HOST_IP, 0);
-    }
-
-    /**
-     * start
-     */
-    public void start() {
-
-    }
-
-    /**
-     * close
-     */
-    public void close() {
-
-    }
-
-    /**
-     * get sourceDataId
-     * 
-     * @return the sourceDataId
-     */
-    public String getSourceDataId() {
-        return sourceDataId;
-    }
-
-    /**
-     * set sourceDataId
-     * 
-     * @param sourceDataId the sourceDataId to set
-     */
-    public void setSourceDataId(String sourceDataId) {
-        this.sourceDataId = sourceDataId;
-    }
-
-    /**
-     * get source
-     * 
-     * @return the source
-     */
-    public AbstractSource getSource() {
-        return source;
-    }
-
-    /**
-     * get allChannels
-     * 
-     * @return the allChannels
-     */
-    public ChannelGroup getAllChannels() {
-        return allChannels;
-    }
-
-    /**
-     * get proxyClusterId
-     * 
-     * @return the proxyClusterId
-     */
-    public String getProxyClusterId() {
-        return proxyClusterId;
-    }
-
-    /**
-     * get sourceId
-     * 
-     * @return the sourceId
-     */
-    public String getSourceId() {
-        return sourceId;
-    }
-
-    /**
-     * get maxConnections
-     * 
-     * @return the maxConnections
-     */
-    public int getMaxConnections() {
-        return maxConnections;
-    }
-
-    /**
-     * get maxMsgLength
-     * 
-     * @return the maxMsgLength
-     */
-    public int getMaxMsgLength() {
-        return maxMsgLength;
-    }
-
-    /**
-     * get metricItemSet
-     * 
-     * @return the metricItemSet
-     */
-    public DataProxyMetricItemSet getMetricItemSet() {
-        return metricItemSet;
-    }
-
-    /**
-     * get parentContext
-     * 
-     * @return the parentContext
-     */
-    public Context getParentContext() {
-        return parentContext;
-    }
-
-    /**
-     * get reloadInterval
-     * 
-     * @return the reloadInterval
-     */
-    public long getReloadInterval() {
-        return reloadInterval;
-    }
-
-    /**
-     * get maxThreads
-     * 
-     * @return the maxThreads
-     */
-    public int getMaxThreads() {
-        return maxThreads;
-    }
-
-    /**
-     * isRejectService
-     * 
-     * @return
-     */
-    public boolean isRejectService() {
-        return isRejectService;
-    }
-
-    /**
-     * setRejectService
-     * 
-     * @param isRejectService
-     */
-    public void setRejectService(boolean isRejectService) {
-        this.isRejectService = isRejectService;
-    }
-
-}
diff --git a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source2/httpMsg/InLongHttpMsgHandler.java b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source/httpMsg/HttpMessageHandler.java
similarity index 98%
rename from inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source2/httpMsg/InLongHttpMsgHandler.java
rename to inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source/httpMsg/HttpMessageHandler.java
index e17b35228b..94245e2628 100644
--- a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source2/httpMsg/InLongHttpMsgHandler.java
+++ b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source/httpMsg/HttpMessageHandler.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.inlong.dataproxy.source2.httpMsg;
+package org.apache.inlong.dataproxy.source.httpMsg;
 
 import org.apache.inlong.common.enums.DataProxyErrCode;
 import org.apache.inlong.common.enums.DataProxyMsgEncType;
@@ -27,7 +27,7 @@ import org.apache.inlong.dataproxy.consts.AttrConstants;
 import org.apache.inlong.dataproxy.consts.ConfigConstants;
 import org.apache.inlong.dataproxy.consts.HttpAttrConst;
 import org.apache.inlong.dataproxy.consts.StatConstants;
-import org.apache.inlong.dataproxy.source2.BaseSource;
+import org.apache.inlong.dataproxy.source.BaseSource;
 import org.apache.inlong.dataproxy.utils.AddressUtils;
 import org.apache.inlong.dataproxy.utils.DateTimeUtils;
 import org.apache.inlong.sdk.commons.protocol.EventConstants;
@@ -68,9 +68,9 @@ import static io.netty.handler.codec.http.HttpUtil.is100ContinueExpected;
 /**
  * HTTP Server message handler
  */
-public class InLongHttpMsgHandler extends SimpleChannelInboundHandler<FullHttpRequest> {
+public class HttpMessageHandler extends SimpleChannelInboundHandler<FullHttpRequest> {
 
-    private static final Logger logger = LoggerFactory.getLogger(InLongHttpMsgHandler.class);
+    private static final Logger logger = LoggerFactory.getLogger(HttpMessageHandler.class);
     // log print count
     private static final LogCounter logCounter = new LogCounter(10, 100000, 30 * 1000);
     private final BaseSource source;
@@ -80,7 +80,7 @@ public class InLongHttpMsgHandler extends SimpleChannelInboundHandler<FullHttpRe
      *
      * @param source AbstractSource
      */
-    public InLongHttpMsgHandler(BaseSource source) {
+    public HttpMessageHandler(BaseSource source) {
         this.source = source;
     }
 
diff --git a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source/tcp/InlongTcpChannelHandler.java b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source/tcp/InlongTcpChannelHandler.java
deleted file mode 100644
index 44e404dfdd..0000000000
--- a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source/tcp/InlongTcpChannelHandler.java
+++ /dev/null
@@ -1,358 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.inlong.dataproxy.source.tcp;
-
-import org.apache.inlong.dataproxy.config.CommonConfigHolder;
-import org.apache.inlong.dataproxy.config.ConfigManager;
-import org.apache.inlong.dataproxy.metrics.DataProxyMetricItem;
-import org.apache.inlong.dataproxy.metrics.audit.AuditUtils;
-import org.apache.inlong.dataproxy.source.SourceContext;
-import org.apache.inlong.dataproxy.utils.AddressUtils;
-import org.apache.inlong.sdk.commons.protocol.EventUtils;
-import org.apache.inlong.sdk.commons.protocol.ProxyEvent;
-import org.apache.inlong.sdk.commons.protocol.ProxyPackEvent;
-import org.apache.inlong.sdk.commons.protocol.ProxySdk.MessagePack;
-import org.apache.inlong.sdk.commons.protocol.ProxySdk.MessagePackHeader;
-import org.apache.inlong.sdk.commons.protocol.ProxySdk.ResponseInfo;
-import org.apache.inlong.sdk.commons.protocol.ProxySdk.ResultCode;
-
-import io.netty.buffer.ByteBuf;
-import io.netty.buffer.Unpooled;
-import io.netty.channel.Channel;
-import io.netty.channel.ChannelHandlerContext;
-import io.netty.channel.ChannelInboundHandlerAdapter;
-import org.apache.flume.Event;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.TimeUnit;
-
-/**
- * InlongTcpChannelHandler
- */
-public class InlongTcpChannelHandler extends ChannelInboundHandlerAdapter {
-
-    public static final Logger LOG = LoggerFactory.getLogger(InlongTcpChannelHandler.class);
-    public static final int LENGTH_PARAM_OFFSET = 0;
-    public static final int LENGTH_PARAM_LENGTH = 4;
-    public static final int VERSION_PARAM_OFFSET = 4;
-    public static final int VERSION_PARAM_LENGTH = 2;
-    public static final int BODY_PARAM_OFFSET = 6;
-
-    public static final int VERSION_1 = 1;
-
-    private SourceContext sourceContext;
-
-    /**
-     * Constructor
-     * 
-     * @param sourceContext
-     */
-    public InlongTcpChannelHandler(SourceContext sourceContext) {
-        this.sourceContext = sourceContext;
-    }
-
-    /**
-     * channelRead
-     * 
-     * @param  ctx
-     * @param  msg
-     * @throws Exception
-     */
-    @Override
-    public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception {
-        LOG.debug("message received");
-        if (msg == null) {
-            LOG.error("get null msg, just skip");
-            this.addMetric(false, 0, null);
-            return;
-        }
-        ByteBuf cb = (ByteBuf) msg;
-        try {
-            int readableLength = cb.readableBytes();
-            if (readableLength == 0) {
-                LOG.warn("skip empty msg.");
-                cb.clear();
-                this.addMetric(false, 0, null);
-                return;
-            }
-            if (readableLength > LENGTH_PARAM_LENGTH + VERSION_PARAM_LENGTH + sourceContext.getMaxMsgLength()) {
-                this.addMetric(false, 0, null);
-                throw new Exception("err msg, MSG_MAX_LENGTH_BYTES "
-                        + "< readableLength, and readableLength=" + readableLength + ", and MSG_MAX_LENGTH_BYTES="
-                        + sourceContext.getMaxMsgLength());
-            }
-            // save index, reset it if buffer is not satisfied.
-            cb.markReaderIndex();
-            int totalPackLength = cb.readInt();
-            if (readableLength < totalPackLength + LENGTH_PARAM_LENGTH) {
-                // reset index.
-                cb.resetReaderIndex();
-                this.addMetric(false, 0, null);
-                throw new Exception("err msg, channel buffer is not satisfied, and  readableLength="
-                        + readableLength + ", and totalPackLength=" + totalPackLength);
-            }
-
-            // read version
-            int version = cb.readShort();
-            switch (version) {
-                case VERSION_1:
-                    // decode version 1
-                    int bodyLength = totalPackLength - VERSION_PARAM_LENGTH;
-                    decodeVersion1(ctx, cb, bodyLength);
-                    break;
-                default:
-                    this.addMetric(false, 0, null);
-                    throw new Exception("err version, unknown version:" + version);
-            }
-        } finally {
-            cb.release();
-        }
-    }
-
-    private void decodeVersion1(ChannelHandlerContext ctx, ByteBuf cb, int bodyLength) throws Exception {
-        // read bytes
-        byte[] msgBytes = new byte[bodyLength];
-        cb.readBytes(msgBytes);
-        // decode
-        MessagePack packObject = MessagePack.parseFrom(msgBytes);
-        // reject service
-        if (sourceContext.isRejectService()) {
-            this.addMetric(false, 0, null);
-            this.responsePackage(ctx, ResultCode.ERR_REJECT, packObject);
-            return;
-        }
-        // uncompress
-        List<ProxyEvent> events = EventUtils.decodeSdkPack(packObject);
-        // response success if event size is zero
-        if (events.size() == 0) {
-            this.responsePackage(ctx, ResultCode.SUCCUSS, packObject);
-        }
-        // process
-        if (CommonConfigHolder.getInstance().isResponseAfterSave()) {
-            this.processAndWaitingSave(ctx, packObject, events);
-        } else {
-            this.processAndResponse(ctx, packObject, events);
-        }
-    }
-
-    /**
-     * processAndWaitingSave
-     * @param ctx
-     * @param packObject
-     * @param events
-     * @throws Exception
-     */
-    private void processAndWaitingSave(ChannelHandlerContext ctx, MessagePack packObject, List<ProxyEvent> events)
-            throws Exception {
-        MessagePackHeader header = packObject.getHeader();
-        InlongTcpSourceCallback callback = new InlongTcpSourceCallback(ctx, header);
-        String inlongGroupId = header.getInlongGroupId();
-        String inlongStreamId = header.getInlongStreamId();
-        ProxyPackEvent packEvent = new ProxyPackEvent(inlongGroupId, inlongStreamId, events, callback);
-        // put to channel
-        try {
-            sourceContext.getSource().getChannelProcessor().processEvent(packEvent);
-            events.forEach(event -> {
-                this.addMetric(true, event.getBody().length, event);
-            });
-            boolean awaitResult = callback.getLatch().await(
-                    CommonConfigHolder.getInstance().getMaxResAfterSaveTimeout(), TimeUnit.MILLISECONDS);
-            if (!awaitResult) {
-                if (!callback.getHasResponsed().getAndSet(true)) {
-                    this.responsePackage(ctx, ResultCode.ERR_REJECT, packObject);
-                }
-            }
-        } catch (Throwable ex) {
-            LOG.error("Process Controller Event error can't write event to channel.", ex);
-            events.forEach(event -> {
-                this.addMetric(false, event.getBody().length, event);
-            });
-            if (!callback.getHasResponsed().getAndSet(true)) {
-                this.responsePackage(ctx, ResultCode.ERR_REJECT, packObject);
-            }
-        }
-    }
-
-    /**
-     * processAndResponse
-     * @param ctx
-     * @param packObject
-     * @param events
-     * @throws Exception
-     */
-    private void processAndResponse(ChannelHandlerContext ctx, MessagePack packObject, List<ProxyEvent> events)
-            throws Exception {
-        for (ProxyEvent event : events) {
-            String topic = ConfigManager.getInstance().getTopicName(
-                    event.getInlongGroupId(), event.getInlongStreamId());
-            if (topic != null) {
-                event.setTopic(topic);
-            }
-            // put to channel
-            try {
-                sourceContext.getSource().getChannelProcessor().processEvent(event);
-                this.addMetric(true, event.getBody().length, event);
-            } catch (Throwable ex) {
-                LOG.error("Process Controller Event error can't write event to channel.", ex);
-                this.addMetric(false, event.getBody().length, event);
-                this.responsePackage(ctx, ResultCode.ERR_REJECT, packObject);
-                return;
-            }
-        }
-        this.responsePackage(ctx, ResultCode.SUCCUSS, packObject);
-    }
-
-    /**
-     * addMetric
-     * 
-     * @param result
-     * @param size
-     * @param event
-     */
-    private void addMetric(boolean result, long size, Event event) {
-        Map<String, String> dimensions = new HashMap<>();
-        dimensions.put(DataProxyMetricItem.KEY_CLUSTER_ID, sourceContext.getProxyClusterId());
-        dimensions.put(DataProxyMetricItem.KEY_SOURCE_ID, sourceContext.getSourceId());
-        dimensions.put(DataProxyMetricItem.KEY_SOURCE_DATA_ID, sourceContext.getSourceDataId());
-        DataProxyMetricItem.fillInlongId(event, dimensions);
-        DataProxyMetricItem.fillAuditFormatTime(event, dimensions);
-        DataProxyMetricItem metricItem = this.sourceContext.getMetricItemSet().findMetricItem(dimensions);
-        if (result) {
-            metricItem.readSuccessCount.incrementAndGet();
-            metricItem.readSuccessSize.addAndGet(size);
-            AuditUtils.add(AuditUtils.AUDIT_ID_DATAPROXY_READ_SUCCESS, event);
-        } else {
-            metricItem.readFailCount.incrementAndGet();
-            metricItem.readFailSize.addAndGet(size);
-        }
-    }
-
-    /**
-     * responsePackage
-     *
-     * @param  ctx
-     * @param  code
-     * @throws Exception
-     */
-    private void responsePackage(ChannelHandlerContext ctx, ResultCode code, MessagePack packObject)
-            throws Exception {
-        ResponseInfo.Builder builder = ResponseInfo.newBuilder();
-        builder.setResult(code);
-        MessagePackHeader header = packObject.getHeader();
-        builder.setPackId(header.getPackId());
-
-        // encode
-        byte[] responseBytes = builder.build().toByteArray();
-        //
-        ByteBuf buffer = Unpooled.wrappedBuffer(responseBytes);
-        Channel remoteChannel = ctx.channel();
-        if (remoteChannel.isWritable()) {
-            remoteChannel.write(buffer);
-        } else {
-            LOG.warn(
-                    "the send buffer2 is full, so disconnect it!please check remote client"
-                            + "; Connection info:" + remoteChannel);
-            buffer.release();
-            throw new Exception(
-                    "the send buffer2 is full,so disconnect it!please check remote client, Connection info:"
-                            + remoteChannel);
-        }
-    }
-
-    /**
-     * exceptionCaught
-     * 
-     * @param  ctx
-     * @param  cause
-     * @throws Exception
-     */
-    @Override
-    public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) {
-        LOG.error("exception caught cause = {}", cause);
-        ctx.fireExceptionCaught(cause);
-        if (ctx.channel() != null) {
-            try {
-                ctx.channel().disconnect();
-                ctx.channel().close();
-            } catch (Exception ex) {
-                LOG.error("Close connection error!", ex);
-            }
-            sourceContext.getAllChannels().remove(ctx.channel());
-        }
-    }
-
-    /**
-     * channelInactive
-     *
-     * @param  ctx
-     * @throws Exception
-     */
-    @Override
-    public void channelInactive(ChannelHandlerContext ctx) {
-        LOG.debug("Connection to {} disconnected.", ctx.channel());
-        ctx.fireChannelInactive();
-        try {
-            ctx.channel().disconnect();
-            ctx.channel().close();
-        } catch (Exception ex) {
-            LOG.error("channelInactive has exception e = {}", ex);
-        }
-        sourceContext.getAllChannels().remove(ctx.channel());
-
-    }
-
-    /**
-     * channelActive
-     *
-     * @param  ctx
-     * @throws Exception
-     */
-    @Override
-    public void channelActive(ChannelHandlerContext ctx) {
-        // check max allowed connection count
-        if (sourceContext.getAllChannels().size() - 1 >= sourceContext.getMaxConnections()) {
-            ctx.channel().disconnect();
-            ctx.channel().close();
-            LOG.warn("{} refuse to connect = {} , connections = {}, maxConnections = {}",
-                    sourceContext.getSource().getName(), ctx.channel(),
-                    (sourceContext.getAllChannels().size() - 1), sourceContext.getMaxConnections());
-            return;
-        }
-        // check illegal ip
-        if (ConfigManager.getInstance().needChkIllegalIP()) {
-            String strRemoteIp = AddressUtils.getChannelRemoteIP(ctx.channel());
-            if (strRemoteIp != null
-                    && ConfigManager.getInstance().isIllegalIP(strRemoteIp)) {
-                ctx.channel().disconnect();
-                ctx.channel().close();
-                LOG.error(strRemoteIp + " is Illegal IP, so refuse it !");
-                return;
-            }
-        }
-        // add legal channel
-        sourceContext.getAllChannels().add(ctx.channel());
-        ctx.fireChannelActive();
-        LOG.info("{} added new channel = {}, current connections = {}, maxConnections = {}",
-                sourceContext.getSource().getName(), ctx.channel(),
-                (sourceContext.getAllChannels().size() - 1), sourceContext.getMaxConnections());
-    }
-}
diff --git a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source/tcp/InlongTcpChannelPipelineFactory.java b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source/tcp/InlongTcpChannelPipelineFactory.java
deleted file mode 100644
index 7446775772..0000000000
--- a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source/tcp/InlongTcpChannelPipelineFactory.java
+++ /dev/null
@@ -1,107 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.inlong.dataproxy.source.tcp;
-
-import org.apache.inlong.dataproxy.consts.ConfigConstants;
-import org.apache.inlong.dataproxy.source.SourceContext;
-
-import com.google.common.base.Preconditions;
-import io.netty.channel.ChannelInboundHandlerAdapter;
-import io.netty.channel.ChannelInitializer;
-import io.netty.channel.socket.SocketChannel;
-import io.netty.handler.codec.LengthFieldBasedFrameDecoder;
-import io.netty.handler.timeout.ReadTimeoutHandler;
-import org.apache.commons.lang3.StringUtils;
-import org.apache.flume.Context;
-import org.apache.flume.conf.Configurable;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.lang.reflect.Constructor;
-import java.util.concurrent.TimeUnit;
-
-/**
- * InlongTcpChannelPipelineFactory
- */
-public class InlongTcpChannelPipelineFactory extends ChannelInitializer<SocketChannel>
-        implements
-            Configurable {
-
-    public static final Logger LOG = LoggerFactory.getLogger(InlongTcpChannelPipelineFactory.class);
-    public static final int DEFAULT_LENGTH_FIELD_OFFSET = 0;
-    public static final int DEFAULT_LENGTH_FIELD_LENGTH = 4;
-    public static final int DEFAULT_LENGTH_ADJUSTMENT = -4;
-    public static final int DEFAULT_INITIAL_BYTES_TO_STRIP = 0;
-    public static final boolean DEFAULT_FAIL_FAST = true;
-    private static final int DEFAULT_READ_IDLE_TIME = 70 * 60 * 1000;
-    private SourceContext sourceContext;
-    private String messageHandlerName;
-    private String protocolType;
-
-    /**
-     * get server factory
-     *
-     * @param sourceContext
-     */
-    public InlongTcpChannelPipelineFactory(SourceContext sourceContext, String protocolType) {
-        this.sourceContext = sourceContext;
-        this.protocolType = protocolType;
-    }
-
-    @Override
-    protected void initChannel(SocketChannel ch) {
-
-        if (StringUtils.isEmpty(protocolType) || this.protocolType
-                .equalsIgnoreCase(ConfigConstants.TCP_PROTOCOL)) {
-            ch.pipeline().addLast("messageDecoder", new LengthFieldBasedFrameDecoder(
-                    sourceContext.getMaxMsgLength(), DEFAULT_LENGTH_FIELD_OFFSET,
-                    DEFAULT_LENGTH_FIELD_LENGTH,
-                    DEFAULT_LENGTH_ADJUSTMENT, DEFAULT_INITIAL_BYTES_TO_STRIP, DEFAULT_FAIL_FAST));
-            ch.pipeline().addLast("readTimeoutHandler",
-                    new ReadTimeoutHandler(DEFAULT_READ_IDLE_TIME, TimeUnit.MILLISECONDS));
-        }
-
-        if (sourceContext.getSource().getChannelProcessor() != null) {
-            try {
-                Class<? extends ChannelInboundHandlerAdapter> clazz =
-                        (Class<? extends ChannelInboundHandlerAdapter>) Class
-                                .forName(messageHandlerName);
-
-                Constructor<?> ctor = clazz.getConstructor(SourceContext.class);
-
-                ChannelInboundHandlerAdapter messageHandler = (ChannelInboundHandlerAdapter) ctor
-                        .newInstance(sourceContext);
-
-                ch.pipeline().addLast("messageHandler", messageHandler);
-            } catch (Exception e) {
-                LOG.error("SimpleChannelHandler.newInstance  has error:"
-                        + sourceContext.getSource().getName(), e);
-            }
-        }
-    }
-
-    @Override
-    public void configure(Context context) {
-        LOG.info("context is {}", context);
-        messageHandlerName = context.getString(ConfigConstants.MESSAGE_HANDLER_NAME,
-                InlongTcpChannelHandler.class.getName());
-        messageHandlerName = messageHandlerName.trim();
-        Preconditions.checkArgument(StringUtils.isNotBlank(messageHandlerName),
-                "messageHandlerName is empty");
-    }
-}
diff --git a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source/tcp/InlongTcpSource.java b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source/tcp/InlongTcpSource.java
deleted file mode 100644
index 25b2326e5e..0000000000
--- a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source/tcp/InlongTcpSource.java
+++ /dev/null
@@ -1,173 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.inlong.dataproxy.source.tcp;
-
-import org.apache.inlong.dataproxy.admin.ProxyServiceMBean;
-import org.apache.inlong.dataproxy.consts.ConfigConstants;
-import org.apache.inlong.dataproxy.source.SimpleTcpSource;
-import org.apache.inlong.dataproxy.source.SourceContext;
-import org.apache.inlong.sdk.commons.admin.AdminServiceRegister;
-
-import com.google.common.base.Preconditions;
-import io.netty.channel.ChannelInitializer;
-import org.apache.commons.lang3.StringUtils;
-import org.apache.flume.Context;
-import org.apache.flume.EventDrivenSource;
-import org.apache.flume.FlumeException;
-import org.apache.flume.conf.Configurable;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.lang.reflect.Constructor;
-
-/**
- * Inlong tcp source
- */
-public class InlongTcpSource extends SimpleTcpSource
-        implements
-            Configurable,
-            EventDrivenSource,
-            ProxyServiceMBean {
-
-    public static final Logger LOG = LoggerFactory.getLogger(InlongTcpSource.class);
-
-    protected SourceContext sourceContext;
-
-    protected String msgFactoryName;
-    protected String messageHandlerName;
-
-    private Configurable pipelineFactoryConfigurable = null;
-
-    /**
-     * Constructor
-     */
-    public InlongTcpSource() {
-        super();
-    }
-
-    /**
-     * start
-     */
-    @SuppressWarnings({"unchecked", "rawtypes"})
-    @Override
-    public synchronized void startSource() {
-        super.startSource();
-    }
-
-    /**
-     * stop
-     */
-    @Override
-    public synchronized void stop() {
-        LOG.info("[STOP SOURCE]{} stopping...", super.toString());
-        super.stop();
-    }
-
-    /**
-     * configure
-     * 
-     * @param context
-     */
-    @Override
-    public void configure(Context context) {
-        try {
-            LOG.info("context is {}", context);
-            super.configure(context);
-            this.sourceContext = new SourceContext(this, allChannels, context);
-            // start
-            this.sourceContext.start();
-
-            msgFactoryName = context.getString(ConfigConstants.MSG_FACTORY_NAME,
-                    InlongTcpChannelPipelineFactory.class.getName()).trim();
-            Preconditions.checkArgument(StringUtils.isNotBlank(msgFactoryName),
-                    "msgFactoryName is empty");
-
-            messageHandlerName = context.getString(ConfigConstants.MESSAGE_HANDLER_NAME,
-                    InlongTcpChannelHandler.class.getName());
-            messageHandlerName = messageHandlerName.trim();
-            Preconditions.checkArgument(StringUtils.isNotBlank(messageHandlerName),
-                    "messageHandlerName is empty");
-
-            if (this.pipelineFactoryConfigurable != null) {
-                this.pipelineFactoryConfigurable.configure(context);
-            }
-            // register
-            AdminServiceRegister.register(ProxyServiceMBean.MBEAN_TYPE, this.getName(), this);
-        } catch (Throwable t) {
-            LOG.error(t.getMessage(), t);
-        }
-    }
-
-    /**
-     * get ChannelInitializerFactory
-     *
-     * @return ChannelInitializer
-     */
-    public ChannelInitializer getChannelInitializerFactory() {
-        LOG.info(new StringBuffer("load msgFactory=").append(msgFactoryName)
-                .append(" and serviceDecoderName=").append(serviceDecoderName).toString());
-
-        ChannelInitializer fac = null;
-        try {
-            Class<? extends ChannelInitializer> clazz = (Class<? extends ChannelInitializer>) Class
-                    .forName(msgFactoryName);
-
-            Constructor ctor = clazz.getConstructor(SourceContext.class, String.class);
-
-            LOG.info("Using channel processor:{}", getChannelProcessor().getClass().getName());
-            fac = (ChannelInitializer) ctor.newInstance(sourceContext,
-                    this.getProtocolName());
-            if (fac instanceof Configurable) {
-                this.pipelineFactoryConfigurable = ((Configurable) fac);
-                this.pipelineFactoryConfigurable.configure(sourceContext.getParentContext());
-            }
-        } catch (Exception e) {
-            LOG.error(
-                    "Inlong Tcp Source start error, fail to construct ChannelPipelineFactory with name {}, ex {}",
-                    msgFactoryName, e);
-            stop();
-            throw new FlumeException(e.getMessage(), e);
-        }
-        return fac;
-    }
-
-    /**
-     * getProtocolName
-     * 
-     * @return
-     */
-    public String getProtocolName() {
-        return "tcp";
-    }
-
-    /**
-     * stopService
-     */
-    @Override
-    public void stopService() {
-        this.sourceContext.setRejectService(true);
-    }
-
-    /**
-     * recoverService
-     */
-    @Override
-    public void recoverService() {
-        this.sourceContext.setRejectService(false);
-    }
-}
diff --git a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source/tcp/InlongTcpSourceCallback.java b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source/tcp/InlongTcpSourceCallback.java
deleted file mode 100644
index 2fba079b1a..0000000000
--- a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source/tcp/InlongTcpSourceCallback.java
+++ /dev/null
@@ -1,113 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.inlong.dataproxy.source.tcp;
-
-import org.apache.inlong.sdk.commons.protocol.ProxySdk.MessagePackHeader;
-import org.apache.inlong.sdk.commons.protocol.ProxySdk.ResponseInfo;
-import org.apache.inlong.sdk.commons.protocol.ProxySdk.ResultCode;
-import org.apache.inlong.sdk.commons.protocol.SourceCallback;
-
-import io.netty.buffer.ByteBuf;
-import io.netty.buffer.Unpooled;
-import io.netty.channel.Channel;
-import io.netty.channel.ChannelHandlerContext;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.atomic.AtomicBoolean;
-
-/**
- * InlongTcpEventCallback
- * 
- */
-public class InlongTcpSourceCallback implements SourceCallback {
-
-    public static final Logger LOG = LoggerFactory.getLogger(InlongTcpSourceCallback.class);
-
-    private final ChannelHandlerContext ctx;
-    private final MessagePackHeader header;
-    private final CountDownLatch latch;
-    private final AtomicBoolean hasResponsed = new AtomicBoolean(false);
-
-    /**
-     * Constructor
-     * @param ctx
-     * @param header
-     */
-    public InlongTcpSourceCallback(ChannelHandlerContext ctx, MessagePackHeader header) {
-        this.ctx = ctx;
-        this.header = header;
-        this.latch = new CountDownLatch(1);
-    }
-
-    /**
-     * callback
-     * @param resultCode
-     */
-    @Override
-    public void callback(ResultCode resultCode) {
-        // If DataProxy have sent timeout response to SDK, DataProxy do not send success response to SDK again when
-        // event is success to save.
-        if (this.hasResponsed.getAndSet(true)) {
-            return;
-        }
-        // response
-        try {
-            ResponseInfo.Builder builder = ResponseInfo.newBuilder();
-            builder.setResult(resultCode);
-            builder.setPackId(header.getPackId());
-
-            // encode
-            byte[] responseBytes = builder.build().toByteArray();
-            //
-            ByteBuf buffer = Unpooled.wrappedBuffer(responseBytes);
-            Channel remoteChannel = ctx.channel();
-            if (remoteChannel.isWritable()) {
-                remoteChannel.write(buffer);
-            } else {
-                LOG.warn("the send buffer2 is full, so disconnect it!"
-                        + "please check remote client; Connection info:{}",
-                        remoteChannel);
-                buffer.release();
-            }
-        } catch (Exception e) {
-            LOG.error(e.getMessage(), e);
-        } finally {
-            // notice TCP session
-            this.latch.countDown();
-        }
-    }
-
-    /**
-     * get hasResponsed
-     * @return the hasResponsed
-     */
-    public AtomicBoolean getHasResponsed() {
-        return hasResponsed;
-    }
-
-    /**
-     * get latch
-     * @return the latch
-     */
-    public CountDownLatch getLatch() {
-        return latch;
-    }
-
-}
diff --git a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source2/v0msg/AbsV0MsgCodec.java b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source/v0msg/AbsV0MsgCodec.java
similarity index 98%
rename from inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source2/v0msg/AbsV0MsgCodec.java
rename to inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source/v0msg/AbsV0MsgCodec.java
index fd367a3522..11a180a1bd 100644
--- a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source2/v0msg/AbsV0MsgCodec.java
+++ b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source/v0msg/AbsV0MsgCodec.java
@@ -15,14 +15,14 @@
  * limitations under the License.
  */
 
-package org.apache.inlong.dataproxy.source2.v0msg;
+package org.apache.inlong.dataproxy.source.v0msg;
 
 import org.apache.inlong.common.enums.DataProxyErrCode;
 import org.apache.inlong.common.enums.DataProxyMsgEncType;
 import org.apache.inlong.common.msg.AttributeConstants;
 import org.apache.inlong.dataproxy.consts.ConfigConstants;
 import org.apache.inlong.dataproxy.consts.StatConstants;
-import org.apache.inlong.dataproxy.source2.BaseSource;
+import org.apache.inlong.dataproxy.source.BaseSource;
 import org.apache.inlong.dataproxy.utils.DateTimeUtils;
 import org.apache.inlong.sdk.commons.protocol.EventConstants;
 
diff --git a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source2/v0msg/CodecBinMsg.java b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source/v0msg/CodecBinMsg.java
similarity index 91%
rename from inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source2/v0msg/CodecBinMsg.java
rename to inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source/v0msg/CodecBinMsg.java
index cd2353f01e..260f38bb8f 100644
--- a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source2/v0msg/CodecBinMsg.java
+++ b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source/v0msg/CodecBinMsg.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.inlong.dataproxy.source2.v0msg;
+package org.apache.inlong.dataproxy.source.v0msg;
 
 import org.apache.inlong.common.enums.DataProxyErrCode;
 import org.apache.inlong.common.msg.AttributeConstants;
@@ -24,7 +24,7 @@ import org.apache.inlong.common.msg.MsgType;
 import org.apache.inlong.dataproxy.base.SinkRspEvent;
 import org.apache.inlong.dataproxy.config.ConfigManager;
 import org.apache.inlong.dataproxy.consts.StatConstants;
-import org.apache.inlong.dataproxy.source2.BaseSource;
+import org.apache.inlong.dataproxy.source.BaseSource;
 
 import io.netty.buffer.ByteBuf;
 import io.netty.channel.Channel;
@@ -36,21 +36,21 @@ import java.nio.ByteBuffer;
 import java.nio.charset.StandardCharsets;
 import java.util.Map;
 
-import static org.apache.inlong.dataproxy.source2.v0msg.MsgFieldConsts.BIN_MSG_ATTRLEN_SIZE;
-import static org.apache.inlong.dataproxy.source2.v0msg.MsgFieldConsts.BIN_MSG_BODYLEN_OFFSET;
-import static org.apache.inlong.dataproxy.source2.v0msg.MsgFieldConsts.BIN_MSG_BODY_OFFSET;
-import static org.apache.inlong.dataproxy.source2.v0msg.MsgFieldConsts.BIN_MSG_CNT_OFFSET;
-import static org.apache.inlong.dataproxy.source2.v0msg.MsgFieldConsts.BIN_MSG_DT_OFFSET;
-import static org.apache.inlong.dataproxy.source2.v0msg.MsgFieldConsts.BIN_MSG_EXTEND_OFFSET;
-import static org.apache.inlong.dataproxy.source2.v0msg.MsgFieldConsts.BIN_MSG_FORMAT_SIZE;
-import static org.apache.inlong.dataproxy.source2.v0msg.MsgFieldConsts.BIN_MSG_GROUPIDNUM_OFFSET;
-import static org.apache.inlong.dataproxy.source2.v0msg.MsgFieldConsts.BIN_MSG_MAGIC;
-import static org.apache.inlong.dataproxy.source2.v0msg.MsgFieldConsts.BIN_MSG_MAGIC_SIZE;
-import static org.apache.inlong.dataproxy.source2.v0msg.MsgFieldConsts.BIN_MSG_MSGTYPE_OFFSET;
-import static org.apache.inlong.dataproxy.source2.v0msg.MsgFieldConsts.BIN_MSG_STREAMIDNUM_OFFSET;
-import static org.apache.inlong.dataproxy.source2.v0msg.MsgFieldConsts.BIN_MSG_TOTALLEN_OFFSET;
-import static org.apache.inlong.dataproxy.source2.v0msg.MsgFieldConsts.BIN_MSG_TOTALLEN_SIZE;
-import static org.apache.inlong.dataproxy.source2.v0msg.MsgFieldConsts.BIN_MSG_UNIQ_OFFSET;
+import static org.apache.inlong.dataproxy.source.v0msg.MsgFieldConsts.BIN_MSG_ATTRLEN_SIZE;
+import static org.apache.inlong.dataproxy.source.v0msg.MsgFieldConsts.BIN_MSG_BODYLEN_OFFSET;
+import static org.apache.inlong.dataproxy.source.v0msg.MsgFieldConsts.BIN_MSG_BODY_OFFSET;
+import static org.apache.inlong.dataproxy.source.v0msg.MsgFieldConsts.BIN_MSG_CNT_OFFSET;
+import static org.apache.inlong.dataproxy.source.v0msg.MsgFieldConsts.BIN_MSG_DT_OFFSET;
+import static org.apache.inlong.dataproxy.source.v0msg.MsgFieldConsts.BIN_MSG_EXTEND_OFFSET;
+import static org.apache.inlong.dataproxy.source.v0msg.MsgFieldConsts.BIN_MSG_FORMAT_SIZE;
+import static org.apache.inlong.dataproxy.source.v0msg.MsgFieldConsts.BIN_MSG_GROUPIDNUM_OFFSET;
+import static org.apache.inlong.dataproxy.source.v0msg.MsgFieldConsts.BIN_MSG_MAGIC;
+import static org.apache.inlong.dataproxy.source.v0msg.MsgFieldConsts.BIN_MSG_MAGIC_SIZE;
+import static org.apache.inlong.dataproxy.source.v0msg.MsgFieldConsts.BIN_MSG_MSGTYPE_OFFSET;
+import static org.apache.inlong.dataproxy.source.v0msg.MsgFieldConsts.BIN_MSG_STREAMIDNUM_OFFSET;
+import static org.apache.inlong.dataproxy.source.v0msg.MsgFieldConsts.BIN_MSG_TOTALLEN_OFFSET;
+import static org.apache.inlong.dataproxy.source.v0msg.MsgFieldConsts.BIN_MSG_TOTALLEN_SIZE;
+import static org.apache.inlong.dataproxy.source.v0msg.MsgFieldConsts.BIN_MSG_UNIQ_OFFSET;
 
 public class CodecBinMsg extends AbsV0MsgCodec {
 
diff --git a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source2/v0msg/CodecTextMsg.java b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source/v0msg/CodecTextMsg.java
similarity index 96%
rename from inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source2/v0msg/CodecTextMsg.java
rename to inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source/v0msg/CodecTextMsg.java
index c7189dbe80..772529f38c 100644
--- a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source2/v0msg/CodecTextMsg.java
+++ b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source/v0msg/CodecTextMsg.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.inlong.dataproxy.source2.v0msg;
+package org.apache.inlong.dataproxy.source.v0msg;
 
 import org.apache.inlong.common.enums.DataProxyErrCode;
 import org.apache.inlong.common.msg.AttributeConstants;
@@ -23,7 +23,7 @@ import org.apache.inlong.common.msg.InLongMsg;
 import org.apache.inlong.common.msg.MsgType;
 import org.apache.inlong.dataproxy.config.ConfigManager;
 import org.apache.inlong.dataproxy.consts.StatConstants;
-import org.apache.inlong.dataproxy.source2.BaseSource;
+import org.apache.inlong.dataproxy.source.BaseSource;
 
 import io.netty.buffer.ByteBuf;
 import io.netty.channel.Channel;
@@ -35,10 +35,10 @@ import org.xerial.snappy.Snappy;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 
-import static org.apache.inlong.dataproxy.source2.v0msg.MsgFieldConsts.TXT_MSG_BODYLEN_OFFSET;
-import static org.apache.inlong.dataproxy.source2.v0msg.MsgFieldConsts.TXT_MSG_BODY_OFFSET;
-import static org.apache.inlong.dataproxy.source2.v0msg.MsgFieldConsts.TXT_MSG_FORMAT_SIZE;
-import static org.apache.inlong.dataproxy.source2.v0msg.MsgFieldConsts.TXT_MSG_TOTALLEN_SIZE;
+import static org.apache.inlong.dataproxy.source.v0msg.MsgFieldConsts.TXT_MSG_BODYLEN_OFFSET;
+import static org.apache.inlong.dataproxy.source.v0msg.MsgFieldConsts.TXT_MSG_BODY_OFFSET;
+import static org.apache.inlong.dataproxy.source.v0msg.MsgFieldConsts.TXT_MSG_FORMAT_SIZE;
+import static org.apache.inlong.dataproxy.source.v0msg.MsgFieldConsts.TXT_MSG_TOTALLEN_SIZE;
 
 public class CodecTextMsg extends AbsV0MsgCodec {
 
diff --git a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source2/v0msg/MsgFieldConsts.java b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source/v0msg/MsgFieldConsts.java
similarity index 98%
rename from inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source2/v0msg/MsgFieldConsts.java
rename to inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source/v0msg/MsgFieldConsts.java
index 20b0e884ec..1b949c9a33 100644
--- a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source2/v0msg/MsgFieldConsts.java
+++ b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source/v0msg/MsgFieldConsts.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.inlong.dataproxy.source2.v0msg;
+package org.apache.inlong.dataproxy.source.v0msg;
 
 public class MsgFieldConsts {
 
diff --git a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source2/v1msg/InlongTcpSourceCallback.java b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source/v1msg/InlongTcpSourceCallback.java
similarity index 98%
rename from inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source2/v1msg/InlongTcpSourceCallback.java
rename to inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source/v1msg/InlongTcpSourceCallback.java
index c11cffa63e..7584f7371a 100644
--- a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source2/v1msg/InlongTcpSourceCallback.java
+++ b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source/v1msg/InlongTcpSourceCallback.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.inlong.dataproxy.source2.v1msg;
+package org.apache.inlong.dataproxy.source.v1msg;
 
 import org.apache.inlong.common.monitor.LogCounter;
 import org.apache.inlong.sdk.commons.protocol.ProxySdk.MessagePackHeader;
diff --git a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source2/BaseSource.java b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source2/BaseSource.java
deleted file mode 100644
index 0d65101098..0000000000
--- a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source2/BaseSource.java
+++ /dev/null
@@ -1,582 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.inlong.dataproxy.source2;
-
-import org.apache.inlong.common.metric.MetricRegister;
-import org.apache.inlong.dataproxy.admin.ProxyServiceMBean;
-import org.apache.inlong.dataproxy.channel.FailoverChannelProcessor;
-import org.apache.inlong.dataproxy.config.CommonConfigHolder;
-import org.apache.inlong.dataproxy.config.ConfigManager;
-import org.apache.inlong.dataproxy.config.holder.ConfigUpdateCallback;
-import org.apache.inlong.dataproxy.consts.AttrConstants;
-import org.apache.inlong.dataproxy.metrics.DataProxyMetricItem;
-import org.apache.inlong.dataproxy.metrics.DataProxyMetricItemSet;
-import org.apache.inlong.dataproxy.metrics.audit.AuditUtils;
-import org.apache.inlong.dataproxy.metrics.stats.MonitorIndex;
-import org.apache.inlong.dataproxy.metrics.stats.MonitorStats;
-import org.apache.inlong.dataproxy.source2.httpMsg.InLongHttpMsgHandler;
-import org.apache.inlong.dataproxy.utils.AddressUtils;
-import org.apache.inlong.dataproxy.utils.ConfStringUtils;
-import org.apache.inlong.dataproxy.utils.FailoverChannelProcessorHolder;
-import org.apache.inlong.sdk.commons.admin.AdminServiceRegister;
-
-import com.google.common.base.Preconditions;
-import io.netty.channel.Channel;
-import io.netty.channel.ChannelFuture;
-import io.netty.channel.ChannelInitializer;
-import io.netty.channel.EventLoopGroup;
-import io.netty.channel.group.ChannelGroup;
-import io.netty.channel.group.DefaultChannelGroup;
-import io.netty.util.concurrent.GlobalEventExecutor;
-import org.apache.commons.lang3.StringUtils;
-import org.apache.flume.ChannelSelector;
-import org.apache.flume.Context;
-import org.apache.flume.Event;
-import org.apache.flume.EventDrivenSource;
-import org.apache.flume.FlumeException;
-import org.apache.flume.conf.Configurable;
-import org.apache.flume.source.AbstractSource;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.lang.reflect.Constructor;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.Map;
-
-/**
- * source base class
- */
-public abstract class BaseSource
-        extends
-            AbstractSource
-        implements
-            ConfigUpdateCallback,
-            ProxyServiceMBean,
-            EventDrivenSource,
-            Configurable {
-
-    private static final Logger logger = LoggerFactory.getLogger(BaseSource.class);
-
-    protected Context context;
-    // whether source reject service
-    protected volatile boolean isRejectService = false;
-    // source service host
-    protected String srcHost;
-    // source serviced port
-    protected int srcPort;
-    protected String strPort;
-    // message factory name
-    protected String msgFactoryName;
-    // message handler name
-    protected String messageHandlerName;
-    // source default topic
-    protected String defTopic = "";
-    // source default append attribute
-    protected String defAttr = "";
-    // allowed max message length
-    protected int maxMsgLength;
-    // whether compress message
-    protected boolean isCompressed;
-    // whether filter empty message
-    protected boolean filterEmptyMsg;
-    // whether custom channel processor
-    protected boolean customProcessor;
-    // max netty worker threads
-    protected int maxWorkerThreads;
-    // max netty accept threads
-    protected int maxAcceptThreads;
-    // max read idle time
-    protected long maxReadIdleTimeMs;
-    // max connection count
-    protected int maxConnections;
-    // reuse address
-    protected boolean reuseAddress;
-    // connect backlog
-    protected int conBacklog;
-    // connect linger
-    protected int conLinger = -1;
-    // netty parameters
-    protected EventLoopGroup acceptorGroup;
-    protected EventLoopGroup workerGroup;
-    protected ChannelGroup allChannels;
-    protected ChannelFuture channelFuture;
-    // receive buffer size
-    protected int maxRcvBufferSize;
-    // send buffer size
-    protected int maxSendBufferSize;
-    // file metric statistic
-    private MonitorIndex monitorIndex = null;
-    private MonitorStats monitorStats = null;
-    // metric set
-    private DataProxyMetricItemSet metricItemSet;
-
-    public BaseSource() {
-        super();
-        allChannels = new DefaultChannelGroup("DefaultChannelGroup", GlobalEventExecutor.INSTANCE);
-    }
-
-    @Override
-    public void configure(Context context) {
-        logger.info("{} start to configure context:{}.", this.getName(), context.toString());
-        this.context = context;
-        this.srcHost = getHostIp(context);
-        this.srcPort = getHostPort(context);
-        this.strPort = String.valueOf(this.srcPort);
-        // get message factory
-        String tmpVal = context.getString(SourceConstants.SRCCXT_MSG_FACTORY_NAME,
-                InLongMessageFactory.class.getName()).trim();
-        Preconditions.checkArgument(StringUtils.isNotBlank(tmpVal),
-                SourceConstants.SRCCXT_MSG_FACTORY_NAME + " config is blank");
-        this.msgFactoryName = tmpVal.trim();
-        // get message handler
-        tmpVal = context.getString(SourceConstants.SRCCXT_MESSAGE_HANDLER_NAME);
-        if (StringUtils.isBlank(tmpVal)) {
-            tmpVal = SourceConstants.SRC_PROTOCOL_TYPE_HTTP.equalsIgnoreCase(getProtocolName())
-                    ? InLongHttpMsgHandler.class.getName()
-                    : InLongMessageHandler.class.getName();
-        }
-        Preconditions.checkArgument(StringUtils.isNotBlank(tmpVal),
-                SourceConstants.SRCCXT_MESSAGE_HANDLER_NAME + " config is blank");
-        this.messageHandlerName = tmpVal;
-        // get default attributes
-        tmpVal = context.getString(SourceConstants.SRCCXT_DEF_ATTR);
-        if (StringUtils.isNotBlank(tmpVal)) {
-            this.defAttr = tmpVal.trim();
-        }
-        // get allowed max message length
-        this.maxMsgLength = ConfStringUtils.getIntValue(context,
-                SourceConstants.SRCCXT_MAX_MSG_LENGTH, SourceConstants.VAL_DEF_MAX_MSG_LENGTH);
-        Preconditions.checkArgument((this.maxMsgLength >= SourceConstants.VAL_MIN_MAX_MSG_LENGTH
-                && this.maxMsgLength <= SourceConstants.VAL_MAX_MAX_MSG_LENGTH),
-                SourceConstants.SRCCXT_MAX_MSG_LENGTH + " must be in ["
-                        + SourceConstants.VAL_MIN_MAX_MSG_LENGTH + ", "
-                        + SourceConstants.VAL_MAX_MAX_MSG_LENGTH + "]");
-        // get whether compress message
-        this.isCompressed = context.getBoolean(SourceConstants.SRCCXT_MSG_COMPRESSED,
-                SourceConstants.VAL_DEF_MSG_COMPRESSED);
-        // get whether filter empty message
-        this.filterEmptyMsg = context.getBoolean(SourceConstants.SRCCXT_FILTER_EMPTY_MSG,
-                SourceConstants.VAL_DEF_FILTER_EMPTY_MSG);
-        // get whether custom channel processor
-        this.customProcessor = context.getBoolean(SourceConstants.SRCCXT_CUSTOM_CHANNEL_PROCESSOR,
-                SourceConstants.VAL_DEF_CUSTOM_CH_PROCESSOR);
-        // get max accept threads
-        this.maxAcceptThreads = ConfStringUtils.getIntValue(context,
-                SourceConstants.SRCCXT_MAX_ACCEPT_THREADS, SourceConstants.VAL_DEF_NET_ACCEPT_THREADS);
-        Preconditions.checkArgument((this.maxAcceptThreads >= SourceConstants.VAL_MIN_ACCEPT_THREADS
-                && this.maxAcceptThreads <= SourceConstants.VAL_MAX_ACCEPT_THREADS),
-                SourceConstants.SRCCXT_MAX_ACCEPT_THREADS + " must be in ["
-                        + SourceConstants.VAL_MIN_ACCEPT_THREADS + ", "
-                        + SourceConstants.VAL_MAX_ACCEPT_THREADS + "]");
-        // get max worker threads
-        this.maxWorkerThreads = ConfStringUtils.getIntValue(context,
-                SourceConstants.SRCCXT_MAX_WORKER_THREADS, SourceConstants.VAL_DEF_WORKER_THREADS);
-        Preconditions.checkArgument((this.maxWorkerThreads >= SourceConstants.VAL_MIN_WORKER_THREADS),
-                SourceConstants.SRCCXT_MAX_WORKER_THREADS + " must be >= "
-                        + SourceConstants.VAL_MIN_WORKER_THREADS);
-        // get max read idle time
-        this.maxReadIdleTimeMs = ConfStringUtils.getLongValue(context,
-                SourceConstants.SRCCXT_MAX_READ_IDLE_TIME_MS, SourceConstants.VAL_DEF_READ_IDLE_TIME_MS);
-        Preconditions.checkArgument((this.maxReadIdleTimeMs >= SourceConstants.VAL_MIN_READ_IDLE_TIME_MS
-                && this.maxReadIdleTimeMs <= SourceConstants.VAL_MAX_READ_IDLE_TIME_MS),
-                SourceConstants.SRCCXT_MAX_READ_IDLE_TIME_MS + " must be in ["
-                        + SourceConstants.VAL_MIN_READ_IDLE_TIME_MS + ", "
-                        + SourceConstants.VAL_MAX_READ_IDLE_TIME_MS + "]");
-        // get max connect count
-        this.maxConnections = ConfStringUtils.getIntValue(context,
-                SourceConstants.SRCCXT_MAX_CONNECTION_CNT, SourceConstants.VAL_DEF_MAX_CONNECTION_CNT);
-        Preconditions.checkArgument(this.maxConnections >= SourceConstants.VAL_MIN_CONNECTION_CNT,
-                SourceConstants.SRCCXT_MAX_CONNECTION_CNT + " must be >= "
-                        + SourceConstants.VAL_MIN_CONNECTION_CNT);
-        // get connect backlog
-        this.conBacklog = ConfStringUtils.getIntValue(context,
-                SourceConstants.SRCCXT_CONN_BACKLOG, SourceConstants.VAL_DEF_CONN_BACKLOG);
-        Preconditions.checkArgument(this.conBacklog >= SourceConstants.VAL_MIN_CONN_BACKLOG,
-                SourceConstants.SRCCXT_CONN_BACKLOG + " must be >= "
-                        + SourceConstants.VAL_MIN_CONN_BACKLOG);
-        // get connect linger
-        Integer tmpValue = context.getInteger(SourceConstants.SRCCXT_CONN_LINGER);
-        if (tmpValue != null && tmpValue >= 0) {
-            this.conLinger = tmpValue;
-        }
-        // get whether reuse address
-        this.reuseAddress = context.getBoolean(SourceConstants.SRCCXT_REUSE_ADDRESS,
-                SourceConstants.VAL_DEF_REUSE_ADDRESS);
-
-        // get whether custom channel processor
-        this.customProcessor = context.getBoolean(SourceConstants.SRCCXT_CUSTOM_CHANNEL_PROCESSOR,
-                SourceConstants.VAL_DEF_CUSTOM_CH_PROCESSOR);
-        // get max receive buffer size
-        this.maxRcvBufferSize = ConfStringUtils.getIntValue(context,
-                SourceConstants.SRCCXT_RECEIVE_BUFFER_SIZE, SourceConstants.VAL_DEF_RECEIVE_BUFFER_SIZE);
-        Preconditions.checkArgument(this.maxRcvBufferSize >= SourceConstants.VAL_MIN_RECEIVE_BUFFER_SIZE,
-                SourceConstants.SRCCXT_RECEIVE_BUFFER_SIZE + " must be >= "
-                        + SourceConstants.VAL_MIN_RECEIVE_BUFFER_SIZE);
-        // get max send buffer size
-        this.maxSendBufferSize = ConfStringUtils.getIntValue(context,
-                SourceConstants.SRCCXT_SEND_BUFFER_SIZE, SourceConstants.VAL_DEF_SEND_BUFFER_SIZE);
-        Preconditions.checkArgument(this.maxSendBufferSize >= SourceConstants.VAL_MIN_SEND_BUFFER_SIZE,
-                SourceConstants.SRCCXT_SEND_BUFFER_SIZE + " must be >= "
-                        + SourceConstants.VAL_MIN_SEND_BUFFER_SIZE);
-    }
-
-    @Override
-    public synchronized void start() {
-        if (customProcessor) {
-            ChannelSelector selector = getChannelProcessor().getSelector();
-            FailoverChannelProcessor newProcessor = new FailoverChannelProcessor(selector);
-            newProcessor.configure(this.context);
-            setChannelProcessor(newProcessor);
-            FailoverChannelProcessorHolder.setChannelProcessor(newProcessor);
-        }
-        super.start();
-        // initial metric item set
-        this.metricItemSet = new DataProxyMetricItemSet(
-                CommonConfigHolder.getInstance().getClusterName(), getName(), String.valueOf(srcPort));
-        MetricRegister.register(metricItemSet);
-        // init monitor logic
-        if (CommonConfigHolder.getInstance().isEnableFileMetric()) {
-            this.monitorIndex = new MonitorIndex(CommonConfigHolder.getInstance().getFileMetricSourceOutName(),
-                    CommonConfigHolder.getInstance().getFileMetricStatInvlSec() * 1000L,
-                    CommonConfigHolder.getInstance().getFileMetricStatCacheCnt());
-            this.monitorIndex.start();
-            this.monitorStats = new MonitorStats(
-                    CommonConfigHolder.getInstance().getFileMetricEventOutName()
-                            + AttrConstants.SEP_HASHTAG + this.getProtocolName(),
-                    CommonConfigHolder.getInstance().getFileMetricStatInvlSec() * 1000L,
-                    CommonConfigHolder.getInstance().getFileMetricStatCacheCnt());
-            this.monitorStats.start();
-        }
-        startSource();
-        // register
-        AdminServiceRegister.register(ProxyServiceMBean.MBEAN_TYPE, this.getName(), this);
-    }
-
-    @Override
-    public synchronized void stop() {
-        logger.info("[STOP {} SOURCE]{} stopping...", this.getProtocolName(), this.getName());
-        // close channels
-        if (!allChannels.isEmpty()) {
-            try {
-                allChannels.close().awaitUninterruptibly();
-            } catch (Exception e) {
-                logger.warn("Close {} netty channels throw exception", this.getName(), e);
-            } finally {
-                allChannels.clear();
-            }
-        }
-        // close channel future
-        if (channelFuture != null) {
-            try {
-                channelFuture.channel().closeFuture().sync();
-            } catch (InterruptedException e) {
-                logger.warn("Close {} channel future throw exception", this.getName(), e);
-            }
-        }
-        // stop super class
-        super.stop();
-        // stop workers
-        if (this.acceptorGroup != null) {
-            this.acceptorGroup.shutdownGracefully();
-        }
-        if (this.workerGroup != null) {
-            this.workerGroup.shutdownGracefully();
-        }
-        // stop file statistic index
-        if (CommonConfigHolder.getInstance().isEnableFileMetric()) {
-            if (monitorIndex != null) {
-                monitorIndex.stop();
-            }
-            if (monitorStats != null) {
-                monitorStats.stop();
-            }
-        }
-        logger.info("[STOP {} SOURCE]{} stopped", this.getProtocolName(), this.getName());
-    }
-
-    @Override
-    public void update() {
-        // check current all links
-        if (ConfigManager.getInstance().needChkIllegalIP()) {
-            int cnt = 0;
-            Channel channel;
-            String strRemoteIP;
-            long startTime = System.currentTimeMillis();
-            Iterator<Channel> iterator = allChannels.iterator();
-            while (iterator.hasNext()) {
-                channel = iterator.next();
-                strRemoteIP = AddressUtils.getChannelRemoteIP(channel);
-                if (strRemoteIP == null) {
-                    continue;
-                }
-                if (ConfigManager.getInstance().isIllegalIP(strRemoteIP)) {
-                    channel.disconnect();
-                    channel.close();
-                    allChannels.remove(channel);
-                    cnt++;
-                    logger.error(strRemoteIP + " is Illegal IP, so disconnect it !");
-                }
-            }
-            logger.info("Source {} channel check, disconnects {} Illegal channels, waist {} ms",
-                    getName(), cnt, (System.currentTimeMillis() - startTime));
-        }
-    }
-
-    /**
-     * get metricItemSet
-     *
-     * @return the metricItemSet
-     */
-    public DataProxyMetricItemSet getMetricItemSet() {
-        return metricItemSet;
-    }
-
-    public Context getContext() {
-        return context;
-    }
-
-    public String getSrcHost() {
-        return srcHost;
-    }
-
-    public int getSrcPort() {
-        return srcPort;
-    }
-
-    public String getStrPort() {
-        return strPort;
-    }
-
-    public String getDefAttr() {
-        return defAttr;
-    }
-
-    public int getMaxMsgLength() {
-        return maxMsgLength;
-    }
-
-    public boolean isCompressed() {
-        return isCompressed;
-    }
-
-    public boolean isFilterEmptyMsg() {
-        return filterEmptyMsg;
-    }
-
-    public boolean isCustomProcessor() {
-        return customProcessor;
-    }
-
-    public int getMaxConnections() {
-        return maxConnections;
-    }
-
-    public ChannelGroup getAllChannels() {
-        return allChannels;
-    }
-
-    public long getMaxReadIdleTimeMs() {
-        return maxReadIdleTimeMs;
-    }
-
-    public String getMessageHandlerName() {
-        return messageHandlerName;
-    }
-
-    public int getMaxWorkerThreads() {
-        return maxWorkerThreads;
-    }
-
-    public void fileMetricIncSumStats(String eventKey) {
-        if (CommonConfigHolder.getInstance().isEnableFileMetric()) {
-            monitorStats.incSumStats(eventKey);
-        }
-    }
-
-    public void fileMetricIncDetailStats(String eventKey) {
-        if (CommonConfigHolder.getInstance().isEnableFileMetric()) {
-            monitorStats.incDetailStats(eventKey);
-        }
-    }
-
-    public void fileMetricAddSuccCnt(String key, int cnt, int packCnt, long packSize) {
-        if (CommonConfigHolder.getInstance().isEnableFileMetric()) {
-            monitorIndex.addSuccStats(key, cnt, packCnt, packSize);
-        }
-    }
-
-    public void fileMetricAddFailCnt(String key, int failCnt) {
-        if (CommonConfigHolder.getInstance().isEnableFileMetric()) {
-            monitorIndex.addFailStats(key, failCnt);
-        }
-    }
-    /**
-     * addMetric
-     *
-     * @param result
-     * @param size
-     * @param event
-     */
-    public void addMetric(boolean result, long size, Event event) {
-        Map<String, String> dimensions = new HashMap<>();
-        dimensions.put(DataProxyMetricItem.KEY_CLUSTER_ID, CommonConfigHolder.getInstance().getClusterName());
-        dimensions.put(DataProxyMetricItem.KEY_SOURCE_ID, getName());
-        dimensions.put(DataProxyMetricItem.KEY_SOURCE_DATA_ID, getStrPort());
-        DataProxyMetricItem.fillInlongId(event, dimensions);
-        DataProxyMetricItem.fillAuditFormatTime(event, dimensions);
-        DataProxyMetricItem metricItem = metricItemSet.findMetricItem(dimensions);
-        if (result) {
-            metricItem.readSuccessCount.incrementAndGet();
-            metricItem.readSuccessSize.addAndGet(size);
-            AuditUtils.add(AuditUtils.AUDIT_ID_DATAPROXY_READ_SUCCESS, event);
-        } else {
-            metricItem.readFailCount.incrementAndGet();
-            metricItem.readFailSize.addAndGet(size);
-        }
-    }
-
-    /**
-     * channel factory
-     *
-     * @return
-     */
-    public ChannelInitializer getChannelInitializerFactory() {
-        ChannelInitializer fac = null;
-        logger.info(this.getName() + " load msgFactory=" + msgFactoryName);
-        try {
-            Class<? extends ChannelInitializer> clazz =
-                    (Class<? extends ChannelInitializer>) Class.forName(msgFactoryName);
-            Constructor ctor = clazz.getConstructor(BaseSource.class);
-            logger.info("Using channel processor:{}", getChannelProcessor().getClass().getName());
-            fac = (ChannelInitializer) ctor.newInstance(this);
-        } catch (Exception e) {
-            logger.error("{} start error, fail to construct ChannelPipelineFactory with name {}",
-                    this.getName(), msgFactoryName, e);
-            stop();
-            throw new FlumeException(e.getMessage());
-        }
-        return fac;
-    }
-
-    public abstract String getProtocolName();
-
-    public abstract void startSource();
-
-    /**
-     * stopService
-     */
-    @Override
-    public void stopService() {
-        this.isRejectService = true;
-    }
-
-    /**
-     * recoverService
-     */
-    @Override
-    public void recoverService() {
-        this.isRejectService = false;
-    }
-
-    /**
-     * isRejectService
-     *
-     * @return
-     */
-    public boolean isRejectService() {
-        return isRejectService;
-    }
-
-    /**
-     * getHostIp
-     *
-     * @param context
-     * @return
-     */
-    private String getHostIp(Context context) {
-        String result = null;
-        // first get host ip from dataProxy.conf
-        String tmpVal = context.getString(SourceConstants.SRCCXT_CONFIG_HOST);
-        if (StringUtils.isNotBlank(tmpVal)) {
-            tmpVal = tmpVal.trim();
-            Preconditions.checkArgument(ConfStringUtils.isValidIp(tmpVal),
-                    SourceConstants.SRCCXT_CONFIG_HOST + "(" + tmpVal + ") config in conf not valid");
-            result = tmpVal;
-        }
-        // second get host ip from system env
-        Map<String, String> envMap = System.getenv();
-        if (envMap.containsKey(SourceConstants.SYSENV_HOST_IP)) {
-            tmpVal = envMap.get(SourceConstants.SYSENV_HOST_IP);
-            Preconditions.checkArgument(ConfStringUtils.isValidIp(tmpVal),
-                    SourceConstants.SYSENV_HOST_IP + "(" + tmpVal + ") config in system env not valid");
-            result = tmpVal.trim();
-        }
-        if (StringUtils.isBlank(result)) {
-            result = SourceConstants.VAL_DEF_HOST_VALUE;
-        }
-        return result;
-    }
-
-    /**
-     * getHostPort
-     *
-     * @param context
-     * @return
-     */
-    private int getHostPort(Context context) {
-        Integer result = null;
-        // first get host port from dataProxy.conf
-        String tmpVal = context.getString(SourceConstants.SRCCXT_CONFIG_PORT);
-        if (StringUtils.isNotBlank(tmpVal)) {
-            tmpVal = tmpVal.trim();
-            try {
-                result = Integer.parseInt(tmpVal);
-            } catch (Throwable e) {
-                throw new IllegalArgumentException(
-                        SourceConstants.SYSENV_HOST_PORT + "(" + tmpVal + ") config in conf not integer");
-            }
-        }
-        if (result != null) {
-            Preconditions.checkArgument(ConfStringUtils.isValidPort(result),
-                    SourceConstants.SRCCXT_CONFIG_PORT + "(" + result + ") config in conf not valid");
-        }
-        // second get host port from system env
-        Map<String, String> envMap = System.getenv();
-        if (envMap.containsKey(SourceConstants.SYSENV_HOST_PORT)) {
-            tmpVal = envMap.get(SourceConstants.SYSENV_HOST_PORT);
-            if (StringUtils.isNotBlank(tmpVal)) {
-                tmpVal = tmpVal.trim();
-                try {
-                    result = Integer.parseInt(tmpVal);
-                } catch (Throwable e) {
-                    throw new IllegalArgumentException(
-                            SourceConstants.SYSENV_HOST_PORT + "(" + tmpVal + ") config in system env not integer");
-                }
-                Preconditions.checkArgument(ConfStringUtils.isValidPort(result),
-                        SourceConstants.SYSENV_HOST_PORT + "(" + tmpVal + ") config in system env not valid");
-            }
-        }
-        if (result == null) {
-            throw new IllegalArgumentException("Required parameter " +
-                    SourceConstants.SRCCXT_CONFIG_PORT + " must exist and may not be null");
-        }
-        return result;
-    }
-
-}
diff --git a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source2/InLongMessageFactory.java b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source2/InLongMessageFactory.java
deleted file mode 100644
index 605dfd8872..0000000000
--- a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source2/InLongMessageFactory.java
+++ /dev/null
@@ -1,85 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.inlong.dataproxy.source2;
-
-import io.netty.channel.ChannelInboundHandlerAdapter;
-import io.netty.channel.ChannelInitializer;
-import io.netty.channel.socket.SocketChannel;
-import io.netty.handler.codec.LengthFieldBasedFrameDecoder;
-import io.netty.handler.codec.http.HttpObjectAggregator;
-import io.netty.handler.codec.http.HttpServerCodec;
-import io.netty.handler.timeout.ReadTimeoutHandler;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.lang.reflect.Constructor;
-import java.util.concurrent.TimeUnit;
-
-public class InLongMessageFactory extends ChannelInitializer<SocketChannel> {
-
-    public static final int INLONG_LENGTH_FIELD_OFFSET = 0;
-    public static final int INLONG_LENGTH_FIELD_LENGTH = 4;
-    public static final int INLONG_LENGTH_ADJUSTMENT = 0;
-    public static final int INLONG_INITIAL_BYTES_TO_STRIP = 0;
-    public static final boolean DEFAULT_FAIL_FAST = true;
-    private static final Logger LOG = LoggerFactory.getLogger(InLongMessageFactory.class);
-    private final BaseSource source;
-
-    /**
-     * get server factory
-     *
-     * @param source
-     */
-    public InLongMessageFactory(BaseSource source) {
-        this.source = source;
-    }
-
-    @Override
-    protected void initChannel(SocketChannel ch) throws Exception {
-
-        if (source.getProtocolName()
-                .equalsIgnoreCase(SourceConstants.SRC_PROTOCOL_TYPE_TCP)) {
-            ch.pipeline().addLast("messageDecoder", new LengthFieldBasedFrameDecoder(
-                    source.getMaxMsgLength(), INLONG_LENGTH_FIELD_OFFSET, INLONG_LENGTH_FIELD_LENGTH,
-                    INLONG_LENGTH_ADJUSTMENT, INLONG_INITIAL_BYTES_TO_STRIP, DEFAULT_FAIL_FAST));
-            ch.pipeline().addLast("readTimeoutHandler",
-                    new ReadTimeoutHandler(source.getMaxReadIdleTimeMs(), TimeUnit.MILLISECONDS));
-        } else if (source.getProtocolName().equalsIgnoreCase(SourceConstants.SRC_PROTOCOL_TYPE_HTTP)) {
-            // add http message codec
-            ch.pipeline().addLast("msgCodec", new HttpServerCodec());
-            ch.pipeline().addLast("msgAggregator", new HttpObjectAggregator(source.getMaxMsgLength()));
-            ch.pipeline().addLast("readTimeoutHandler",
-                    new ReadTimeoutHandler(source.getMaxReadIdleTimeMs(), TimeUnit.MILLISECONDS));
-
-        }
-        // build message handler
-        if (source.getChannelProcessor() != null) {
-            try {
-                Class<? extends ChannelInboundHandlerAdapter> clazz =
-                        (Class<? extends ChannelInboundHandlerAdapter>) Class.forName(source.getMessageHandlerName());
-                Constructor<?> ctor = clazz.getConstructor(BaseSource.class);
-                ChannelInboundHandlerAdapter messageHandler =
-                        (ChannelInboundHandlerAdapter) ctor.newInstance(source);
-                ch.pipeline().addLast("messageHandler", messageHandler);
-            } catch (Exception e) {
-                LOG.error("{} newInstance {} failure!", source.getName(),
-                        source.getMessageHandlerName(), e);
-            }
-        }
-    }
-}
diff --git a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source2/InLongMessageHandler.java b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source2/InLongMessageHandler.java
deleted file mode 100644
index 0a2d27c1c8..0000000000
--- a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source2/InLongMessageHandler.java
+++ /dev/null
@@ -1,700 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.inlong.dataproxy.source2;
-
-import org.apache.inlong.common.enums.DataProxyErrCode;
-import org.apache.inlong.common.monitor.LogCounter;
-import org.apache.inlong.common.msg.AttributeConstants;
-import org.apache.inlong.common.msg.MsgType;
-import org.apache.inlong.dataproxy.config.CommonConfigHolder;
-import org.apache.inlong.dataproxy.config.ConfigManager;
-import org.apache.inlong.dataproxy.consts.AttrConstants;
-import org.apache.inlong.dataproxy.consts.ConfigConstants;
-import org.apache.inlong.dataproxy.consts.StatConstants;
-import org.apache.inlong.dataproxy.source2.v0msg.AbsV0MsgCodec;
-import org.apache.inlong.dataproxy.source2.v0msg.CodecBinMsg;
-import org.apache.inlong.dataproxy.source2.v0msg.CodecTextMsg;
-import org.apache.inlong.dataproxy.source2.v1msg.InlongTcpSourceCallback;
-import org.apache.inlong.dataproxy.utils.AddressUtils;
-import org.apache.inlong.dataproxy.utils.DateTimeUtils;
-import org.apache.inlong.sdk.commons.protocol.EventUtils;
-import org.apache.inlong.sdk.commons.protocol.ProxyEvent;
-import org.apache.inlong.sdk.commons.protocol.ProxyPackEvent;
-import org.apache.inlong.sdk.commons.protocol.ProxySdk;
-
-import io.netty.buffer.ByteBuf;
-import io.netty.buffer.ByteBufAllocator;
-import io.netty.buffer.Unpooled;
-import io.netty.channel.Channel;
-import io.netty.channel.ChannelHandlerContext;
-import io.netty.channel.ChannelInboundHandlerAdapter;
-import org.apache.commons.lang3.StringUtils;
-import org.apache.flume.Event;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.nio.charset.StandardCharsets;
-import java.util.List;
-import java.util.concurrent.TimeUnit;
-
-import static org.apache.inlong.dataproxy.source2.InLongMessageFactory.INLONG_LENGTH_FIELD_LENGTH;
-import static org.apache.inlong.dataproxy.source2.v0msg.MsgFieldConsts.BIN_HB_ATTRLEN_SIZE;
-import static org.apache.inlong.dataproxy.source2.v0msg.MsgFieldConsts.BIN_HB_BODYLEN_OFFSET;
-import static org.apache.inlong.dataproxy.source2.v0msg.MsgFieldConsts.BIN_HB_BODY_OFFSET;
-import static org.apache.inlong.dataproxy.source2.v0msg.MsgFieldConsts.BIN_HB_FIXED_CONTENT_SIZE;
-import static org.apache.inlong.dataproxy.source2.v0msg.MsgFieldConsts.BIN_HB_FORMAT_SIZE;
-import static org.apache.inlong.dataproxy.source2.v0msg.MsgFieldConsts.BIN_HB_TOTALLEN_SIZE;
-import static org.apache.inlong.dataproxy.source2.v0msg.MsgFieldConsts.BIN_HB_VERSION_OFFSET;
-import static org.apache.inlong.dataproxy.source2.v0msg.MsgFieldConsts.BIN_MSG_FIXED_CONTENT_SIZE;
-import static org.apache.inlong.dataproxy.source2.v0msg.MsgFieldConsts.BIN_MSG_MAGIC;
-import static org.apache.inlong.dataproxy.source2.v0msg.MsgFieldConsts.TXT_MSG_FIXED_CONTENT_SIZE;
-
-/**
- * Server message handler
- *
- */
-public class InLongMessageHandler extends ChannelInboundHandlerAdapter {
-
-    private static final Logger logger = LoggerFactory.getLogger(InLongMessageHandler.class);
-    // log print count
-    private static final LogCounter logCounter = new LogCounter(10, 100000, 30 * 1000);
-
-    private static final int INLONG_MSG_V1 = 1;
-
-    private static final ConfigManager configManager = ConfigManager.getInstance();
-    private final BaseSource source;
-
-    /**
-     * Constructor
-     *
-     * @param source AbstractSource
-     */
-    public InLongMessageHandler(BaseSource source) {
-        this.source = source;
-    }
-
-    @Override
-    public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception {
-        if (msg == null) {
-            source.fileMetricIncSumStats(StatConstants.EVENT_PKG_READABLE_EMPTY);
-            return;
-        }
-        ByteBuf cb = (ByteBuf) msg;
-        try {
-            int readableLength = cb.readableBytes();
-            if (readableLength == 0 && source.isFilterEmptyMsg()) {
-                cb.clear();
-                source.fileMetricIncSumStats(StatConstants.EVENT_PKG_READABLE_EMPTY);
-                return;
-            }
-            if (readableLength > source.getMaxMsgLength()) {
-                source.fileMetricIncSumStats(StatConstants.EVENT_PKG_READABLE_OVERMAX);
-                throw new Exception("Error msg, readableLength(" + readableLength +
-                        ") > max allowed message length (" + source.getMaxMsgLength() + ")");
-            }
-            // save index
-            cb.markReaderIndex();
-            // read total data length
-            int totalDataLen = cb.readInt();
-            if (readableLength < totalDataLen + INLONG_LENGTH_FIELD_LENGTH) {
-                // reset index when buffer is not satisfied.
-                cb.resetReaderIndex();
-                source.fileMetricIncSumStats(StatConstants.EVENT_PKG_READABLE_UNFILLED);
-                throw new Exception("Error msg, buffer is unfilled, readableLength="
-                        + readableLength + ", totalPackLength=" + totalDataLen + " + 4");
-            }
-            // read type
-            int msgTypeValue = cb.readByte();
-            if (msgTypeValue == 0x0) {
-                // process v1 messsages
-                msgTypeValue = cb.readByte();
-                if (msgTypeValue == INLONG_MSG_V1) {
-                    // decode version 1
-                    int bodyLength = totalDataLen - 2;
-                    processV1Msg(ctx, cb, bodyLength);
-                } else {
-                    // unknown message type
-                    source.fileMetricIncSumStats(StatConstants.EVENT_PKG_MSGTYPE_V1_INVALID);
-                    throw new Exception("Unknown V1 message version, version = " + msgTypeValue);
-                }
-            } else {
-                // process v0 messages
-                Channel channel = ctx.channel();
-                MsgType msgType = MsgType.valueOf(msgTypeValue);
-                final long msgRcvTime = System.currentTimeMillis();
-                if (MsgType.MSG_UNKNOWN == msgType) {
-                    source.fileMetricIncSumStats(StatConstants.EVENT_PKG_MSGTYPE_V0_INVALID);
-                    if (logger.isDebugEnabled()) {
-                        logger.debug("Received unknown message, channel {}", channel);
-                    }
-                    throw new Exception("Unknown V0 message type, type = " + msgTypeValue);
-                } else if (MsgType.MSG_HEARTBEAT == msgType) {
-                    // send response message
-                    flushV0MsgPackage(source, channel,
-                            buildHeartBeatMsgRspPackage(), MsgType.MSG_HEARTBEAT.name());
-                    return;
-                } else if (MsgType.MSG_BIN_HEARTBEAT == msgType) {
-                    procBinHeartbeatMsg(source, channel, cb, totalDataLen);
-                    return;
-                }
-                // process msgType in {2,3,4,5,6,7}
-                AbsV0MsgCodec msgCodec;
-                String strRemoteIP = AddressUtils.getChannelRemoteIP(channel);
-                // check whether totalDataLen is valid.
-                if (MsgType.MSG_BIN_MULTI_BODY == msgType) {
-                    if (totalDataLen < BIN_MSG_FIXED_CONTENT_SIZE) {
-                        source.fileMetricIncSumStats(StatConstants.EVENT_MSG_BIN_TOTALLEN_BELOWMIN);
-                        String errMsg = String.format("Malformed msg, totalDataLen(%d) < min bin7-msg length(%d)",
-                                totalDataLen, BIN_MSG_FIXED_CONTENT_SIZE);
-                        if (logger.isDebugEnabled()) {
-                            logger.debug(errMsg + ", channel {}", channel);
-                        }
-                        throw new Exception(errMsg);
-                    }
-                    msgCodec = new CodecBinMsg(totalDataLen, msgTypeValue, msgRcvTime, strRemoteIP);
-                } else {
-                    if (totalDataLen < TXT_MSG_FIXED_CONTENT_SIZE) {
-                        source.fileMetricIncSumStats(StatConstants.EVENT_MSG_TXT_TOTALLEN_BELOWMIN);
-                        String errMsg = String.format("Malformed msg, totalDataLen(%d) < min txt-msg length(%d)",
-                                totalDataLen, TXT_MSG_FIXED_CONTENT_SIZE);
-                        if (logger.isDebugEnabled()) {
-                            logger.debug(errMsg + ", channel {}", channel);
-                        }
-                        throw new Exception(errMsg);
-                    }
-                    msgCodec = new CodecTextMsg(totalDataLen, msgTypeValue, msgRcvTime, strRemoteIP);
-                }
-                // process request
-                processV0Msg(channel, cb, msgCodec);
-            }
-        } finally {
-            cb.release();
-        }
-    }
-
-    @Override
-    public void channelActive(ChannelHandlerContext ctx) throws Exception {
-        // check illegal ip
-        if (ConfigManager.getInstance().needChkIllegalIP()) {
-            String strRemoteIp = AddressUtils.getChannelRemoteIP(ctx.channel());
-            if (strRemoteIp != null
-                    && ConfigManager.getInstance().isIllegalIP(strRemoteIp)) {
-                source.fileMetricIncSumStats(StatConstants.EVENT_VISIT_ILLEGAL);
-                ctx.channel().disconnect();
-                ctx.channel().close();
-                logger.error(strRemoteIp + " is Illegal IP, so refuse it !");
-                return;
-            }
-        }
-        // check max allowed connection count
-        if (source.getAllChannels().size() >= source.getMaxConnections()) {
-            source.fileMetricIncSumStats(StatConstants.EVENT_VISIT_OVERMAX);
-            ctx.channel().disconnect();
-            ctx.channel().close();
-            logger.warn("{} refuse to connect = {} , connections = {}, maxConnections = {}",
-                    source.getName(), ctx.channel(), source.getAllChannels().size(), source.getMaxConnections());
-            return;
-        }
-        // add legal channel
-        source.getAllChannels().add(ctx.channel());
-        ctx.fireChannelActive();
-        source.fileMetricIncSumStats(StatConstants.EVENT_VISIT_LINKIN);
-        logger.info("{} added new channel {}, current connections = {}, maxConnections = {}",
-                source.getName(), ctx.channel(), source.getAllChannels().size(), source.getMaxConnections());
-    }
-
-    @Override
-    public void channelInactive(ChannelHandlerContext ctx) {
-        source.fileMetricIncSumStats(StatConstants.EVENT_VISIT_LINKOUT);
-        ctx.fireChannelInactive();
-        source.getAllChannels().remove(ctx.channel());
-    }
-
-    @Override
-    public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception {
-        source.fileMetricIncSumStats(StatConstants.EVENT_VISIT_EXCEPTION);
-        if (logCounter.shouldPrint()) {
-            logger.warn("{} received an exception from channel {}",
-                    source.getName(), ctx.channel(), cause);
-        }
-        if (ctx.channel() != null) {
-            source.getAllChannels().remove(ctx.channel());
-            try {
-                ctx.channel().disconnect();
-                ctx.channel().close();
-            } catch (Exception ex) {
-                //
-            }
-        }
-        ctx.close();
-    }
-
-    private void processV0Msg(Channel channel, ByteBuf cb, AbsV0MsgCodec msgCodec) throws Exception {
-        final StringBuilder strBuff = new StringBuilder(512);
-        // decode the request message
-        if (!msgCodec.descMsg(source, cb)) {
-            responseV0Msg(channel, msgCodec, strBuff);
-            return;
-        }
-        // check service status.
-        if (source.isRejectService()) {
-            source.fileMetricIncSumStats(StatConstants.EVENT_SERVICE_CLOSED);
-            msgCodec.setFailureInfo(DataProxyErrCode.SERVICE_CLOSED);
-            responseV0Msg(channel, msgCodec, strBuff);
-            return;
-        }
-        // check if the node is linked to the Manager.
-        if (!ConfigManager.getInstance().isMqClusterReady()) {
-            source.fileMetricIncSumStats(StatConstants.EVENT_SERVICE_SINK_UNREADY);
-            msgCodec.setFailureInfo(DataProxyErrCode.SINK_SERVICE_UNREADY);
-            responseV0Msg(channel, msgCodec, strBuff);
-            return;
-        }
-        // valid and fill extra fields.
-        if (!msgCodec.validAndFillFields(source, strBuff)) {
-            responseV0Msg(channel, msgCodec, strBuff);
-            return;
-        }
-        // build InLong event.
-        Event event = msgCodec.encEventPackage(source, channel);
-        // build metric data item
-        long longDataTime = msgCodec.getDataTimeMs() / 1000 / 60 / 10;
-        longDataTime = longDataTime * 1000 * 60 * 10;
-        String statsKey = strBuff.append(source.getProtocolName()).append(AttrConstants.SEP_HASHTAG)
-                .append(msgCodec.getGroupId()).append(AttrConstants.SEP_HASHTAG)
-                .append(msgCodec.getStreamId()).append(AttrConstants.SEP_HASHTAG)
-                .append(msgCodec.getStrRemoteIP()).append(AttrConstants.SEP_HASHTAG)
-                .append(source.getSrcHost()).append(AttrConstants.SEP_HASHTAG)
-                .append(msgCodec.getMsgProcType()).append(AttrConstants.SEP_HASHTAG)
-                .append(DateTimeUtils.ms2yyyyMMddHHmm(longDataTime)).append(AttrConstants.SEP_HASHTAG)
-                .append(DateTimeUtils.ms2yyyyMMddHHmm(msgCodec.getMsgRcvTime())).toString();
-        strBuff.delete(0, strBuff.length());
-        try {
-            source.getChannelProcessor().processEvent(event);
-            source.fileMetricIncSumStats(StatConstants.EVENT_MSG_V0_POST_SUCCESS);
-            source.fileMetricAddSuccCnt(statsKey, msgCodec.getMsgCount(), 1, msgCodec.getBodyLength());
-            source.addMetric(true, event.getBody().length, event);
-            if (msgCodec.isNeedResp() && !msgCodec.isOrderOrProxy()) {
-                msgCodec.setSuccessInfo();
-                responseV0Msg(channel, msgCodec, strBuff);
-            }
-        } catch (Throwable ex) {
-            source.fileMetricIncSumStats(StatConstants.EVENT_MSG_V0_POST_FAILURE);
-            source.fileMetricAddFailCnt(statsKey, 1);
-            source.addMetric(false, event.getBody().length, event);
-            if (msgCodec.isNeedResp() && !msgCodec.isOrderOrProxy()) {
-                msgCodec.setFailureInfo(DataProxyErrCode.PUT_EVENT_TO_CHANNEL_FAILURE,
-                        strBuff.append("Put event to channel failure: ").append(ex.getMessage()).toString());
-                strBuff.delete(0, strBuff.length());
-                responseV0Msg(channel, msgCodec, strBuff);
-            }
-            if (logCounter.shouldPrint()) {
-                logger.error("Error writing event to channel failure.", ex);
-            }
-        }
-    }
-
-    private void processV1Msg(ChannelHandlerContext ctx, ByteBuf cb, int bodyLength) throws Exception {
-        // read bytes
-        byte[] msgBytes = new byte[bodyLength];
-        cb.readBytes(msgBytes);
-        // decode
-        ProxySdk.MessagePack packObject = ProxySdk.MessagePack.parseFrom(msgBytes);
-        // reject service
-        if (source.isRejectService()) {
-            source.addMetric(false, 0, null);
-            source.fileMetricIncSumStats(StatConstants.EVENT_SERVICE_CLOSED);
-            this.responsePackage(ctx, ProxySdk.ResultCode.ERR_REJECT, packObject);
-            return;
-        }
-        // uncompress
-        List<ProxyEvent> events = EventUtils.decodeSdkPack(packObject);
-        // response success if event size is zero
-        if (events.size() == 0) {
-            this.responsePackage(ctx, ProxySdk.ResultCode.SUCCUSS, packObject);
-        }
-        // process
-        if (CommonConfigHolder.getInstance().isResponseAfterSave()) {
-            this.processAndWaitingSave(ctx, packObject, events);
-        } else {
-            this.processAndResponse(ctx, packObject, events);
-        }
-    }
-
-    /**
-     * responsePackage
-     *
-     * @param  ctx
-     * @param  code
-     * @throws Exception
-     */
-    private void responsePackage(ChannelHandlerContext ctx,
-            ProxySdk.ResultCode code,
-            ProxySdk.MessagePack packObject) throws Exception {
-        ProxySdk.ResponseInfo.Builder builder = ProxySdk.ResponseInfo.newBuilder();
-        builder.setResult(code);
-        ProxySdk.MessagePackHeader header = packObject.getHeader();
-        builder.setPackId(header.getPackId());
-
-        // encode
-        byte[] responseBytes = builder.build().toByteArray();
-        //
-        ByteBuf buffer = Unpooled.wrappedBuffer(responseBytes);
-        Channel remoteChannel = ctx.channel();
-        if (remoteChannel.isWritable()) {
-            remoteChannel.write(buffer);
-        } else {
-            buffer.release();
-            logger.warn("Send buffer2 is not writable, disconnect {}", remoteChannel);
-            throw new Exception("Send buffer2 is not writable, disconnect " + remoteChannel);
-        }
-    }
-
-    /**
-     * processAndWaitingSave
-     * @param ctx
-     * @param packObject
-     * @param events
-     * @throws Exception
-     */
-    private void processAndWaitingSave(ChannelHandlerContext ctx,
-            ProxySdk.MessagePack packObject,
-            List<ProxyEvent> events) throws Exception {
-        ProxySdk.MessagePackHeader header = packObject.getHeader();
-        InlongTcpSourceCallback callback = new InlongTcpSourceCallback(ctx, header);
-        String inlongGroupId = header.getInlongGroupId();
-        String inlongStreamId = header.getInlongStreamId();
-        ProxyPackEvent packEvent = new ProxyPackEvent(inlongGroupId, inlongStreamId, events, callback);
-        // put to channel
-        try {
-            source.getChannelProcessor().processEvent(packEvent);
-            events.forEach(event -> {
-                source.addMetric(true, event.getBody().length, event);
-                source.fileMetricIncSumStats(StatConstants.EVENT_MSG_V1_POST_SUCCESS);
-            });
-            boolean awaitResult = callback.getLatch().await(
-                    CommonConfigHolder.getInstance().getMaxResAfterSaveTimeout(), TimeUnit.MILLISECONDS);
-            if (!awaitResult) {
-                if (!callback.getHasResponsed().getAndSet(true)) {
-                    this.responsePackage(ctx, ProxySdk.ResultCode.ERR_REJECT, packObject);
-                }
-            }
-        } catch (Throwable ex) {
-            logger.error("Process Controller Event error can't write event to channel.", ex);
-            events.forEach(event -> {
-                source.addMetric(false, event.getBody().length, event);
-                source.fileMetricIncSumStats(StatConstants.EVENT_MSG_V1_POST_DROPPED);
-            });
-            if (!callback.getHasResponsed().getAndSet(true)) {
-                this.responsePackage(ctx, ProxySdk.ResultCode.ERR_REJECT, packObject);
-            }
-        }
-    }
-
-    /**
-     * processAndResponse
-     * @param ctx
-     * @param packObject
-     * @param events
-     * @throws Exception
-     */
-    private void processAndResponse(ChannelHandlerContext ctx,
-            ProxySdk.MessagePack packObject,
-            List<ProxyEvent> events) throws Exception {
-        for (ProxyEvent event : events) {
-            // get configured topic name
-            String topic = configManager.getTopicName(event.getInlongGroupId(), event.getInlongStreamId());
-            if (StringUtils.isBlank(topic)) {
-                source.fileMetricIncSumStats(StatConstants.EVENT_CONFIG_TOPIC_MISSING);
-                source.addMetric(false, event.getBody().length, event);
-                this.responsePackage(ctx, ProxySdk.ResultCode.ERR_ID_ERROR, packObject);
-                return;
-            }
-            event.setTopic(topic);
-            // put to channel
-            try {
-                source.getChannelProcessor().processEvent(event);
-                source.addMetric(true, event.getBody().length, event);
-                source.fileMetricIncSumStats(StatConstants.EVENT_MSG_V1_POST_SUCCESS);
-            } catch (Throwable ex) {
-                logger.error("Process Controller Event error can't write event to channel.", ex);
-                source.addMetric(false, event.getBody().length, event);
-                this.responsePackage(ctx, ProxySdk.ResultCode.ERR_REJECT, packObject);
-                source.fileMetricIncSumStats(StatConstants.EVENT_MSG_V1_POST_DROPPED);
-                return;
-            }
-        }
-        this.responsePackage(ctx, ProxySdk.ResultCode.SUCCUSS, packObject);
-    }
-
-    /**
-     *  Return response to client in source
-     */
-    private void responseV0Msg(Channel channel, AbsV0MsgCodec msgObj, StringBuilder strBuff) throws Exception {
-        // check channel status
-        if (channel == null || !channel.isWritable()) {
-            source.fileMetricIncSumStats(StatConstants.EVENT_REMOTE_UNWRITABLE);
-            if (logCounter.shouldPrint()) {
-                logger.warn("Prepare send msg but channel full, msgType={}, attr={}, channel={}",
-                        msgObj.getMsgType(), msgObj.getAttr(), channel);
-            }
-            throw new Exception("Prepare send msg but channel full");
-        }
-        // check whether return response message
-        if (!msgObj.isNeedResp()) {
-            return;
-        }
-        // build return attribute string
-        strBuff.append(ConfigConstants.DATAPROXY_IP_KEY)
-                .append(AttributeConstants.KEY_VALUE_SEPARATOR).append(source.getSrcHost());
-        if (msgObj.getErrCode() != DataProxyErrCode.SUCCESS) {
-            strBuff.append(AttributeConstants.SEPARATOR).append(AttributeConstants.MESSAGE_PROCESS_ERRCODE)
-                    .append(AttributeConstants.KEY_VALUE_SEPARATOR).append(msgObj.getErrCode().getErrCodeStr());
-            if (StringUtils.isNotEmpty(msgObj.getErrMsg())) {
-                strBuff.append(AttributeConstants.SEPARATOR).append(AttributeConstants.MESSAGE_PROCESS_ERRMSG)
-                        .append(AttributeConstants.KEY_VALUE_SEPARATOR).append(msgObj.getErrMsg());
-            }
-        }
-        if (StringUtils.isNotEmpty(msgObj.getAttr())) {
-            strBuff.append(AttributeConstants.SEPARATOR).append(msgObj.getAttr());
-        }
-        // build and send response message
-        ByteBuf retData;
-        MsgType msgType = MsgType.valueOf(msgObj.getMsgType());
-        if (MsgType.MSG_BIN_MULTI_BODY.equals(msgType)) {
-            retData = buildBinMsgRspPackage(strBuff.toString(), msgObj.getUniq());
-        } else {
-            retData = buildTxtMsgRspPackage(msgType, strBuff.toString(), msgObj);
-        }
-        strBuff.delete(0, strBuff.length());
-        flushV0MsgPackage(source, channel, retData, msgObj.getAttr());
-    }
-
-    /**
-     * extract and process bin heart beat msg, message type is 8
-     */
-    private void procBinHeartbeatMsg(BaseSource source, Channel channel,
-            ByteBuf cb, int totalDataLen) throws Exception {
-        // Check if the message is complete and legal
-        if (totalDataLen < BIN_HB_FIXED_CONTENT_SIZE) {
-            source.fileMetricIncSumStats(StatConstants.EVENT_MSG_HB_TOTALLEN_BELOWMIN);
-            String errMsg = String.format("Malformed msg, totalDataLen(%d) < min hb-msg length(%d)",
-                    totalDataLen, BIN_HB_FIXED_CONTENT_SIZE);
-            if (logger.isDebugEnabled()) {
-                logger.debug(errMsg + ", channel {}", channel);
-            }
-            throw new Exception(errMsg);
-        }
-        // check validation
-        int msgHeadPos = cb.readerIndex() - 5;
-        int bodyLen = cb.getInt(msgHeadPos + BIN_HB_BODYLEN_OFFSET);
-        int attrLen = cb.getShort(msgHeadPos + BIN_HB_BODY_OFFSET + bodyLen);
-        int msgMagic = cb.getUnsignedShort(msgHeadPos
-                + BIN_HB_BODY_OFFSET + bodyLen + BIN_HB_ATTRLEN_SIZE + attrLen);
-        if (msgMagic != BIN_MSG_MAGIC) {
-            source.fileMetricIncSumStats(StatConstants.EVENT_MSG_HB_MAGIC_UNEQUAL);
-            String errMsg = String.format(
-                    "Malformed msg, msgMagic(%d) != %d", msgMagic, BIN_MSG_MAGIC);
-            if (logger.isDebugEnabled()) {
-                logger.debug(errMsg + ", channel {}", channel);
-            }
-            throw new Exception(errMsg);
-        }
-        if (totalDataLen + BIN_HB_TOTALLEN_SIZE < (bodyLen + attrLen + BIN_HB_FORMAT_SIZE)) {
-            source.fileMetricIncSumStats(StatConstants.EVENT_MSG_HB_LEN_MALFORMED);
-            String errMsg = String.format(
-                    "Malformed msg, bodyLen(%d) + attrLen(%d) > totalDataLen(%d)",
-                    bodyLen, attrLen, totalDataLen);
-            if (logger.isDebugEnabled()) {
-                logger.debug(errMsg + ", channel {}", channel);
-            }
-            throw new Exception(errMsg);
-        }
-        // read message content
-        byte version = cb.getByte(msgHeadPos + BIN_HB_VERSION_OFFSET);
-        byte[] attrData = null;
-        if (attrLen > 0) {
-            attrData = new byte[attrLen];
-            cb.getBytes(msgHeadPos + BIN_HB_BODY_OFFSET
-                    + bodyLen + BIN_HB_ATTRLEN_SIZE, attrData, 0, attrLen);
-        }
-        // build and send response message
-        flushV0MsgPackage(source, channel,
-                buildHBRspPackage(attrData, version, 0), MsgType.MSG_BIN_HEARTBEAT.name());
-    }
-
-    /**
-     * Build bin-msg response message ByteBuf
-     *
-     * @param attrs   the return attribute
-     * @param uniqVal sequence Id
-     * @return ByteBuf
-     */
-    public static ByteBuf buildBinMsgRspPackage(String attrs, long uniqVal) {
-        // calculate total length
-        // binTotalLen = mstType + uniq + attrsLen + attrs + magic
-        int binTotalLen = 1 + 4 + 2 + 2;
-        if (null != attrs) {
-            binTotalLen += attrs.length();
-        }
-        // allocate buffer and write fields
-        ByteBuf binBuffer = ByteBufAllocator.DEFAULT.buffer(4 + binTotalLen);
-        binBuffer.writeInt(binTotalLen);
-        binBuffer.writeByte(MsgType.MSG_BIN_MULTI_BODY.getValue());
-        byte[] uniq = new byte[4];
-        uniq[0] = (byte) ((uniqVal >> 24) & 0xFF);
-        uniq[1] = (byte) ((uniqVal >> 16) & 0xFF);
-        uniq[2] = (byte) ((uniqVal >> 8) & 0xFF);
-        uniq[3] = (byte) (uniqVal & 0xFF);
-        binBuffer.writeBytes(uniq);
-        if (null != attrs) {
-            binBuffer.writeShort(attrs.length());
-            binBuffer.writeBytes(attrs.getBytes(StandardCharsets.UTF_8));
-        } else {
-            binBuffer.writeShort(0x0);
-        }
-        binBuffer.writeShort(0xee01);
-        return binBuffer;
-    }
-
-    /**
-     * Build default-msg response message ByteBuf
-     *
-     * @param msgType  the message type
-     * @param attrs    the return attribute
-     * @return ByteBuf
-     */
-    public static ByteBuf buildTxtMsgRspPackage(MsgType msgType, String attrs) {
-        int attrsLen = 0;
-        int bodyLen = 0;
-        if (attrs != null) {
-            attrsLen = attrs.length();
-        }
-        // backTotalLen = mstType + bodyLen + body + attrsLen + attrs
-        int backTotalLen = 1 + 4 + bodyLen + 4 + attrsLen;
-        ByteBuf buffer = ByteBufAllocator.DEFAULT.buffer(4 + backTotalLen);
-        buffer.writeInt(backTotalLen);
-        buffer.writeByte(msgType.getValue());
-        buffer.writeInt(bodyLen);
-        buffer.writeInt(attrsLen);
-        if (attrsLen > 0) {
-            buffer.writeBytes(attrs.getBytes(StandardCharsets.UTF_8));
-        }
-        return buffer;
-    }
-
-    /**
-     * Build default-msg response message ByteBuf
-     *
-     * @param msgType  the message type
-     * @param attrs    the return attribute
-     * @param msgObj   the request message object
-     * @return ByteBuf
-     */
-    private ByteBuf buildTxtMsgRspPackage(MsgType msgType, String attrs, AbsV0MsgCodec msgObj) {
-        int attrsLen = 0;
-        int bodyLen = 0;
-        byte[] backBody = null;
-        if (attrs != null) {
-            attrsLen = attrs.length();
-        }
-        if (MsgType.MSG_ORIGINAL_RETURN.equals(msgType)) {
-            backBody = msgObj.getOrigBody();
-            if (backBody != null) {
-                bodyLen = backBody.length;
-            }
-        }
-        // backTotalLen = mstType + bodyLen + body + attrsLen + attrs
-        int backTotalLen = 1 + 4 + bodyLen + 4 + attrsLen;
-        ByteBuf buffer = ByteBufAllocator.DEFAULT.buffer(4 + backTotalLen);
-        buffer.writeInt(backTotalLen);
-        buffer.writeByte(msgType.getValue());
-        buffer.writeInt(bodyLen);
-        if (bodyLen > 0) {
-            buffer.writeBytes(backBody);
-        }
-        buffer.writeInt(attrsLen);
-        if (attrsLen > 0) {
-            buffer.writeBytes(attrs.getBytes(StandardCharsets.UTF_8));
-        }
-        return buffer;
-    }
-
-    /**
-     * Build heartbeat response message ByteBuf
-     *
-     * @param attrData  the attribute data
-     * @param version   the version
-     * @param loadValue the node load value
-     * @return ByteBuf
-     */
-    private ByteBuf buildHBRspPackage(byte[] attrData, byte version, int loadValue) {
-        // calculate total length
-        // binTotalLen = mstType + dataTime + version + bodyLen + body + attrsLen + attrs + magic
-        int binTotalLen = 1 + 4 + 1 + 4 + 2 + 2 + 2;
-        if (null != attrData) {
-            binTotalLen += attrData.length;
-        }
-        // check load value
-        if (loadValue == 0 || loadValue == (-1)) {
-            loadValue = 0xffff;
-        }
-        // allocate buffer and write fields
-        ByteBuf binBuffer = ByteBufAllocator.DEFAULT.buffer(4 + binTotalLen);
-        binBuffer.writeInt(binTotalLen);
-        binBuffer.writeByte(MsgType.MSG_BIN_HEARTBEAT.getValue());
-        binBuffer.writeInt((int) (System.currentTimeMillis() / 1000));
-        binBuffer.writeByte(version);
-        binBuffer.writeInt(2);
-        binBuffer.writeShort(loadValue);
-        if (null != attrData) {
-            binBuffer.writeShort(attrData.length);
-            binBuffer.writeBytes(attrData);
-        } else {
-            binBuffer.writeShort(0x0);
-        }
-        binBuffer.writeShort(0xee01);
-        return binBuffer;
-    }
-
-    /**
-     * Build hearbeat(1)-msg response message ByteBuf
-     *
-     * @return ByteBuf
-     */
-    private ByteBuf buildHeartBeatMsgRspPackage() {
-        ByteBuf buffer = ByteBufAllocator.DEFAULT.buffer(5);
-        // magic data
-        buffer.writeBytes(new byte[]{0, 0, 0, 1, 1});
-        return buffer;
-    }
-
-    private void flushV0MsgPackage(BaseSource source, Channel channel,
-            ByteBuf binBuffer, String orgAttr) throws Exception {
-        if (channel == null || !channel.isWritable()) {
-            // release allocated ByteBuf
-            binBuffer.release();
-            source.fileMetricIncSumStats(StatConstants.EVENT_REMOTE_UNWRITABLE);
-            if (logCounter.shouldPrint()) {
-                logger.warn("Send msg but channel full, attr={}, channel={}", orgAttr, channel);
-            }
-            throw new Exception("Send response but channel full");
-        }
-        channel.writeAndFlush(binBuffer);
-    }
-}
diff --git a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source2/SimpleTcpSource.java b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source2/SimpleTcpSource.java
deleted file mode 100644
index ef298ce805..0000000000
--- a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source2/SimpleTcpSource.java
+++ /dev/null
@@ -1,127 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.inlong.dataproxy.source2;
-
-import org.apache.inlong.dataproxy.config.ConfigManager;
-import org.apache.inlong.dataproxy.utils.ConfStringUtils;
-import org.apache.inlong.dataproxy.utils.EventLoopUtil;
-
-import com.google.common.base.Preconditions;
-import io.netty.bootstrap.ServerBootstrap;
-import io.netty.buffer.ByteBufAllocator;
-import io.netty.channel.ChannelOption;
-import io.netty.util.concurrent.DefaultThreadFactory;
-import org.apache.flume.Context;
-import org.apache.flume.conf.Configurable;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.net.InetSocketAddress;
-
-/**
- * Simple tcp source
- */
-public class SimpleTcpSource extends BaseSource implements Configurable {
-
-    private static final Logger logger = LoggerFactory.getLogger(SimpleTcpSource.class);
-
-    private ServerBootstrap bootstrap;
-    private boolean tcpNoDelay;
-    private boolean tcpKeepAlive;
-    private int highWaterMark;
-    private boolean enableBusyWait;
-
-    public SimpleTcpSource() {
-        super();
-        ConfigManager.getInstance().regIPVisitConfigChgCallback(this);
-    }
-
-    @Override
-    public void configure(Context context) {
-        logger.info("Source {} context is {}", getName(), context);
-        super.configure(context);
-        // get tcp no-delay parameter
-        this.tcpNoDelay = context.getBoolean(SourceConstants.SRCCXT_TCP_NO_DELAY,
-                SourceConstants.VAL_DEF_TCP_NO_DELAY);
-        // get tcp keep-alive parameter
-        this.tcpKeepAlive = context.getBoolean(SourceConstants.SRCCXT_TCP_KEEP_ALIVE,
-                SourceConstants.VAL_DEF_TCP_KEEP_ALIVE);
-        // get tcp enable busy-wait
-        this.enableBusyWait = context.getBoolean(SourceConstants.SRCCXT_TCP_ENABLE_BUSY_WAIT,
-                SourceConstants.VAL_DEF_TCP_ENABLE_BUSY_WAIT);
-        // get tcp high watermark
-        this.highWaterMark = ConfStringUtils.getIntValue(context,
-                SourceConstants.SRCCXT_TCP_HIGH_WATER_MARK, SourceConstants.VAL_DEF_TCP_HIGH_WATER_MARK);
-        Preconditions.checkArgument((this.highWaterMark >= SourceConstants.VAL_MIN_TCP_HIGH_WATER_MARK),
-                SourceConstants.VAL_DEF_TCP_HIGH_WATER_MARK + " must be >= "
-                        + SourceConstants.VAL_MIN_TCP_HIGH_WATER_MARK);
-    }
-
-    @Override
-    public synchronized void startSource() {
-        logger.info("start " + this.getName());
-        // build accept group
-        this.acceptorGroup = EventLoopUtil.newEventLoopGroup(maxAcceptThreads, enableBusyWait,
-                new DefaultThreadFactory(this.getName() + "-boss-group"));
-        // build worker group
-        this.workerGroup = EventLoopUtil.newEventLoopGroup(maxWorkerThreads, enableBusyWait,
-                new DefaultThreadFactory(this.getName() + "-worker-group"));
-        // init boostrap
-        bootstrap = new ServerBootstrap();
-        if (conLinger >= 0) {
-            bootstrap.option(ChannelOption.SO_LINGER, conLinger);
-        }
-        bootstrap.option(ChannelOption.SO_BACKLOG, conBacklog);
-        bootstrap.option(ChannelOption.SO_REUSEADDR, reuseAddress);
-        bootstrap.childOption(ChannelOption.ALLOCATOR, ByteBufAllocator.DEFAULT);
-        bootstrap.childOption(ChannelOption.TCP_NODELAY, tcpNoDelay);
-        bootstrap.childOption(ChannelOption.SO_KEEPALIVE, tcpKeepAlive);
-        bootstrap.childOption(ChannelOption.SO_RCVBUF, maxRcvBufferSize);
-        bootstrap.childOption(ChannelOption.SO_SNDBUF, maxSendBufferSize);
-        bootstrap.childOption(ChannelOption.WRITE_BUFFER_HIGH_WATER_MARK, highWaterMark);
-        bootstrap.channel(EventLoopUtil.getServerSocketChannelClass(workerGroup));
-        EventLoopUtil.enableTriggeredMode(bootstrap);
-        bootstrap.group(acceptorGroup, workerGroup);
-        bootstrap.childHandler(this.getChannelInitializerFactory());
-        try {
-            if (srcHost == null) {
-                channelFuture = bootstrap.bind(new InetSocketAddress(srcPort)).sync();
-            } else {
-                channelFuture = bootstrap.bind(new InetSocketAddress(srcHost, srcPort)).sync();
-            }
-        } catch (Exception e) {
-            logger.error("Source {} bind ({}:{}) error, program will exit! e = {}",
-                    this.getName(), srcHost, srcPort, e);
-            System.exit(-1);
-        }
-        ConfigManager.getInstance().addSourceReportInfo(
-                srcHost, String.valueOf(srcPort), getProtocolName().toUpperCase());
-        logger.info("Source {} started at ({}:{})!", this.getName(), srcHost, srcPort);
-    }
-
-    @Override
-    public synchronized void stop() {
-        super.stop();
-    }
-
-    @Override
-    public String getProtocolName() {
-        return SourceConstants.SRC_PROTOCOL_TYPE_TCP;
-    }
-
-}
diff --git a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source2/SimpleUdpSource.java b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source2/SimpleUdpSource.java
deleted file mode 100644
index 7010c4d933..0000000000
--- a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source2/SimpleUdpSource.java
+++ /dev/null
@@ -1,89 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.inlong.dataproxy.source2;
-
-import org.apache.inlong.dataproxy.config.ConfigManager;
-
-import io.netty.bootstrap.Bootstrap;
-import io.netty.channel.ChannelOption;
-import io.netty.channel.socket.nio.NioDatagramChannel;
-import org.apache.flume.Context;
-import org.apache.flume.conf.Configurable;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.net.InetSocketAddress;
-
-public class SimpleUdpSource extends BaseSource implements Configurable {
-
-    private static final Logger logger = LoggerFactory
-            .getLogger(SimpleUdpSource.class);
-
-    private Bootstrap bootstrap;
-
-    public SimpleUdpSource() {
-        super();
-    }
-
-    @Override
-    public void configure(Context context) {
-        logger.info("Source {} context is {}", getName(), context);
-        super.configure(context);
-    }
-
-    @SuppressWarnings({"unchecked", "rawtypes"})
-    @Override
-    public void startSource() {
-        // setup Netty server
-        logger.info("start " + this.getName());
-        bootstrap = new Bootstrap();
-        bootstrap.channel(NioDatagramChannel.class);
-        if (conLinger >= 0) {
-            bootstrap.option(ChannelOption.SO_LINGER, conLinger);
-        }
-        bootstrap.option(ChannelOption.SO_BACKLOG, conBacklog);
-        bootstrap.option(ChannelOption.SO_REUSEADDR, reuseAddress);
-        bootstrap.option(ChannelOption.SO_RCVBUF, maxRcvBufferSize);
-        bootstrap.option(ChannelOption.SO_SNDBUF, maxSendBufferSize);
-        bootstrap.handler(this.getChannelInitializerFactory());
-        try {
-            if (srcHost == null) {
-                channelFuture = bootstrap.bind(new InetSocketAddress(srcPort)).sync();
-            } else {
-                channelFuture = bootstrap.bind(new InetSocketAddress(srcHost, srcPort)).sync();
-            }
-        } catch (Exception e) {
-            logger.error("Source {} bind ({}:{}) error, program will exit! e = {}",
-                    this.getName(), srcHost, srcPort, e);
-            System.exit(-1);
-        }
-        ConfigManager.getInstance().addSourceReportInfo(
-                srcHost, String.valueOf(srcPort), getProtocolName().toUpperCase());
-        logger.info("Source {} started at ({}:{})!", this.getName(), srcHost, srcPort);
-    }
-
-    @Override
-    public void stop() {
-        super.stop();
-    }
-
-    @Override
-    public String getProtocolName() {
-        return SourceConstants.SRC_PROTOCOL_TYPE_UDP;
-    }
-}