You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iotdb.apache.org by qi...@apache.org on 2019/06/29 12:24:34 UTC

[incubator-iotdb] branch feature_async_close_tsfile updated: remove mod file not exist exception stack print and remove some comment

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

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


The following commit(s) were added to refs/heads/feature_async_close_tsfile by this push:
     new d933235  remove mod file not exist exception stack print and remove some comment
d933235 is described below

commit d933235d13f9f6c961ace5e7eb6c8c36a3f86490
Author: qiaojialin <64...@qq.com>
AuthorDate: Sat Jun 29 20:23:53 2019 +0800

    remove mod file not exist exception stack print and remove some comment
---
 .../db/engine/filenodeV2/FileNodeManagerV2.java    |  2 +-
 .../db/engine/filenodeV2/FileNodeProcessorV2.java  | 51 ++--------------------
 .../filenodeV2/UnsealedTsFileProcessorV2.java      |  5 +--
 .../iotdb/db/engine/memtable/ChunkBufferPool.java  |  9 ----
 .../db/engine/memtable/IWritableMemChunk.java      |  2 -
 .../db/engine/memtable/WritableMemChunkV2.java     |  3 --
 .../db/engine/modification/ModificationFile.java   |  6 +--
 .../io/LocalTextModificationAccessor.java          | 17 +++-----
 .../engine/modification/io/ModificationReader.java |  2 +-
 .../iotdb/db/query/context/QueryContext.java       |  3 +-
 .../db/query/control/QueryResourceManager.java     |  5 +--
 .../db/query/factory/SeriesReaderFactoryImpl.java  | 25 +++--------
 .../engine/filenodeV2/FileNodeProcessorV2Test.java |  8 +---
 13 files changed, 27 insertions(+), 111 deletions(-)

