You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@carbondata.apache.org by ja...@apache.org on 2018/12/27 01:11:33 UTC

carbondata git commit: [CARBONDATA-3142]Add timestamp with thread name which created by CarbonThreadFactory [Forced Update!]

Repository: carbondata
Updated Branches:
  refs/heads/master 7d944be68 -> 04b52568d (forced update)


[CARBONDATA-3142]Add timestamp with thread name which created by CarbonThreadFactory

Add timestamp with thread name which created by CarbonThreadFactory
Because the names of threads created by CarbonThreadFactory are all the same, such as ProducerPool_, this logs are confused, we can't distinguish threads in the thread pool

This closes #2970


Project: http://git-wip-us.apache.org/repos/asf/carbondata/repo
Commit: http://git-wip-us.apache.org/repos/asf/carbondata/commit/04b52568
Tree: http://git-wip-us.apache.org/repos/asf/carbondata/tree/04b52568
Diff: http://git-wip-us.apache.org/repos/asf/carbondata/diff/04b52568

Branch: refs/heads/master
Commit: 04b52568d7431e6ab42a0faca6dc817eda5804e2
Parents: 128a6c8
Author: qiuchenjian <80...@qq.com>
Authored: Mon Dec 3 15:19:37 2018 +0800
Committer: Jacky Li <ja...@qq.com>
Committed: Thu Dec 27 09:10:36 2018 +0800

----------------------------------------------------------------------
 .../carbondata/core/util/CarbonThreadFactory.java      | 13 ++++++++++++-
 .../store/CarbonFactDataHandlerColumnar.java           |  8 ++++----
 2 files changed, 16 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/04b52568/core/src/main/java/org/apache/carbondata/core/util/CarbonThreadFactory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/util/CarbonThreadFactory.java b/core/src/main/java/org/apache/carbondata/core/util/CarbonThreadFactory.java
index 689365d..b471638 100644
--- a/core/src/main/java/org/apache/carbondata/core/util/CarbonThreadFactory.java
+++ b/core/src/main/java/org/apache/carbondata/core/util/CarbonThreadFactory.java
@@ -34,14 +34,25 @@ public class CarbonThreadFactory implements ThreadFactory {
    */
   private String name;
 
+  private boolean withTime = false;
+
   public CarbonThreadFactory(String name) {
     this.defaultFactory = Executors.defaultThreadFactory();
     this.name = name;
   }
 
+  public CarbonThreadFactory(String name, boolean withTime) {
+    this(name);
+    this.withTime = withTime;
+  }
+
   @Override public Thread newThread(Runnable r) {
     final Thread thread = defaultFactory.newThread(r);
-    thread.setName(name);
+    if (withTime) {
+      thread.setName(name + "_" + System.currentTimeMillis());
+    } else {
+      thread.setName(name);
+    }
     return thread;
   }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/04b52568/processing/src/main/java/org/apache/carbondata/processing/store/CarbonFactDataHandlerColumnar.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/store/CarbonFactDataHandlerColumnar.java b/processing/src/main/java/org/apache/carbondata/processing/store/CarbonFactDataHandlerColumnar.java
index c58f8d8..96fd544 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/store/CarbonFactDataHandlerColumnar.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/store/CarbonFactDataHandlerColumnar.java
@@ -144,14 +144,14 @@ public class CarbonFactDataHandlerColumnar implements CarbonFactHandler {
     blockletProcessingCount = new AtomicInteger(0);
     producerExecutorService = Executors.newFixedThreadPool(model.getNumberOfCores(),
         new CarbonThreadFactory(
-            "ProducerPool_" + System.nanoTime() + ":" + model.getTableName() + ", range: " + model
-                .getBucketId()));
+            String.format("ProducerPool:%s, range: %d",
+                model.getTableName(),model.getBucketId()), true));
     producerExecutorServiceTaskList =
         new ArrayList<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
     LOGGER.debug("Initializing writer executors");
     consumerExecutorService = Executors.newFixedThreadPool(1, new CarbonThreadFactory(
-        "ConsumerPool_" + System.nanoTime() + ":" + model.getTableName() + ", range: " + model
-            .getBucketId()));
+        String.format("ConsumerPool:%s, range: %d",
+                model.getTableName(),model.getBucketId()), true));
     consumerExecutorServiceTaskList = new ArrayList<>(1);
     semaphore = new Semaphore(numberOfCores);
     tablePageList = new TablePageList();