You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@doris.apache.org by GitBox <gi...@apache.org> on 2018/11/20 10:59:20 UTC

[GitHub] chaoyli closed pull request #329: Change PaloMetrics' name and Catalog's Id generator

chaoyli closed pull request #329: Change PaloMetrics' name and Catalog's Id generator
URL: https://github.com/apache/incubator-doris/pull/329
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git a/fe/src/main/java/org/apache/doris/catalog/Catalog.java b/fe/src/main/java/org/apache/doris/catalog/Catalog.java
index 5b57002d..e79aa1fa 100644
--- a/fe/src/main/java/org/apache/doris/catalog/Catalog.java
+++ b/fe/src/main/java/org/apache/doris/catalog/Catalog.java
@@ -279,8 +279,8 @@
     private int masterHttpPort;
     private String masterIp;
 
-    // For metadata persistence
-    private AtomicLong nextId = new AtomicLong(NEXT_ID_INIT_VALUE);
+    private CatalogIdGenerator idGenerator = new CatalogIdGenerator(NEXT_ID_INIT_VALUE);
+
     private String metaDir;
     private EditLog editLog;
     private int clusterId;
@@ -565,6 +565,7 @@ public void initialize(String[] args) throws Exception {
         loadImage(IMAGE_DIR); // load image file
         editLog.open(); // open bdb env or local output stream
         this.globalTransactionMgr.setEditLog(editLog);
+        this.idGenerator.setEditLog(editLog);
 
         // 4. start load label cleaner thread
         createCleaner();
@@ -1281,7 +1282,7 @@ public long loadHeader(DataInputStream dis, long checksum) throws IOException {
         newChecksum ^= replayedJournalId;
         long id = dis.readLong();
         newChecksum ^= id;
-        nextId.set(id);
+        idGenerator.setId(id);
 
         if (Catalog.getCurrentCatalogJournalVersion() >= FeMetaVersion.VERSION_32) {
             isDefaultClusterCreated = dis.readBoolean();
@@ -1685,7 +1686,7 @@ public long saveHeader(DataOutputStream dos, long replayedJournalId, long checks
         dos.writeLong(replayedJournalId);
 
         // Write id
-        long id = nextId.getAndIncrement();
+        long id = idGenerator.getBatchEndId();
         checksum ^= id;
         dos.writeLong(id);
 
@@ -4093,8 +4094,7 @@ public EditLog getEditLog() {
 
     // Get the next available, need't lock because of nextId is atomic.
     public long getNextId() {
-        long id = nextId.getAndIncrement();
-        editLog.logSaveNextId(id);
+        long id = idGenerator.getNextId();
         return id;
     }
 
@@ -4371,9 +4371,7 @@ public void setEditLog(EditLog editLog) {
     }
 
     public void setNextId(long id) {
-        if (nextId.get() < id) {
-            nextId.set(id);
-        }
+        idGenerator.setId(id);
     }
 
     public void setHaProtocol(HAProtocol protocol) {
diff --git a/fe/src/main/java/org/apache/doris/catalog/CatalogIdGenerator.java b/fe/src/main/java/org/apache/doris/catalog/CatalogIdGenerator.java
new file mode 100644
index 00000000..4aa9bb93
--- /dev/null
+++ b/fe/src/main/java/org/apache/doris/catalog/CatalogIdGenerator.java
@@ -0,0 +1,64 @@
+// 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.doris.catalog;
+
+import org.apache.doris.persist.EditLog;
+
+// This new Id generator is just same as TransactionIdGenerator.
+// But we can't just use TransactionIdGenerator to replace the old catalog's 'nextId' for compatibility reason.
+// cause they are using different edit log operation type.
+public class CatalogIdGenerator {
+    private static final int BATCH_ID_INTERVAL = 1000;
+
+    private long nextId;
+    private long batchEndId;
+
+    private EditLog editLog;
+
+    public CatalogIdGenerator(long initValue) {
+        nextId = initValue + 1;
+        batchEndId = initValue;
+    }
+
+    public void setEditLog(EditLog editLog) {
+        this.editLog = editLog;
+    }
+
+    // performance is more quickly
+    public synchronized long getNextId() {
+        if (nextId < batchEndId) {
+            return nextId++;
+        } else {
+            batchEndId = batchEndId + BATCH_ID_INTERVAL;
+            editLog.logSaveNextId(batchEndId);
+            return nextId++;
+        }
+    }
+
+    public synchronized void setId(long id) {
+        if (id > batchEndId) {
+            batchEndId = id;
+            nextId = id;
+        }
+    }
+
+    // just for checkpoint, so no need to synchronize
+    public long getBatchEndId() {
+        return batchEndId;
+    }
+}
diff --git a/fe/src/main/java/org/apache/doris/metric/PaloCounterMetric.java b/fe/src/main/java/org/apache/doris/metric/CounterMetric.java
similarity index 88%
rename from fe/src/main/java/org/apache/doris/metric/PaloCounterMetric.java
rename to fe/src/main/java/org/apache/doris/metric/CounterMetric.java
index dec52f2c..9cdd452d 100644
--- a/fe/src/main/java/org/apache/doris/metric/PaloCounterMetric.java
+++ b/fe/src/main/java/org/apache/doris/metric/CounterMetric.java
@@ -20,9 +20,9 @@
 /*
  * Counter metric can only be increased
  */
-public abstract class PaloCounterMetric<T> extends PaloMetric<T> {
+public abstract class CounterMetric<T> extends Metric<T> {
 
-    public PaloCounterMetric(String name, String description) {
+    public CounterMetric(String name, String description) {
         super(name, MetricType.COUNTER, description);
     }
 
diff --git a/fe/src/main/java/org/apache/doris/metric/PaloMetricRegistry.java b/fe/src/main/java/org/apache/doris/metric/DorisMetricRegistry.java
similarity index 82%
rename from fe/src/main/java/org/apache/doris/metric/PaloMetricRegistry.java
rename to fe/src/main/java/org/apache/doris/metric/DorisMetricRegistry.java
index 8dd12336..461b5116 100644
--- a/fe/src/main/java/org/apache/doris/metric/PaloMetricRegistry.java
+++ b/fe/src/main/java/org/apache/doris/metric/DorisMetricRegistry.java
@@ -22,19 +22,19 @@
 import java.util.List;
 import java.util.stream.Collectors;
 
-public class PaloMetricRegistry {
+public class DorisMetricRegistry {
 
-    private List<PaloMetric> paloMetrics = Lists.newArrayList();
+    private List<Metric> paloMetrics = Lists.newArrayList();
 
-    public PaloMetricRegistry() {
+    public DorisMetricRegistry() {
 
     }
 
-    public synchronized void addPaloMetrics(PaloMetric paloMetric) {
+    public synchronized void addPaloMetrics(Metric paloMetric) {
         paloMetrics.add(paloMetric);
     }
 
-    public synchronized List<PaloMetric> getPaloMetrics() {
+    public synchronized List<Metric> getPaloMetrics() {
         return Lists.newArrayList(paloMetrics);
     }
 
diff --git a/fe/src/main/java/org/apache/doris/metric/PaloDoubleCounterMetric.java b/fe/src/main/java/org/apache/doris/metric/DoubleCounterMetric.java
similarity index 89%
rename from fe/src/main/java/org/apache/doris/metric/PaloDoubleCounterMetric.java
rename to fe/src/main/java/org/apache/doris/metric/DoubleCounterMetric.java
index f664a22f..b4b92406 100644
--- a/fe/src/main/java/org/apache/doris/metric/PaloDoubleCounterMetric.java
+++ b/fe/src/main/java/org/apache/doris/metric/DoubleCounterMetric.java
@@ -19,9 +19,9 @@
 
 import com.google.common.util.concurrent.AtomicDouble;
 
-public class PaloDoubleCounterMetric extends PaloCounterMetric<Double> {
+public class DoubleCounterMetric extends CounterMetric<Double> {
 
-    public PaloDoubleCounterMetric(String name, String description) {
+    public DoubleCounterMetric(String name, String description) {
         super(name, description);
     }
 
diff --git a/fe/src/main/java/org/apache/doris/metric/PaloGaugeMetric.java b/fe/src/main/java/org/apache/doris/metric/GaugeMetric.java
similarity index 88%
rename from fe/src/main/java/org/apache/doris/metric/PaloGaugeMetric.java
rename to fe/src/main/java/org/apache/doris/metric/GaugeMetric.java
index 8e5bbd5b..581da720 100644
--- a/fe/src/main/java/org/apache/doris/metric/PaloGaugeMetric.java
+++ b/fe/src/main/java/org/apache/doris/metric/GaugeMetric.java
@@ -20,9 +20,9 @@
 /*
  * Gauge metric is updated every time it is visited
  */
-public abstract class PaloGaugeMetric<T> extends PaloMetric<T> {
+public abstract class GaugeMetric<T> extends Metric<T> {
 
-    public PaloGaugeMetric(String name, String description) {
+    public GaugeMetric(String name, String description) {
         super(name, MetricType.GAUGE, description);
     }
 }
diff --git a/fe/src/main/java/org/apache/doris/metric/PaloLongCounterMetric.java b/fe/src/main/java/org/apache/doris/metric/LongCounterMetric.java
similarity index 89%
rename from fe/src/main/java/org/apache/doris/metric/PaloLongCounterMetric.java
rename to fe/src/main/java/org/apache/doris/metric/LongCounterMetric.java
index 816615c7..cf4c6520 100644
--- a/fe/src/main/java/org/apache/doris/metric/PaloLongCounterMetric.java
+++ b/fe/src/main/java/org/apache/doris/metric/LongCounterMetric.java
@@ -19,9 +19,9 @@
 
 import java.util.concurrent.atomic.AtomicLong;
 
-public class PaloLongCounterMetric extends PaloCounterMetric<Long> {
+public class LongCounterMetric extends CounterMetric<Long> {
 
-    public PaloLongCounterMetric(String name, String description) {
+    public LongCounterMetric(String name, String description) {
         super(name, description);
     }
 
diff --git a/fe/src/main/java/org/apache/doris/metric/PaloMetric.java b/fe/src/main/java/org/apache/doris/metric/Metric.java
similarity index 90%
rename from fe/src/main/java/org/apache/doris/metric/PaloMetric.java
rename to fe/src/main/java/org/apache/doris/metric/Metric.java
index 47c32712..c4e6302c 100644
--- a/fe/src/main/java/org/apache/doris/metric/PaloMetric.java
+++ b/fe/src/main/java/org/apache/doris/metric/Metric.java
@@ -21,7 +21,7 @@
 
 import java.util.List;
 
-public abstract class PaloMetric<T> {
+public abstract class Metric<T> {
     public enum MetricType {
         GAUGE, COUNTER
     }
@@ -31,7 +31,7 @@
     protected List<MetricLabel> labels = Lists.newArrayList();
     protected String description;
 
-    public PaloMetric(String name, MetricType type, String description) {
+    public Metric(String name, MetricType type, String description) {
         this.name = name;
         this.type = type;
         this.description = description;
@@ -49,7 +49,7 @@ public String getDescription() {
         return description;
     }
 
-    public PaloMetric<T> addLabel(MetricLabel label) {
+    public Metric<T> addLabel(MetricLabel label) {
         if (labels.contains(label)) {
             return this;
         }
diff --git a/fe/src/main/java/org/apache/doris/metric/MetricRepo.java b/fe/src/main/java/org/apache/doris/metric/MetricRepo.java
index 7f69f005..0b98e108 100644
--- a/fe/src/main/java/org/apache/doris/metric/MetricRepo.java
+++ b/fe/src/main/java/org/apache/doris/metric/MetricRepo.java
@@ -17,6 +17,9 @@
 
 package org.apache.doris.metric;
 
+import com.codahale.metrics.Histogram;
+import com.codahale.metrics.MetricRegistry;
+
 import org.apache.doris.alter.Alter;
 import org.apache.doris.alter.AlterJob.JobType;
 import org.apache.doris.catalog.Catalog;
@@ -30,10 +33,6 @@
 import org.apache.doris.service.ExecuteEnv;
 import org.apache.doris.system.Backend;
 import org.apache.doris.system.SystemInfoService;
-
-import com.codahale.metrics.Histogram;
-import com.codahale.metrics.MetricRegistry;
-
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 
@@ -45,20 +44,21 @@
     private static final Logger LOG = LogManager.getLogger(MetricRepo.class);
 
     private static final MetricRegistry METRIC_REGISTER = new MetricRegistry();
-    private static final PaloMetricRegistry PALO_METRIC_REGISTER = new PaloMetricRegistry();
+    private static final DorisMetricRegistry PALO_METRIC_REGISTER = new DorisMetricRegistry();
     
     public static AtomicBoolean isInit = new AtomicBoolean(false);
 
-    public static PaloLongCounterMetric COUNTER_REQUEST_ALL;
-    public static PaloLongCounterMetric COUNTER_QUERY_ALL;
-    public static PaloLongCounterMetric COUNTER_QUERY_ERR;
-    public static PaloLongCounterMetric COUNTER_LOAD_ADD;
-    public static PaloLongCounterMetric COUNTER_LOAD_FINISHED;
-    public static PaloLongCounterMetric COUNTER_EDIT_LOG_WRITE;
-    public static PaloLongCounterMetric COUNTER_EDIT_LOG_READ;
-    public static PaloLongCounterMetric COUNTER_IMAGE_WRITE;
-    public static PaloLongCounterMetric COUNTER_IMAGE_PUSH;
+    public static LongCounterMetric COUNTER_REQUEST_ALL;
+    public static LongCounterMetric COUNTER_QUERY_ALL;
+    public static LongCounterMetric COUNTER_QUERY_ERR;
+    public static LongCounterMetric COUNTER_LOAD_ADD;
+    public static LongCounterMetric COUNTER_LOAD_FINISHED;
+    public static LongCounterMetric COUNTER_EDIT_LOG_WRITE;
+    public static LongCounterMetric COUNTER_EDIT_LOG_READ;
+    public static LongCounterMetric COUNTER_IMAGE_WRITE;
+    public static LongCounterMetric COUNTER_IMAGE_PUSH;
     public static Histogram HISTO_QUERY_LATENCY;
+    public static Histogram HISTO_EDIT_LOG_WRITE_LATENCY;
 
     public static synchronized void init() {
         if (isInit.get()) {
@@ -70,7 +70,7 @@ public static synchronized void init() {
         Load load = Catalog.getInstance().getLoadInstance();
         for (EtlJobType jobType : EtlJobType.values()) {
             for (JobState state : JobState.values()) {
-                PaloGaugeMetric<Integer> gauge = (PaloGaugeMetric<Integer>) new PaloGaugeMetric<Integer>("job",
+                GaugeMetric<Integer> gauge = (GaugeMetric<Integer>) new GaugeMetric<Integer>("job",
                         "job statistics") {
                     @Override
                     public Integer getValue() {
@@ -94,7 +94,7 @@ public Integer getValue() {
                 continue;
             }
             
-            PaloGaugeMetric<Integer> gauge = (PaloGaugeMetric<Integer>) new PaloGaugeMetric<Integer>("job",
+            GaugeMetric<Integer> gauge = (GaugeMetric<Integer>) new GaugeMetric<Integer>("job",
                     "job statistics") {
                 @Override
                 public Integer getValue() {
@@ -118,7 +118,7 @@ public Integer getValue() {
         generateCapacityMetrics();
 
         // connections
-        PaloGaugeMetric<Integer> conections = (PaloGaugeMetric<Integer>) new PaloGaugeMetric<Integer>(
+        GaugeMetric<Integer> conections = (GaugeMetric<Integer>) new GaugeMetric<Integer>(
                 "connection_total", "total connections") {
             @Override
             public Integer getValue() {
@@ -128,7 +128,7 @@ public Integer getValue() {
         PALO_METRIC_REGISTER.addPaloMetrics(conections);
 
         // journal id
-        PaloGaugeMetric<Long> maxJournalId = (PaloGaugeMetric<Long>) new PaloGaugeMetric<Long>(
+        GaugeMetric<Long> maxJournalId = (GaugeMetric<Long>) new GaugeMetric<Long>(
                 "max_journal_id", "max journal id of this frontends") {
             @Override
             public Long getValue() {
@@ -142,31 +142,32 @@ public Long getValue() {
         PALO_METRIC_REGISTER.addPaloMetrics(maxJournalId);
 
         // 2. counter
-        COUNTER_REQUEST_ALL = new PaloLongCounterMetric("request_total", "total request");
+        COUNTER_REQUEST_ALL = new LongCounterMetric("request_total", "total request");
         PALO_METRIC_REGISTER.addPaloMetrics(COUNTER_REQUEST_ALL);
-        COUNTER_QUERY_ALL = new PaloLongCounterMetric("query_total", "total query");
+        COUNTER_QUERY_ALL = new LongCounterMetric("query_total", "total query");
         PALO_METRIC_REGISTER.addPaloMetrics(COUNTER_QUERY_ALL);
-        COUNTER_QUERY_ERR = new PaloLongCounterMetric("query_err", "total error query");
+        COUNTER_QUERY_ERR = new LongCounterMetric("query_err", "total error query");
         PALO_METRIC_REGISTER.addPaloMetrics(COUNTER_QUERY_ERR);
-        COUNTER_LOAD_ADD = new PaloLongCounterMetric("load_add", "total laod submit");
+        COUNTER_LOAD_ADD = new LongCounterMetric("load_add", "total laod submit");
         PALO_METRIC_REGISTER.addPaloMetrics(COUNTER_LOAD_ADD);
-        COUNTER_LOAD_FINISHED = new PaloLongCounterMetric("load_finished", "total laod finished");
+        COUNTER_LOAD_FINISHED = new LongCounterMetric("load_finished", "total laod finished");
         PALO_METRIC_REGISTER.addPaloMetrics(COUNTER_LOAD_FINISHED);
-        COUNTER_EDIT_LOG_WRITE = new PaloLongCounterMetric("edit_log_write", "counter of edit log write into bdbje");
+        COUNTER_EDIT_LOG_WRITE = new LongCounterMetric("edit_log_write", "counter of edit log write into bdbje");
         PALO_METRIC_REGISTER.addPaloMetrics(COUNTER_EDIT_LOG_WRITE);
-        COUNTER_EDIT_LOG_READ = new PaloLongCounterMetric("edit_log_read", "counter of edit log read from bdbje");
+        COUNTER_EDIT_LOG_READ = new LongCounterMetric("edit_log_read", "counter of edit log read from bdbje");
         PALO_METRIC_REGISTER.addPaloMetrics(COUNTER_EDIT_LOG_READ);
-        COUNTER_IMAGE_WRITE = new PaloLongCounterMetric("image_write", "counter of image generated");
+        COUNTER_IMAGE_WRITE = new LongCounterMetric("image_write", "counter of image generated");
         PALO_METRIC_REGISTER.addPaloMetrics(COUNTER_IMAGE_WRITE);
-        COUNTER_IMAGE_PUSH = new PaloLongCounterMetric("image_push",
+        COUNTER_IMAGE_PUSH = new LongCounterMetric("image_push",
                 "counter of image succeeded in pushing to other frontends");
         PALO_METRIC_REGISTER.addPaloMetrics(COUNTER_IMAGE_PUSH);
 
         // 3. histogram
         HISTO_QUERY_LATENCY = METRIC_REGISTER.histogram(MetricRegistry.name("query", "latency", "ms"));
+        HISTO_EDIT_LOG_WRITE_LATENCY = METRIC_REGISTER.histogram(MetricRegistry.name("editlog", "write", "latency",
+                                                                                     "ms"));
 
         isInit.set(true);
-        ;
     }
 
     // this metric is reentrant, so that we can add or remove metric along with the backend add or remove
@@ -187,7 +188,7 @@ public static void generateCapacityMetrics() {
             LOG.debug("get backend: {}", be);
             for (DiskInfo diskInfo : be.getDisks().values()) {
                 LOG.debug("get disk: {}", diskInfo);
-                PaloGaugeMetric<Long> total = (PaloGaugeMetric<Long>) new PaloGaugeMetric<Long>(CAPACITY,
+                GaugeMetric<Long> total = (GaugeMetric<Long>) new GaugeMetric<Long>(CAPACITY,
                         "disk capacity") {
                     @Override
                     public Long getValue() {
@@ -202,7 +203,7 @@ public Long getValue() {
                         .addLabel(new MetricLabel("type", "total"));
                 PALO_METRIC_REGISTER.addPaloMetrics(total);
                 
-                PaloGaugeMetric<Long> used = (PaloGaugeMetric<Long>) new PaloGaugeMetric<Long>(CAPACITY,
+                GaugeMetric<Long> used = (GaugeMetric<Long>) new GaugeMetric<Long>(CAPACITY,
                         "disk capacity") {
                     @Override
                     public Long getValue() {
@@ -221,7 +222,7 @@ public Long getValue() {
         }
     }
 
-    public static synchronized String getMetric(PaloMetricVisitor visitor) {
+    public static synchronized String getMetric(MetricVisitor visitor) {
         if (!isInit.get()) {
             return "";
         }
@@ -232,7 +233,7 @@ public static synchronized String getMetric(PaloMetricVisitor visitor) {
         sb.append(visitor.visitJvm(jvmStats)).append("\n");
 
         // palo metrics
-        for (PaloMetric metric : PALO_METRIC_REGISTER.getPaloMetrics()) {
+        for (Metric metric : PALO_METRIC_REGISTER.getPaloMetrics()) {
             sb.append(visitor.visit(metric)).append("\n");
         }
 
diff --git a/fe/src/main/java/org/apache/doris/metric/PaloMetricVisitor.java b/fe/src/main/java/org/apache/doris/metric/MetricVisitor.java
similarity index 89%
rename from fe/src/main/java/org/apache/doris/metric/PaloMetricVisitor.java
rename to fe/src/main/java/org/apache/doris/metric/MetricVisitor.java
index 3462a02c..9060860d 100644
--- a/fe/src/main/java/org/apache/doris/metric/PaloMetricVisitor.java
+++ b/fe/src/main/java/org/apache/doris/metric/MetricVisitor.java
@@ -21,17 +21,17 @@
 
 import com.codahale.metrics.Histogram;
 
-public abstract class PaloMetricVisitor {
+public abstract class MetricVisitor {
 
     protected String prefix;
 
-    public PaloMetricVisitor(String prefix) {
+    public MetricVisitor(String prefix) {
         this.prefix = prefix;
     }
 
     public abstract String visitJvm(JvmStats jvmStats);
 
-    public abstract String visit(PaloMetric metric);
+    public abstract String visit(Metric metric);
 
     public abstract String visitHistogram(String name, Histogram histogram);
 
diff --git a/fe/src/main/java/org/apache/doris/metric/PrometheusMetricVisitor.java b/fe/src/main/java/org/apache/doris/metric/PrometheusMetricVisitor.java
index cc3d2b0f..f9008d3a 100644
--- a/fe/src/main/java/org/apache/doris/metric/PrometheusMetricVisitor.java
+++ b/fe/src/main/java/org/apache/doris/metric/PrometheusMetricVisitor.java
@@ -38,7 +38,7 @@
  * # TYPE palo_fe_job_load_broker_cost_ms gauge 
  * palo_fe_job{job="load", type="mini", state="pending"} 0
  */
-public class PrometheusMetricVisitor extends PaloMetricVisitor {
+public class PrometheusMetricVisitor extends MetricVisitor {
     // jvm
     private static final String JVM_HEAP_SIZE_BYTES = "jvm_heap_size_bytes";
     private static final String JVM_NON_HEAP_SIZE_BYTES = "jvm_non_heap_size_bytes";
@@ -131,7 +131,7 @@ public String visitJvm(JvmStats jvmStats) {
     }
 
     @Override
-    public String visit(@SuppressWarnings("rawtypes") PaloMetric metric) {
+    public String visit(@SuppressWarnings("rawtypes") Metric metric) {
         // title
         final String fullName = prefix + "_" + metric.getName();
         StringBuilder sb = new StringBuilder();
diff --git a/fe/src/main/java/org/apache/doris/persist/EditLog.java b/fe/src/main/java/org/apache/doris/persist/EditLog.java
index e7e33109..63d3c593 100644
--- a/fe/src/main/java/org/apache/doris/persist/EditLog.java
+++ b/fe/src/main/java/org/apache/doris/persist/EditLog.java
@@ -57,7 +57,6 @@
 import org.apache.doris.system.Backend;
 import org.apache.doris.system.Frontend;
 import org.apache.doris.transaction.TransactionState;
-
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 
@@ -135,7 +134,7 @@ public static void loadJournal(Catalog catalog, JournalEntity journal) {
                 case OperationType.OP_SAVE_TRANSACTION_ID: {
                     String idString = ((Text) journal.getData()).toString();
                     long id = Long.parseLong(idString);
-                    catalog.getCurrentGlobalTransactionMgr().getTransactionIDGenerator().initTransactionId(id + 1);
+                    Catalog.getCurrentGlobalTransactionMgr().getTransactionIDGenerator().initTransactionId(id + 1);
                     break;
                 }
                 case OperationType.OP_CREATE_DB: {
@@ -651,7 +650,7 @@ private synchronized void logEdit(short op, Writable writable) {
             journal.write(op, writable);
         } catch (Exception e) {
             LOG.error("Fatal Error : write stream Exception", e);
-            Runtime.getRuntime().exit(-1);
+            System.exit(-1);
         }
 
         // get a new transactionId
@@ -661,6 +660,9 @@ private synchronized void logEdit(short op, Writable writable) {
         long end = System.currentTimeMillis();
         numTransactions++;
         totalTimeTransactions += (end - start);
+        if (MetricRepo.isInit.get()) {
+            MetricRepo.HISTO_EDIT_LOG_WRITE_LATENCY.update((end - start));
+        }
 
         if (LOG.isDebugEnabled()) {
             LOG.debug("nextId = {}, numTransactions = {}, totalTimeTransactions = {}, op = {}",
diff --git a/fe/src/main/java/org/apache/doris/transaction/GlobalTransactionMgr.java b/fe/src/main/java/org/apache/doris/transaction/GlobalTransactionMgr.java
index 620d62b4..7dbc126c 100644
--- a/fe/src/main/java/org/apache/doris/transaction/GlobalTransactionMgr.java
+++ b/fe/src/main/java/org/apache/doris/transaction/GlobalTransactionMgr.java
@@ -85,7 +85,7 @@
     private Map<Long, TransactionState> idToTransactionState;
     private com.google.common.collect.Table<Long, String, Long> dbIdToTxnLabels; 
     private Map<Long, Integer> runningTxnNums;
-    private TransactionIDGenerator idGenerator;
+    private TransactionIdGenerator idGenerator;
     
     private Catalog catalog;
     
@@ -94,7 +94,7 @@ public GlobalTransactionMgr(Catalog catalog) {
         dbIdToTxnLabels = HashBasedTable.create();  
         runningTxnNums = Maps.newHashMap();
         this.catalog = catalog;
-        this.idGenerator = new TransactionIDGenerator();
+        this.idGenerator = new TransactionIdGenerator();
     }
 
     /**
@@ -1145,7 +1145,7 @@ public int getTransactionNum() {
         return this.idToTransactionState.size();
     }
     
-    public TransactionIDGenerator getTransactionIDGenerator() {
+    public TransactionIdGenerator getTransactionIDGenerator() {
         return this.idGenerator;
     }
     
diff --git a/fe/src/main/java/org/apache/doris/transaction/TransactionIDGenerator.java b/fe/src/main/java/org/apache/doris/transaction/TransactionIdGenerator.java
similarity index 96%
rename from fe/src/main/java/org/apache/doris/transaction/TransactionIDGenerator.java
rename to fe/src/main/java/org/apache/doris/transaction/TransactionIdGenerator.java
index 749d1bf2..41217861 100644
--- a/fe/src/main/java/org/apache/doris/transaction/TransactionIDGenerator.java
+++ b/fe/src/main/java/org/apache/doris/transaction/TransactionIdGenerator.java
@@ -23,7 +23,7 @@
 
 import org.apache.doris.persist.EditLog;
 
-public class TransactionIDGenerator {
+public class TransactionIdGenerator {
 
     public static final long NEXT_ID_INIT_VALUE = 1;
     private static final int BATCH_ID_INTERVAL = 1000;
@@ -34,7 +34,7 @@
     
     private EditLog editLog;
     
-    public TransactionIDGenerator() {
+    public TransactionIdGenerator() {
     }
     
     public void setEditLog(EditLog editLog) {
diff --git a/fe/src/test/java/org/apache/doris/transaction/FakeTransactionIDGenerator.java b/fe/src/test/java/org/apache/doris/transaction/FakeTransactionIDGenerator.java
index 677a413d..06c5e2e2 100644
--- a/fe/src/test/java/org/apache/doris/transaction/FakeTransactionIDGenerator.java
+++ b/fe/src/test/java/org/apache/doris/transaction/FakeTransactionIDGenerator.java
@@ -26,7 +26,7 @@
 import mockit.Mock;
 import mockit.MockUp;
 
-public final class FakeTransactionIDGenerator extends MockUp<TransactionIDGenerator> {
+public final class FakeTransactionIDGenerator extends MockUp<TransactionIdGenerator> {
 
     private long currentId = 1000L;
 


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@doris.apache.org
For additional commands, e-mail: dev-help@doris.apache.org