diff --git a/iotdb/src/main/java/org/apache/iotdb/db/engine/filenodeV2/FileNodeManagerV2.java b/iotdb/src/main/java/org/apache/iotdb/db/engine/filenodeV2/FileNodeManagerV2.java
index 505fd83..233620c 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/engine/filenodeV2/FileNodeManagerV2.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/engine/filenodeV2/FileNodeManagerV2.java
@@ -247,7 +247,7 @@ public class FileNodeManagerV2 implements IService {
    * query data.
    */
   public QueryDataSourceV2 query(SingleSeriesExpression seriesExpression, QueryContext context)
-      throws FileNodeManagerException, ProcessorException {
+      throws FileNodeManagerException {
     String deviceId = seriesExpression.getSeriesPath().getDevice();
     String measurementId = seriesExpression.getSeriesPath().getMeasurement();
     FileNodeProcessorV2 fileNodeProcessor = getProcessor(deviceId);
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/engine/filenodeV2/FileNodeProcessorV2.java b/iotdb/src/main/java/org/apache/iotdb/db/engine/filenodeV2/FileNodeProcessorV2.java
index 61d8163..2aac48f 100755
--- a/iotdb/src/main/java/org/apache/iotdb/db/engine/filenodeV2/FileNodeProcessorV2.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/engine/filenodeV2/FileNodeProcessorV2.java
@@ -265,8 +265,8 @@ public class FileNodeProcessorV2 {
       throws IOException {
     UnsealedTsFileProcessorV2 unsealedTsFileProcessor;
     boolean result;
+
     // create a new BufferWriteProcessor
-//    long start1 = System.currentTimeMillis();
     if (sequence) {
       if (workSequenceTsFileProcessor == null) {
         workSequenceTsFileProcessor = createTsFileProcessor(true);
@@ -280,19 +280,9 @@ public class FileNodeProcessorV2 {
       }
       unsealedTsFileProcessor = workUnSequenceTsFileProcessor;
     }
-//    start1 = System.currentTimeMillis() - start1;
-//    if (start1 > 1000) {
-//      LOGGER.info("FNP {} create a new unsealed file processor cost: {}", storageGroupName, start1);
-//    }
 
     // insert BufferWrite
-//    long start2 = System.currentTimeMillis();
     result = unsealedTsFileProcessor.insert(insertPlan, sequence);
-//    start2 = System.currentTimeMillis() - start2;
-//    if (start2 > 1000) {
-//      LOGGER.info("FNP {} insert a record into unsealed file processor cost: {}", storageGroupName,
-//          start2);
-//    }
 
     // try to update the latest time of the device of this tsRecord
     if (result && latestTimeForEachDevice.get(insertPlan.getDeviceId()) < insertPlan.getTime()) {
@@ -300,7 +290,6 @@ public class FileNodeProcessorV2 {
     }
 
     // check memtable size and may asyncFlush the workMemtable
-//    long time1 = System.currentTimeMillis();
     if (unsealedTsFileProcessor.shouldFlush()) {
 
       LOGGER.info("The memtable size {} reaches the threshold, async flush it to tsfile: {}",
@@ -313,11 +302,6 @@ public class FileNodeProcessorV2 {
         unsealedTsFileProcessor.asyncFlush();
       }
     }
-//    time1 = System.currentTimeMillis() - time1;
-//    if (time1 > 1000) {
-//      LOGGER.info("FNP {} check flush and close cost: {}ms", storageGroupName, time1);
-//    }
-
     return result;
   }
 
@@ -371,8 +355,7 @@ public class FileNodeProcessorV2 {
 
 
   // TODO need a read lock, please consider the concurrency with flush manager threads.
-  public QueryDataSourceV2 query(String deviceId, String measurementId)
-      throws FileNodeProcessorException {
+  public QueryDataSourceV2 query(String deviceId, String measurementId) {
     insertLock.readLock().lock();
     try {
       List<TsFileResourceV2> seqResources = getFileReSourceListForQuery(sequenceFileList,
@@ -386,23 +369,11 @@ public class FileNodeProcessorV2 {
   }
 
   private void writeLock() {
-//    long time = System.currentTimeMillis();
     insertLock.writeLock().lock();
-//    time = System.currentTimeMillis() - time;
-//    if (time > 1000) {
-//      LOGGER.info("storage group {} wait for write lock cost: {}", storageGroupName, time,
-//          new RuntimeException());
-//    }
-//    timerr.set(System.currentTimeMillis());
   }
 
   private void writeUnlock() {
     insertLock.writeLock().unlock();
-//    long time = System.currentTimeMillis() - timerr.get();
-//    if (time > 1000) {
-//      LOGGER.info("storage group {} take lock for {}ms", storageGroupName, time,
-//          new RuntimeException());
-//    }
   }
 
 
@@ -411,7 +382,7 @@ public class FileNodeProcessorV2 {
    * @return fill unsealed tsfile resources with memory data and ChunkMetadataList of data in disk
    */
   private List<TsFileResourceV2> getFileReSourceListForQuery(List<TsFileResourceV2> tsFileResources,
-      String deviceId, String measurementId) throws FileNodeProcessorException {
+      String deviceId, String measurementId) {
 
     MeasurementSchema mSchema = fileSchema.getMeasurementSchema(measurementId);
     TSDataType dataType = mSchema.getType();
@@ -428,13 +399,9 @@ public class FileNodeProcessorV2 {
             tsfileResourcesForQuery.add(tsFileResource);
           } else {
             Pair<ReadOnlyMemChunk, List<ChunkMetaData>> pair;
-            try {
-              pair = tsFileResource
+            pair = tsFileResource
                   .getUnsealedFileProcessor()
                   .query(deviceId, measurementId, dataType, mSchema.getProps());
-            } catch (UnsealedTsFileProcessorException e) {
-              throw new FileNodeProcessorException(e);
-            }
             tsfileResourcesForQuery
                 .add(new TsFileResourceV2(tsFileResource.getFile(),
                     tsFileResource.getStartTimeMap(),
@@ -612,25 +579,15 @@ public class FileNodeProcessorV2 {
 
 
   public boolean updateLatestFlushTimeCallback() {
-//    long time = System.currentTimeMillis();
     writeLock();
     try {
       // update the largest timestamp in the last flushing memtable
-//      long time1 = System.currentTimeMillis();
       for (Entry<String, Long> entry : latestTimeForEachDevice.entrySet()) {
         latestFlushedTimeForEachDevice.put(entry.getKey(), entry.getValue());
       }
-//      time1 = System.currentTimeMillis() - time1;
-//      if (time1 > 1000) {
-//        LOGGER.info("update latest flush time call back cost {}ms", time1);
-//      }
     } finally {
       writeUnlock();
     }
-//    time = System.currentTimeMillis() - time;
-//    if (time > 1000) {
-//      LOGGER.info("update latest flush time call back all cost {}ms", time);
-//    }
     return true;
   }
 
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/engine/filenodeV2/UnsealedTsFileProcessorV2.java b/iotdb/src/main/java/org/apache/iotdb/db/engine/filenodeV2/UnsealedTsFileProcessorV2.java
index d0dca77..57bafc7 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/engine/filenodeV2/UnsealedTsFileProcessorV2.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/engine/filenodeV2/UnsealedTsFileProcessorV2.java
@@ -434,8 +434,7 @@ public class UnsealedTsFileProcessorV2 {
    * @return corresponding chunk data and chunk metadata in memory
    */
   public Pair<ReadOnlyMemChunk, List<ChunkMetaData>> query(String deviceId,
-      String measurementId, TSDataType dataType, Map<String, String> props)
-      throws UnsealedTsFileProcessorException {
+      String measurementId, TSDataType dataType, Map<String, String> props) {
     flushQueryLock.readLock().lock();
     try {
       MemSeriesLazyMerger memSeriesLazyMerger = new MemSeriesLazyMerger();
@@ -467,8 +466,6 @@ public class UnsealedTsFileProcessorV2 {
           (List<Modification>) modificationFile.getModifications());
 
       return new Pair<>(timeValuePairSorter, chunkMetaDataList);
-    } catch (IOException e) {
-      throw new UnsealedTsFileProcessorException(e);
     } finally {
       flushQueryLock.readLock().unlock();
     }
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/engine/memtable/ChunkBufferPool.java b/iotdb/src/main/java/org/apache/iotdb/db/engine/memtable/ChunkBufferPool.java
index a41fa04..e8366ee 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/engine/memtable/ChunkBufferPool.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/engine/memtable/ChunkBufferPool.java
@@ -53,13 +53,8 @@ public class ChunkBufferPool {
               .getInstance().getConfig().getMemtableNumber();
       if (availableChunkBuffer.isEmpty() && size < capacity) {
         size++;
-//        LOGGER.info("For fask, generated a new ChunkBuffer for {}, system ChunkBuffer size: {}, stack size: {}",
-//            applier, size, availableChunkBuffer.size());
         return new ChunkBuffer(schema);
       } else if (!availableChunkBuffer.isEmpty()) {
-//        LOGGER
-//            .info("ReusableChunkBuffer size: {}, stack size: {}, then get a ChunkBuffer from stack for {}",
-//                size, availableChunkBuffer.size(), applier);
         ChunkBuffer chunkBuffer = availableChunkBuffer.pop();
         chunkBuffer.reInit(schema);
         return chunkBuffer;
@@ -69,9 +64,6 @@ public class ChunkBufferPool {
       int waitCount = 1;
       while (true) {
         if (!availableChunkBuffer.isEmpty()) {
-//          LOGGER.info(
-//              "ReusableChunkBuffer size: {}, stack size: {}, then get a ChunkBuffer from stack for {}",
-//              size, availableChunkBuffer.size(), applier);
           return availableChunkBuffer.pop();
         }
         try {
@@ -98,7 +90,6 @@ public class ChunkBufferPool {
         availableChunkBuffer.push(chunkBuffer);
       }
       availableChunkBuffer.notify();
-//      LOGGER.info("a chunk buffer returned, stack size {}", availableChunkBuffer.size());
     }
   }
 
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/engine/memtable/IWritableMemChunk.java b/iotdb/src/main/java/org/apache/iotdb/db/engine/memtable/IWritableMemChunk.java
index 0d4c280..fdd6306 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/engine/memtable/IWritableMemChunk.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/engine/memtable/IWritableMemChunk.java
@@ -51,8 +51,6 @@ public interface IWritableMemChunk extends TimeValuePairSorter {
    */
   void setTimeOffset(long offset);
 
-  void releasePrimitiveArrayList();
-
   /**
    * served for query requests.
    * @return
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/engine/memtable/WritableMemChunkV2.java b/iotdb/src/main/java/org/apache/iotdb/db/engine/memtable/WritableMemChunkV2.java
index 596e715..dba6db7 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/engine/memtable/WritableMemChunkV2.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/engine/memtable/WritableMemChunkV2.java
@@ -198,9 +198,6 @@ public class WritableMemChunkV2 implements IWritableMemChunk {
   }
 
   @Override
-  public void releasePrimitiveArrayList(){}
-
-  @Override
   public boolean isEmpty() {
     return list.size() == 0;
   }
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/engine/modification/ModificationFile.java b/iotdb/src/main/java/org/apache/iotdb/db/engine/modification/ModificationFile.java
index 598a0b9..964d89e 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/engine/modification/ModificationFile.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/engine/modification/ModificationFile.java
@@ -52,13 +52,13 @@ public class ModificationFile {
     this.filePath = filePath;
   }
 
-  private void init() throws IOException {
+  private void init() {
     synchronized (this) {
       modifications = (List<Modification>) reader.read();
     }
   }
 
-  private void checkInit() throws IOException {
+  private void checkInit() {
     if (modifications == null) {
       init();
     }
@@ -103,7 +103,7 @@ public class ModificationFile {
    *
    * @return an ArrayList of modifications.
    */
-  public Collection<Modification> getModifications() throws IOException {
+  public Collection<Modification> getModifications() {
     synchronized (this) {
       checkInit();
       return new ArrayList<>(modifications);
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/engine/modification/io/LocalTextModificationAccessor.java b/iotdb/src/main/java/org/apache/iotdb/db/engine/modification/io/LocalTextModificationAccessor.java
index 76433af..5584f2e 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/engine/modification/io/LocalTextModificationAccessor.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/engine/modification/io/LocalTextModificationAccessor.java
@@ -21,6 +21,7 @@ package org.apache.iotdb.db.engine.modification.io;
 
 import java.io.BufferedReader;
 import java.io.BufferedWriter;
+import java.io.File;
 import java.io.FileNotFoundException;
 import java.io.FileReader;
 import java.io.FileWriter;
@@ -56,19 +57,17 @@ public class LocalTextModificationAccessor implements ModificationReader, Modifi
   public LocalTextModificationAccessor(String filePath) {
     this.filePath = filePath;
   }
+
   @Override
-  public Collection<Modification> read() throws IOException {
-    BufferedReader reader;
-    try {
-      reader = new BufferedReader(new FileReader(filePath));
-    } catch (FileNotFoundException e) {
-      logger.debug("No modification has been written to this file", e);
+  public Collection<Modification> read() {
+    if (!new File(filePath).exists()) {
+      logger.debug("No modification has been written to this file");
       return new ArrayList<>();
     }
-    String line;
 
+    String line;
     List<Modification> modificationList = new ArrayList<>();
-    try {
+    try (BufferedReader reader = new BufferedReader(new FileReader(filePath))){
       while ((line = reader.readLine()) != null) {
         if (line.equals(ABORT_MARK) && !modificationList.isEmpty()) {
           modificationList.remove(modificationList.size() - 1);
@@ -79,8 +78,6 @@ public class LocalTextModificationAccessor implements ModificationReader, Modifi
     } catch (IOException e) {
       logger.error("An error occurred when reading modifications, and the remaining modifications "
               + "were ignored.", e);
-    } finally {
-      reader.close();
     }
     return modificationList;
   }
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/engine/modification/io/ModificationReader.java b/iotdb/src/main/java/org/apache/iotdb/db/engine/modification/io/ModificationReader.java
index 1abfadd..12ce781 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/engine/modification/io/ModificationReader.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/engine/modification/io/ModificationReader.java
@@ -34,7 +34,7 @@ public interface ModificationReader {
    *
    * @return a list of modifications contained the medium.
    */
-  Collection<Modification> read() throws IOException;
+  Collection<Modification> read();
 
   /**
    * Release resources like streams.
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/context/QueryContext.java b/iotdb/src/main/java/org/apache/iotdb/db/query/context/QueryContext.java
index 565b7d8..a329ad2 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/query/context/QueryContext.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/context/QueryContext.java
@@ -57,8 +57,7 @@ public class QueryContext {
    * Find the modifications of timeseries 'path' in 'modFile'. If they are not in the cache, read
    * them from 'modFile' and put then into the cache.
    */
-  public List<Modification> getPathModifications(ModificationFile modFile, String path)
-      throws IOException {
+  public List<Modification> getPathModifications(ModificationFile modFile, String path) {
 
     Map<String, List<Modification>> fileModifications =
         filePathModCache.computeIfAbsent(modFile.getFilePath(), k -> new HashMap<>());
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/control/QueryResourceManager.java b/iotdb/src/main/java/org/apache/iotdb/db/query/control/QueryResourceManager.java
index 36d5c37..c452300 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/query/control/QueryResourceManager.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/control/QueryResourceManager.java
@@ -158,7 +158,7 @@ public class QueryResourceManager {
 
   public QueryDataSourceV2 getQueryDataSource(Path selectedPath,
       QueryContext context)
-      throws FileNodeManagerException, ProcessorException {
+      throws FileNodeManagerException {
 
     SingleSeriesExpression singleSeriesExpression = new SingleSeriesExpression(selectedPath, null);
     QueryDataSourceV2 queryDataSource = FileNodeManagerV2.getInstance()
@@ -171,8 +171,7 @@ public class QueryResourceManager {
   }
 
   public QueryDataSourceV2 getQueryDataSourceV2(Path selectedPath,
-      QueryContext context)
-      throws FileNodeManagerException, ProcessorException {
+      QueryContext context) throws FileNodeManagerException {
 
     SingleSeriesExpression singleSeriesExpression = new SingleSeriesExpression(selectedPath, null);
     QueryDataSourceV2 queryDataSource = FileNodeManagerV2.getInstance().query(singleSeriesExpression, context);
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/factory/SeriesReaderFactoryImpl.java b/iotdb/src/main/java/org/apache/iotdb/db/query/factory/SeriesReaderFactoryImpl.java
index 127b06f..a183572 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/query/factory/SeriesReaderFactoryImpl.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/factory/SeriesReaderFactoryImpl.java
@@ -193,14 +193,9 @@ public class SeriesReaderFactoryImpl implements ISeriesReaderFactory {
 
     for (Path path : paths) {
 
-      QueryDataSourceV2 queryDataSource = null;
-      try {
-        queryDataSource = QueryResourceManager.getInstance()
-                .getQueryDataSourceV2(path,
-                        context);
-      } catch (ProcessorException e) {
-        throw new FileNodeManagerException(e);
-      }
+      QueryDataSourceV2 queryDataSource = QueryResourceManager.getInstance()
+          .getQueryDataSourceV2(path,
+              context);
 
       SeriesReaderByTimestamp mergeReaderByTimestamp = new SeriesReaderByTimestamp();
 
@@ -224,13 +219,8 @@ public class SeriesReaderFactoryImpl implements ISeriesReaderFactory {
   public IPointReader createSeriesReaderWithoutValueFilter(Path path, Filter timeFilter,
                                                            QueryContext context)
           throws FileNodeManagerException, IOException {
-    QueryDataSourceV2 queryDataSource = null;
-    try {
-      queryDataSource = QueryResourceManager.getInstance()
+    QueryDataSourceV2 queryDataSource = QueryResourceManager.getInstance()
               .getQueryDataSourceV2(path, context);
-    } catch (ProcessorException e) {
-      throw new FileNodeManagerException(e);
-    }
 
     // sequence reader for one sealed tsfile
     SequenceSeriesReader tsFilesReader;
@@ -256,13 +246,8 @@ public class SeriesReaderFactoryImpl implements ISeriesReaderFactory {
   @Override
   public IPointReader createSeriesReaderWithValueFilter(Path path, Filter filter, QueryContext context)
           throws FileNodeManagerException, IOException {
-    QueryDataSourceV2 queryDataSource = null;
-    try {
-      queryDataSource = QueryResourceManager.getInstance()
+    QueryDataSourceV2 queryDataSource = QueryResourceManager.getInstance()
               .getQueryDataSourceV2(path, context);
-    } catch (ProcessorException e) {
-      throw new FileNodeManagerException(e);
-    }
 
     // sequence reader for one sealed tsfile
     SequenceSeriesReader tsFilesReader;
diff --git a/iotdb/src/test/java/org/apache/iotdb/db/engine/filenodeV2/FileNodeProcessorV2Test.java b/iotdb/src/test/java/org/apache/iotdb/db/engine/filenodeV2/FileNodeProcessorV2Test.java
index 0dc60fd..3b96806 100644
--- a/iotdb/src/test/java/org/apache/iotdb/db/engine/filenodeV2/FileNodeProcessorV2Test.java
+++ b/iotdb/src/test/java/org/apache/iotdb/db/engine/filenodeV2/FileNodeProcessorV2Test.java
@@ -64,12 +64,8 @@ public class FileNodeProcessorV2Test {
     }
 
     processor.syncCloseFileNode();
-    QueryDataSourceV2 queryDataSource = null;
-    try {
-      queryDataSource = processor.query(deviceId, measurementId);
-    } catch (FileNodeProcessorException e) {
-      e.printStackTrace();
-    }
+    QueryDataSourceV2 queryDataSource = processor.query(deviceId, measurementId);
+
     Assert.assertEquals(queryDataSource.getSeqResources().size(), 10);
     for (TsFileResourceV2 resource : queryDataSource.getSeqResources()) {
       Assert.assertTrue(resource.isClosed());