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