This is an automated email from the ASF dual-hosted git repository.

qiaojialin pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-iotdb.git


The following commit(s) were added to refs/heads/master by this push:
     new 50d710b  Avoid unnecessary close file in timed merge (#1063)
50d710b is described below

commit 50d710b7e875631ac7c1e480a11e755d5e91ac0c
Author: Jialin Qiao <[email protected]>
AuthorDate: Fri Apr 17 12:58:57 2020 +0800

    Avoid unnecessary close file in timed merge (#1063)
    
    * avoid unnecessary close in timed merge
---
 .../engine/storagegroup/StorageGroupProcessor.java |  3 +-
 .../db/engine/storagegroup/TsFileProcessor.java    | 38 ++++++++++++++--------
 2 files changed, 27 insertions(+), 14 deletions(-)

diff --git 
a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessor.java
 
b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessor.java
index 273148d..1ac1e85 100755
--- 
a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessor.java
+++ 
b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessor.java
@@ -1349,11 +1349,12 @@ public class StorageGroupProcessor {
       }
       logger.info("{} will close all files for starting a merge (fullmerge = 
{})", storageGroupName,
           fullMerge);
-      syncCloseAllWorkingTsFileProcessors();
+
       if (unSequenceFileList.isEmpty() || sequenceFileTreeSet.isEmpty()) {
         logger.info("{} no files to be merged", storageGroupName);
         return;
       }
+      syncCloseAllWorkingTsFileProcessors();
 
       long budget = 
IoTDBDescriptor.getInstance().getConfig().getMergeMemoryBudget();
       long timeLowerBound = System.currentTimeMillis() - dataTTL;
diff --git 
a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileProcessor.java
 
b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileProcessor.java
index 7d7dafe..e2913d2 100644
--- 
a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileProcessor.java
+++ 
b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileProcessor.java
@@ -275,7 +275,11 @@ public class TsFileProcessor {
     long fileSize = tsFileResource.getFileSize();
     long fileSizeThreshold = IoTDBDescriptor.getInstance().getConfig()
         .getTsFileSizeThreshold();
-    return fileSize > fileSizeThreshold;
+    if (fileSize >= fileSizeThreshold) {
+      logger.info("{} fileSize {} >= fileSizeThreshold {}", 
tsFileResource.getPath(),
+          fileSize, fileSizeThreshold);
+    }
+    return fileSize >= fileSizeThreshold;
   }
 
   void syncClose() {
@@ -317,7 +321,20 @@ public class TsFileProcessor {
       logger.debug(FLUSH_QUERY_WRITE_LOCKED, storageGroupName, 
tsFileResource.getFile().getName());
     }
     try {
-      logger.info("Async close the file: {}", 
tsFileResource.getFile().getAbsolutePath());
+
+      if (logger.isInfoEnabled()) {
+        if (workMemTable != null) {
+          logger.info(
+              "{}: flush a working memtable in async close tsfile {}, memtable 
size: {}, tsfile size: {}",
+              storageGroupName, tsFileResource.getFile().getAbsolutePath(), 
workMemTable.memSize(),
+              tsFileResource.getFileSize());
+        } else {
+          logger.info("{}: flush a NotifyFlushMemTable in async close tsfile 
{}, tsfile size: {}",
+              storageGroupName, tsFileResource.getFile().getAbsolutePath(),
+              tsFileResource.getFileSize());
+        }
+      }
+
       if (shouldClose) {
         return;
       }
@@ -332,11 +349,7 @@ public class TsFileProcessor {
       //we have to add the memtable into flushingList first and then set the 
shouldClose tag.
       // see https://issues.apache.org/jira/browse/IOTDB-510
       IMemTable tmpMemTable = workMemTable == null ? new NotifyFlushMemTable() 
: workMemTable;
-      if (logger.isDebugEnabled()) {
-        logger
-            .debug("{}: {} async flush a memtable (signal = {}) when async 
close",
-                storageGroupName, tsFileResource.getFile().getName(), 
tmpMemTable.isSignalMemTable());
-      }
+
       try {
         addAMemtableIntoFlushingList(tmpMemTable);
         shouldClose = true;
@@ -575,7 +588,7 @@ public class TsFileProcessor {
         endFile();
       } catch (Exception e) {
         logger.error("{} meet error when flush FileMetadata to {}, change 
system mode to read-only",
-            storageGroupName, tsFileResource.getFile().getAbsolutePath());
+            storageGroupName, tsFileResource.getFile().getAbsolutePath(), e);
         IoTDBDescriptor.getInstance().getConfig().setReadOnly(true);
         try {
           writer.reset();
@@ -607,17 +620,16 @@ public class TsFileProcessor {
     // mark the TsFileResource closed, no need writer anymore
     closeTsFileCallback.call(this);
 
-    writer = null;
-
     if (logger.isInfoEnabled()) {
       long closeEndTime = System.currentTimeMillis();
-      logger.info("Storage group {} close the file {}, start time is {}, end 
time is {}, "
+      logger.info("Storage group {} close the file {}, TsFile size is {}, "
               + "time consumption of flushing metadata is {}ms",
           storageGroupName, tsFileResource.getFile().getAbsoluteFile(),
-          DatetimeUtils.convertMillsecondToZonedDateTime(closeStartTime),
-          DatetimeUtils.convertMillsecondToZonedDateTime(closeEndTime),
+          writer.getFile().length(),
           closeEndTime - closeStartTime);
     }
+
+    writer = null;
   }
 
 

Reply via email to