You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@inlong.apache.org by do...@apache.org on 2022/01/21 08:17:43 UTC
[incubator-inlong] branch master updated: [INLONG-2255] DataProxy support audit sdk. (#2260)
This is an automated email from the ASF dual-hosted git repository.
dockerzhang pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-inlong.git
The following commit(s) were added to refs/heads/master by this push:
new 57cf8a7 [INLONG-2255] DataProxy support audit sdk. (#2260)
57cf8a7 is described below
commit 57cf8a740593a8dd123a9eb9455909bd29f95ecd
Author: 卢春亮 <94...@qq.com>
AuthorDate: Fri Jan 21 16:17:23 2022 +0800
[INLONG-2255] DataProxy support audit sdk. (#2260)
---
inlong-dataproxy/dataproxy-source/pom.xml | 5 +
.../dataproxy/metrics/DataProxyMetricItem.java | 48 ++++++-
.../inlong/dataproxy/metrics/audit/AuditUtils.java | 149 +++++++++++++++++++++
.../apache/inlong/dataproxy/node/Application.java | 11 +-
.../org/apache/inlong/dataproxy/sink/MetaSink.java | 23 ++--
.../dataproxy/sink/SimpleMessageTubeSink.java | 24 ++--
.../pulsar/federation/PulsarProducerCluster.java | 22 +--
.../dataproxy/source/ServerMessageHandler.java | 47 +++----
.../dataproxy/source/SimpleMessageHandler.java | 37 +++--
9 files changed, 284 insertions(+), 82 deletions(-)
diff --git a/inlong-dataproxy/dataproxy-source/pom.xml b/inlong-dataproxy/dataproxy-source/pom.xml
index 477d39d..2e6730a 100644
--- a/inlong-dataproxy/dataproxy-source/pom.xml
+++ b/inlong-dataproxy/dataproxy-source/pom.xml
@@ -43,5 +43,10 @@
<version>${project.version}</version>
<scope>compile</scope>
</dependency>
+ <dependency>
+ <groupId>org.apache.inlong</groupId>
+ <artifactId>audit-sdk</artifactId>
+ <version>${project.version}</version>
+ </dependency>
</dependencies>
</project>
\ No newline at end of file
diff --git a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/metrics/DataProxyMetricItem.java b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/metrics/DataProxyMetricItem.java
index dfb4cbe..51bb03d 100644
--- a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/metrics/DataProxyMetricItem.java
+++ b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/metrics/DataProxyMetricItem.java
@@ -25,6 +25,10 @@ import org.apache.inlong.commons.config.metrics.CountMetric;
import org.apache.inlong.commons.config.metrics.Dimension;
import org.apache.inlong.commons.config.metrics.MetricDomain;
import org.apache.inlong.commons.config.metrics.MetricItem;
+import org.apache.inlong.dataproxy.config.holder.CommonPropertiesHolder;
+import org.apache.inlong.dataproxy.consts.AttributeConstants;
+import org.apache.inlong.dataproxy.consts.ConfigConstants;
+import org.apache.inlong.dataproxy.metrics.audit.AuditUtils;
import org.apache.inlong.dataproxy.utils.Constants;
/**
@@ -112,13 +116,53 @@ public class DataProxyMetricItem extends MetricItem {
*/
public static void fillInlongId(Event event, Map<String, String> dimensions) {
Map<String, String> headers = event.getHeaders();
- String inlongGroupId = headers.getOrDefault(Constants.INLONG_GROUP_ID, "");
- String inlongStreamId = headers.getOrDefault(Constants.INLONG_STREAM_ID, "");
+ String inlongGroupId = getInlongGroupId(headers);
+ String inlongStreamId = getInlongStreamId(headers);
dimensions.put(KEY_INLONG_GROUP_ID, inlongGroupId);
dimensions.put(KEY_INLONG_STREAM_ID, inlongStreamId);
}
/**
+ * fillAuditFormatTime
+ *
+ * @param event
+ * @param dimensions
+ */
+ public static void fillAuditFormatTime(Event event, Map<String, String> dimensions) {
+ long msgTime = AuditUtils.getLogTime(event);
+ long auditFormatTime = msgTime - msgTime % CommonPropertiesHolder.getAuditFormatInterval();
+ dimensions.put(DataProxyMetricItem.KEY_MESSAGE_TIME, String.valueOf(auditFormatTime));
+ }
+
+ /**
+ * getInlongGroupId
+ *
+ * @param headers
+ * @return
+ */
+ public static String getInlongGroupId(Map<String, String> headers) {
+ String inlongGroupId = headers.get(Constants.INLONG_GROUP_ID);
+ if (inlongGroupId == null) {
+ inlongGroupId = headers.getOrDefault(ConfigConstants.TOPIC_KEY, "");
+ }
+ return inlongGroupId;
+ }
+
+ /**
+ * getInlongStreamId
+ *
+ * @param headers
+ * @return
+ */
+ public static String getInlongStreamId(Map<String, String> headers) {
+ String inlongStreamId = headers.get(Constants.INLONG_STREAM_ID);
+ if (inlongStreamId == null) {
+ inlongStreamId = headers.getOrDefault(AttributeConstants.INTERFACE_ID, "");
+ }
+ return inlongStreamId;
+ }
+
+ /**
* get clusterId
*
* @return the clusterId
diff --git a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/metrics/audit/AuditUtils.java b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/metrics/audit/AuditUtils.java
new file mode 100644
index 0000000..95674f5
--- /dev/null
+++ b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/metrics/audit/AuditUtils.java
@@ -0,0 +1,149 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.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.metrics.audit;
+
+import java.util.HashSet;
+import java.util.Map;
+
+import org.apache.commons.lang.BooleanUtils;
+import org.apache.commons.lang.math.NumberUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.flume.Event;
+import org.apache.inlong.audit.AuditImp;
+import org.apache.inlong.audit.util.AuditConfig;
+import org.apache.inlong.dataproxy.config.ConfigManager;
+import org.apache.inlong.dataproxy.config.holder.CommonPropertiesHolder;
+import org.apache.inlong.dataproxy.consts.AttributeConstants;
+import org.apache.inlong.dataproxy.metrics.DataProxyMetricItem;
+import org.apache.inlong.dataproxy.utils.Constants;
+
+/**
+ *
+ * AuditUtils
+ */
+public class AuditUtils {
+
+ public static final String AUDIT_KEY_FILE_PATH = "audit.filePath";
+ public static final String AUDIT_DEFAULT_FILE_PATH = "/data/inlong/audit/";
+ public static final String AUDIT_KEY_MAX_CACHE_ROWS = "audit.maxCacheRows";
+ public static final int AUDIT_DEFAULT_MAX_CACHE_ROWS = 2000000;
+ public static final String AUDIT_KEY_PROXYS = "audit.proxys";
+ public static final String AUDIT_KEY_IS_AUDIT = "audit.isAudit";
+
+ public static final int AUDIT_ID_DATAPROXY_READ_SUCCESS = 5;
+ public static final int AUDIT_ID_DATAPROXY_SEND_SUCCESS = 6;
+
+ private static boolean IS_AUDIT = true;
+
+ /**
+ * initAudit
+ */
+ public static void initAudit() {
+ // IS_AUDIT
+ IS_AUDIT = BooleanUtils.toBoolean(ConfigManager.getInstance().getCommonProperties().get(AUDIT_KEY_IS_AUDIT));
+ if (IS_AUDIT) {
+ // AuditProxy
+ String strIpPorts = ConfigManager.getInstance().getCommonProperties().get(AUDIT_KEY_PROXYS);
+ HashSet<String> proxys = new HashSet<>();
+ if (!StringUtils.isBlank(strIpPorts)) {
+ String[] ipPorts = strIpPorts.split("\\s+");
+ for (String ipPort : ipPorts) {
+ proxys.add(ipPort);
+ }
+ }
+ AuditImp.getInstance().setAuditProxy(proxys);
+ // AuditConfig
+ String filePath = ConfigManager.getInstance().getCommonProperties().getOrDefault(AUDIT_KEY_FILE_PATH,
+ AUDIT_DEFAULT_FILE_PATH);
+ int maxCacheRow = NumberUtils.toInt(
+ ConfigManager.getInstance().getCommonProperties().get(AUDIT_KEY_MAX_CACHE_ROWS),
+ AUDIT_DEFAULT_MAX_CACHE_ROWS);
+ AuditConfig auditConfig = new AuditConfig(filePath, maxCacheRow);
+ AuditImp.getInstance().setAuditConfig(auditConfig);
+ }
+ }
+
+ /**
+ * add
+ *
+ * @param auditID
+ * @param event
+ */
+ public static void add(int auditID, Event event) {
+ if (IS_AUDIT && event != null) {
+ Map<String, String> headers = event.getHeaders();
+ String inlongGroupId = DataProxyMetricItem.getInlongGroupId(headers);
+ String inlongStreamId = DataProxyMetricItem.getInlongStreamId(headers);
+ long logTime = getLogTime(headers);
+ AuditImp.getInstance().add(auditID, inlongGroupId, inlongStreamId, logTime, 1, event.getBody().length);
+ }
+ }
+
+ /**
+ * getLogTime
+ *
+ * @param headers
+ * @return
+ */
+ public static long getLogTime(Map<String, String> headers) {
+ String strLogTime = headers.get(Constants.HEADER_KEY_MSG_TIME);
+ if (strLogTime == null) {
+ strLogTime = headers.get(AttributeConstants.DATA_TIME);
+ }
+ if (strLogTime == null) {
+ return System.currentTimeMillis();
+ }
+ long logTime = NumberUtils.toLong(strLogTime, 0);
+ if (logTime == 0) {
+ logTime = System.currentTimeMillis();
+ }
+ return logTime;
+ }
+
+ /**
+ * getLogTime
+ *
+ * @param event
+ * @return
+ */
+ public static long getLogTime(Event event) {
+ if (event != null) {
+ Map<String, String> headers = event.getHeaders();
+ return getLogTime(headers);
+ }
+ return System.currentTimeMillis();
+ }
+
+ /**
+ * getAuditFormatTime
+ *
+ * @param msgTime
+ * @return
+ */
+ public static long getAuditFormatTime(long msgTime) {
+ long auditFormatTime = msgTime - msgTime % CommonPropertiesHolder.getAuditFormatInterval();
+ return auditFormatTime;
+ }
+
+ /**
+ * sendReport
+ */
+ public static void sendReport() {
+ AuditImp.getInstance().sendReport();
+ }
+}
diff --git a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/node/Application.java b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/node/Application.java
index dde88fa..4311f52 100644
--- a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/node/Application.java
+++ b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/node/Application.java
@@ -55,6 +55,7 @@ import org.apache.inlong.commons.config.IDataProxyConfigHolder;
import org.apache.inlong.dataproxy.config.ConfigManager;
import org.apache.inlong.dataproxy.config.RemoteConfigManager;
import org.apache.inlong.dataproxy.metrics.MetricObserver;
+import org.apache.inlong.dataproxy.metrics.audit.AuditUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -269,6 +270,7 @@ public class Application {
/**
* main
+ *
* @param args
*/
public static void main(String[] args) {
@@ -390,10 +392,12 @@ public class Application {
application.handleConfigurationEvent(configurationProvider.getConfiguration());
}
}
- //metrics
+ // metrics
MetricObserver.init(ConfigManager.getInstance().getCommonProperties());
-
- //start application
+ // audit
+ AuditUtils.initAudit();
+
+ // start application
application.start();
final Application appReference = application;
@@ -401,6 +405,7 @@ public class Application {
@Override
public void run() {
+ AuditUtils.sendReport();
appReference.stop();
}
});
diff --git a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/sink/MetaSink.java b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/sink/MetaSink.java
index 159f860..001d5db 100644
--- a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/sink/MetaSink.java
+++ b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/sink/MetaSink.java
@@ -46,6 +46,7 @@ import org.apache.inlong.dataproxy.consts.AttributeConstants;
import org.apache.inlong.dataproxy.consts.ConfigConstants;
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.Constants;
import org.apache.inlong.dataproxy.utils.NetworkUtils;
import org.apache.inlong.tubemq.client.config.TubeClientConfig;
@@ -571,27 +572,25 @@ public class MetaSink extends AbstractSink implements Configurable {
/**
* addMetric
*
- * @param currentRecord
- * @param topic
+ * @param event
* @param result
- * @param size
+ * @param sendTime
*/
- private void addMetric(Event currentRecord, boolean result, long sendTime) {
+ private void addMetric(Event event, boolean result, long sendTime) {
Map<String, String> dimensions = new HashMap<>();
dimensions.put(DataProxyMetricItem.KEY_CLUSTER_ID, MetaSink.this.getName());
dimensions.put(DataProxyMetricItem.KEY_SINK_ID, MetaSink.this.getName());
- if (currentRecord.getHeaders().containsKey(TOPIC)) {
- dimensions.put(DataProxyMetricItem.KEY_SINK_DATA_ID, currentRecord.getHeaders().get(TOPIC));
- } else {
- dimensions.put(DataProxyMetricItem.KEY_SINK_DATA_ID, "");
- }
+ dimensions.put(DataProxyMetricItem.KEY_SINK_DATA_ID, event.getHeaders().getOrDefault(TOPIC, ""));
+ DataProxyMetricItem.fillInlongId(event, dimensions);
+ DataProxyMetricItem.fillAuditFormatTime(event, dimensions);
DataProxyMetricItem metricItem = MetaSink.this.metricItemSet.findMetricItem(dimensions);
if (result) {
metricItem.sendSuccessCount.incrementAndGet();
- metricItem.sendSuccessSize.addAndGet(currentRecord.getBody().length);
+ metricItem.sendSuccessSize.addAndGet(event.getBody().length);
+ AuditUtils.add(AuditUtils.AUDIT_ID_DATAPROXY_SEND_SUCCESS, event);
if (sendTime > 0) {
long currentTime = System.currentTimeMillis();
- long msgTime = NumberUtils.toLong(currentRecord.getHeaders().get(Constants.HEADER_KEY_MSG_TIME),
+ long msgTime = NumberUtils.toLong(event.getHeaders().get(Constants.HEADER_KEY_MSG_TIME),
sendTime);
long sinkDuration = currentTime - sendTime;
long nodeDuration = currentTime - NumberUtils.toLong(Constants.HEADER_KEY_SOURCE_TIME, msgTime);
@@ -602,7 +601,7 @@ public class MetaSink extends AbstractSink implements Configurable {
}
} else {
metricItem.sendFailCount.incrementAndGet();
- metricItem.sendFailSize.addAndGet(currentRecord.getBody().length);
+ metricItem.sendFailSize.addAndGet(event.getBody().length);
}
}
diff --git a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/sink/SimpleMessageTubeSink.java b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/sink/SimpleMessageTubeSink.java
index 0a63148..f52c254 100644
--- a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/sink/SimpleMessageTubeSink.java
+++ b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/sink/SimpleMessageTubeSink.java
@@ -46,6 +46,7 @@ import org.apache.inlong.dataproxy.consts.AttributeConstants;
import org.apache.inlong.dataproxy.consts.ConfigConstants;
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.Constants;
import org.apache.inlong.dataproxy.utils.NetworkUtils;
import org.apache.inlong.tubemq.client.config.TubeClientConfig;
@@ -579,27 +580,26 @@ public class SimpleMessageTubeSink extends AbstractSink implements Configurable
/**
* addMetric
*
- * @param currentRecord
- * @param topic
+ * @param event
* @param result
- * @param size
+ * @param sendTime
*/
- private void addMetric(Event currentRecord, boolean result, long sendTime) {
+ private void addMetric(Event event, boolean result, long sendTime) {
Map<String, String> dimensions = new HashMap<>();
dimensions.put(DataProxyMetricItem.KEY_CLUSTER_ID, SimpleMessageTubeSink.this.getName());
dimensions.put(DataProxyMetricItem.KEY_SINK_ID, SimpleMessageTubeSink.this.getName());
- if (currentRecord.getHeaders().containsKey(TOPIC)) {
- dimensions.put(DataProxyMetricItem.KEY_SINK_DATA_ID, currentRecord.getHeaders().get(TOPIC));
- } else {
- dimensions.put(DataProxyMetricItem.KEY_SINK_DATA_ID, "");
- }
+ dimensions.put(DataProxyMetricItem.KEY_SINK_DATA_ID, event.getHeaders().getOrDefault(TOPIC, ""));
+ DataProxyMetricItem.fillInlongId(event, dimensions);
+ DataProxyMetricItem.fillAuditFormatTime(event, dimensions);
+
DataProxyMetricItem metricItem = SimpleMessageTubeSink.this.metricItemSet.findMetricItem(dimensions);
if (result) {
metricItem.sendSuccessCount.incrementAndGet();
- metricItem.sendSuccessSize.addAndGet(currentRecord.getBody().length);
+ metricItem.sendSuccessSize.addAndGet(event.getBody().length);
+ AuditUtils.add(AuditUtils.AUDIT_ID_DATAPROXY_SEND_SUCCESS, event);
if (sendTime > 0) {
long currentTime = System.currentTimeMillis();
- long msgTime = NumberUtils.toLong(currentRecord.getHeaders().get(Constants.HEADER_KEY_MSG_TIME),
+ long msgTime = NumberUtils.toLong(event.getHeaders().get(Constants.HEADER_KEY_MSG_TIME),
sendTime);
long sinkDuration = currentTime - sendTime;
long nodeDuration = currentTime - NumberUtils.toLong(Constants.HEADER_KEY_SOURCE_TIME, msgTime);
@@ -610,7 +610,7 @@ public class SimpleMessageTubeSink extends AbstractSink implements Configurable
}
} else {
metricItem.sendFailCount.incrementAndGet();
- metricItem.sendFailSize.addAndGet(currentRecord.getBody().length);
+ metricItem.sendFailSize.addAndGet(event.getBody().length);
}
}
diff --git a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/sink/pulsar/federation/PulsarProducerCluster.java b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/sink/pulsar/federation/PulsarProducerCluster.java
index 3ebc741..b347402 100644
--- a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/sink/pulsar/federation/PulsarProducerCluster.java
+++ b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/sink/pulsar/federation/PulsarProducerCluster.java
@@ -29,9 +29,9 @@ import org.apache.flume.Context;
import org.apache.flume.Event;
import org.apache.flume.lifecycle.LifecycleAware;
import org.apache.flume.lifecycle.LifecycleState;
-import org.apache.inlong.dataproxy.config.holder.CommonPropertiesHolder;
import org.apache.inlong.dataproxy.config.pojo.CacheClusterConfig;
import org.apache.inlong.dataproxy.metrics.DataProxyMetricItem;
+import org.apache.inlong.dataproxy.metrics.audit.AuditUtils;
import org.apache.inlong.dataproxy.utils.Constants;
import org.apache.pulsar.client.api.AuthenticationFactory;
import org.apache.pulsar.client.api.CompressionType;
@@ -267,26 +267,26 @@ public class PulsarProducerCluster implements LifecycleAware {
/**
* addMetric
*
- * @param currentRecord
+ * @param event
* @param topic
* @param result
- * @param size
+ * @param sendTime
*/
- private void addMetric(Event currentRecord, String topic, boolean result, long sendTime) {
+ private void addMetric(Event event, String topic, boolean result, long sendTime) {
+ // metric
Map<String, String> dimensions = new HashMap<>();
dimensions.put(DataProxyMetricItem.KEY_CLUSTER_ID, this.sinkContext.getProxyClusterId());
- // metric
- DataProxyMetricItem.fillInlongId(currentRecord, dimensions);
dimensions.put(DataProxyMetricItem.KEY_SINK_ID, this.cacheClusterName);
dimensions.put(DataProxyMetricItem.KEY_SINK_DATA_ID, topic);
- long msgTime = NumberUtils.toLong(currentRecord.getHeaders().get(Constants.HEADER_KEY_MSG_TIME), sendTime);
- long auditFormatTime = msgTime - msgTime % CommonPropertiesHolder.getAuditFormatInterval();
- dimensions.put(DataProxyMetricItem.KEY_MESSAGE_TIME, String.valueOf(auditFormatTime));
+ DataProxyMetricItem.fillInlongId(event, dimensions);
+ DataProxyMetricItem.fillAuditFormatTime(event, dimensions);
DataProxyMetricItem metricItem = this.sinkContext.getMetricItemSet().findMetricItem(dimensions);
if (result) {
metricItem.sendSuccessCount.incrementAndGet();
- metricItem.sendSuccessSize.addAndGet(currentRecord.getBody().length);
+ metricItem.sendSuccessSize.addAndGet(event.getBody().length);
+ AuditUtils.add(AuditUtils.AUDIT_ID_DATAPROXY_SEND_SUCCESS, event);
if (sendTime > 0) {
+ long msgTime = AuditUtils.getLogTime(event);
long currentTime = System.currentTimeMillis();
long sinkDuration = currentTime - sendTime;
long nodeDuration = currentTime - NumberUtils.toLong(Constants.HEADER_KEY_SOURCE_TIME, msgTime);
@@ -297,7 +297,7 @@ public class PulsarProducerCluster implements LifecycleAware {
}
} else {
metricItem.sendFailCount.incrementAndGet();
- metricItem.sendFailSize.addAndGet(currentRecord.getBody().length);
+ metricItem.sendFailSize.addAndGet(event.getBody().length);
}
}
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 2ab09f5..44dab2c 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
@@ -22,8 +22,6 @@ import static org.apache.inlong.dataproxy.consts.ConfigConstants.SLA_METRIC_DATA
import static org.apache.inlong.dataproxy.consts.ConfigConstants.SLA_METRIC_GROUPID;
import static org.apache.inlong.dataproxy.source.SimpleTcpSource.blacklist;
-import com.google.common.base.Joiner;
-import com.google.common.base.Splitter;
import java.io.IOException;
import java.net.SocketAddress;
import java.nio.ByteBuffer;
@@ -34,12 +32,15 @@ import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.atomic.AtomicReference;
+
import org.apache.commons.lang.StringUtils;
import org.apache.flume.ChannelException;
import org.apache.flume.Event;
import org.apache.flume.channel.ChannelProcessor;
import org.apache.flume.event.EventBuilder;
import org.apache.flume.source.AbstractSource;
+import org.apache.inlong.commons.monitor.MonitorIndex;
+import org.apache.inlong.commons.monitor.MonitorIndexExt;
import org.apache.inlong.commons.msg.TDMsg1;
import org.apache.inlong.dataproxy.base.ProxyMessage;
import org.apache.inlong.dataproxy.config.ConfigManager;
@@ -49,8 +50,7 @@ import org.apache.inlong.dataproxy.exception.ErrorCode;
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.commons.monitor.MonitorIndex;
-import org.apache.inlong.commons.monitor.MonitorIndexExt;
+import org.apache.inlong.dataproxy.metrics.audit.AuditUtils;
import org.apache.inlong.dataproxy.utils.NetworkUtils;
import org.jboss.netty.buffer.ChannelBuffer;
import org.jboss.netty.buffer.ChannelBuffers;
@@ -64,6 +64,9 @@ import org.jboss.netty.channel.group.ChannelGroup;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+import com.google.common.base.Joiner;
+import com.google.common.base.Splitter;
+
/**
* Server message handler
*
@@ -469,7 +472,7 @@ public class ServerMessageHandler extends SimpleChannelHandler {
try {
processor.processEvent(event);
monitorIndexExt.incrementAndGet("EVENT_SUCCESS");
- this.addMetric(true, data.length);
+ this.addMetric(true, data.length, event);
monitorIndex.addAndGet(new String(newbase),
Integer.parseInt(proxyMetricMsgCnt), 1, data.length, 0);
} catch (Throwable ex) {
@@ -477,7 +480,7 @@ public class ServerMessageHandler extends SimpleChannelHandler {
monitorIndexExt.incrementAndGet("EVENT_DROPPED");
monitorIndex.addAndGet(new String(newbase), 0,0,0,
Integer.parseInt(proxyMetricMsgCnt));
- this.addMetric(false, data.length);
+ this.addMetric(false, data.length, event);
throw new ChannelException("ProcessEvent error can't write event to channel.");
}
}
@@ -579,7 +582,7 @@ public class ServerMessageHandler extends SimpleChannelHandler {
logger.debug("message received");
if (e == null) {
logger.error("get null messageevent, just skip");
- this.addMetric(false, 0);
+ this.addMetric(false, 0, null);
return;
}
ChannelBuffer cb = ((ChannelBuffer) e.getMessage());
@@ -589,7 +592,7 @@ public class ServerMessageHandler extends SimpleChannelHandler {
if (len == 0 && this.filterEmptyMsg) {
logger.warn("skip empty msg.");
cb.clear();
- this.addMetric(false, 0);
+ this.addMetric(false, 0, null);
return;
}
@@ -599,25 +602,25 @@ public class ServerMessageHandler extends SimpleChannelHandler {
resultMap = serviceDecoder.extractData(cb, remoteChannel, e);
} catch (MessageIDException ex) {
logger.error("MessageIDException ex = {}", ex);
- this.addMetric(false, 0);
+ this.addMetric(false, 0, null);
throw new IOException(ex.getCause());
}
if (resultMap == null) {
logger.info("result is null");
- this.addMetric(false, 0);
+ this.addMetric(false, 0, null);
return;
}
MsgType msgType = (MsgType) resultMap.get(ConfigConstants.MSG_TYPE);
if (MsgType.MSG_HEARTBEAT.equals(msgType)) {
remoteChannel.write(heartbeatBuffer, remoteSocketAddress);
- this.addMetric(false, 0);
+ this.addMetric(false, 0, null);
return;
}
if (MsgType.MSG_BIN_HEARTBEAT.equals(msgType)) {
- this.addMetric(false, 0);
+ this.addMetric(false, 0, null);
return;
}
@@ -649,10 +652,10 @@ public class ServerMessageHandler extends SimpleChannelHandler {
Event event = EventBuilder.withBody(body, headers);
try {
processor.processEvent(event);
- this.addMetric(true, body.length);
+ this.addMetric(true, body.length, event);
} catch (Throwable ex) {
logger.error("Error writing to controller,data will discard.", ex);
- this.addMetric(false, body.length);
+ this.addMetric(false, body.length, event);
throw new ChannelException(
"Process Controller Event error can't write event to channel.");
}
@@ -669,10 +672,10 @@ public class ServerMessageHandler extends SimpleChannelHandler {
Event event = EventBuilder.withBody(body, headers);
try {
processor.processEvent(event);
- this.addMetric(true, body.length);
+ this.addMetric(true, body.length, event);
} catch (Throwable ex) {
logger.error("Error writing to controller,data will discard.", ex);
- this.addMetric(false, body.length);
+ this.addMetric(false, body.length, event);
throw new ChannelException(
"Process Controller Event error can't write event to channel.");
}
@@ -726,25 +729,23 @@ public class ServerMessageHandler extends SimpleChannelHandler {
/**
* addMetric
-<<<<<<< HEAD
*
-=======
- *
->>>>>>> add udp feature
* @param result
* @param size
+ * @param event
*/
- private void addMetric(boolean result, long size) {
+ 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());
- dimensions.put(DataProxyMetricItem.KEY_INLONG_GROUP_ID, "");
- dimensions.put(DataProxyMetricItem.KEY_INLONG_STREAM_ID, "");
+ 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/SimpleMessageHandler.java b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source/SimpleMessageHandler.java
index a0cbe05..2159b41 100644
--- 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
@@ -43,12 +43,12 @@ import org.apache.flume.source.AbstractSource;
import org.apache.inlong.commons.msg.TDMsg1;
import org.apache.inlong.dataproxy.base.ProxyMessage;
import org.apache.inlong.dataproxy.config.ConfigManager;
-import org.apache.inlong.dataproxy.config.holder.CommonPropertiesHolder;
import org.apache.inlong.dataproxy.consts.AttributeConstants;
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.Constants;
import org.jboss.netty.buffer.ChannelBuffer;
import org.jboss.netty.buffer.ChannelBuffers;
@@ -453,10 +453,10 @@ public class SimpleMessageHandler extends SimpleChannelHandler {
Event event = this.parseProxyMessage2Event(commonHeaders, message);
try {
processor.processEvent(event);
- this.addMetric(true, event.getBody().length);
+ 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);
+ this.addMetric(false, event.getBody().length, event);
throw new ChannelException("ProcessEvent error can't write event to channel.");
}
}
@@ -580,7 +580,7 @@ public class SimpleMessageHandler extends SimpleChannelHandler {
logger.info("message received");
if (msgEvent == null) {
logger.error("get null messageevent, just skip");
- this.addMetric(false, 0);
+ this.addMetric(false, 0, null);
return;
}
ChannelBuffer cb = ((ChannelBuffer) msgEvent.getMessage());
@@ -590,7 +590,7 @@ public class SimpleMessageHandler extends SimpleChannelHandler {
if (len == 0 && this.filterEmptyMsg) {
logger.warn("skip empty msg.");
cb.clear();
- this.addMetric(false, 0);
+ this.addMetric(false, 0, null);
return;
}
@@ -599,27 +599,27 @@ public class SimpleMessageHandler extends SimpleChannelHandler {
try {
resultMap = serviceProcessor.extractData(cb, remoteChannel, msgEvent);
} catch (MessageIDException ex) {
- this.addMetric(false, 0);
+ this.addMetric(false, 0, null);
throw new IOException(ex.getCause());
}
if (resultMap == null) {
logger.info("result is null");
- this.addMetric(false, 0);
+ this.addMetric(false, 0, null);
return;
}
MsgType msgType = (MsgType) resultMap.get(ConfigConstants.MSG_TYPE);
if (MsgType.MSG_HEARTBEAT.equals(msgType)) {
remoteChannel.write(heartbeatBuffer, remoteSocketAddress);
- this.addMetric(false, 0);
+ this.addMetric(false, 0, null);
return;
}
if (MsgType.MSG_BIN_HEARTBEAT.equals(msgType)) {
// ChannelBuffer binBuffer = getBinHeart(resultMap,msgType);
// remoteChannel.write(binBuffer, remoteSocketAddress);
- this.addMetric(false, 0);
+ this.addMetric(false, 0, null);
return;
}
@@ -650,10 +650,10 @@ public class SimpleMessageHandler extends SimpleChannelHandler {
Event event = EventBuilder.withBody(body, headers);
try {
processor.processEvent(event);
- this.addMetric(true, body.length);
+ this.addMetric(true, body.length, event);
} catch (Throwable ex) {
logger.error("Error writing to controller,data will discard.", ex);
- this.addMetric(false, body.length);
+ this.addMetric(false, body.length, event);
throw new ChannelException(
"Process Controller Event error can't write event to channel.");
}
@@ -671,10 +671,10 @@ public class SimpleMessageHandler extends SimpleChannelHandler {
Event event = EventBuilder.withBody(body, headers);
try {
processor.processEvent(event);
- this.addMetric(true, body.length);
+ this.addMetric(true, body.length, event);
} catch (Throwable ex) {
logger.error("Error writing to controller,data will discard.", ex);
- this.addMetric(false, body.length);
+ this.addMetric(false, body.length, event);
throw new ChannelException(
"Process Controller Event error can't write event to channel.");
}
@@ -722,21 +722,20 @@ public class SimpleMessageHandler extends SimpleChannelHandler {
*
* @param result
* @param size
+ * @param event
*/
- private void addMetric(boolean result, long size) {
+ 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());
- dimensions.put(DataProxyMetricItem.KEY_INLONG_GROUP_ID, "");
- dimensions.put(DataProxyMetricItem.KEY_INLONG_STREAM_ID, "");
- long msgTime = System.currentTimeMillis();
- long auditFormatTime = msgTime - msgTime % CommonPropertiesHolder.getAuditFormatInterval();
- dimensions.put(DataProxyMetricItem.KEY_MESSAGE_TIME, String.valueOf(auditFormatTime));
+ 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);