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;
- }
-}