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/21 01:36:00 UTC

[incubator-iotdb] 01/03: add filenodeprocessor test half

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

commit eb775f862b71989e511aca69e51952fbdd9ca30b
Author: qiaojialin <64...@qq.com>
AuthorDate: Fri Jun 21 09:22:13 2019 +0800

    add filenodeprocessor test half
---
 .../db/engine/filenodeV2/FileNodeProcessorV2.java  | 14 +--------
 .../db/engine/querycontext/QueryDataSourceV2.java  | 18 +++++------
 .../engine/filenodeV2/FileNodeProcessorV2Test.java | 36 ++++++++++++++++++++--
 3 files changed, 44 insertions(+), 24 deletions(-)

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 b3ef56f..02cea35 100644
--- 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
@@ -109,25 +109,13 @@ public class FileNodeProcessorV2 {
           "directory {}", storageGroupName, storageGroupDir.getAbsolutePath());
     }
 
-    /**
-     * restore
-     */
-    File restoreFolder = new File(absoluteBaseDir + storageGroupName);
-    if (!restoreFolder.exists()) {
-      restoreFolder.mkdirs();
-      LOGGER.info("The restore directory of the filenode processor {} doesn't exist. Create new " +
-          "directory {}", storageGroupName, restoreFolder.getAbsolutePath());
-    }
-
-    String absoluteFileNodeRestoreFilePath = new File(restoreFolder, storageGroupName + RESTORE_FILE_SUFFIX).getAbsolutePath();
-
     recovery();
 
     /**
      * version controller
      */
     try {
-      versionController = new SimpleFileVersionController(restoreFolder.getPath());
+      versionController = new SimpleFileVersionController(storageGroupDir.getAbsolutePath());
     } catch (IOException e) {
       throw new FileNodeProcessorException(e);
     }
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/engine/querycontext/QueryDataSourceV2.java b/iotdb/src/main/java/org/apache/iotdb/db/engine/querycontext/QueryDataSourceV2.java
index b39e3e1..fb12950 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/engine/querycontext/QueryDataSourceV2.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/engine/querycontext/QueryDataSourceV2.java
@@ -21,22 +21,22 @@ package org.apache.iotdb.db.engine.querycontext;
 public class QueryDataSourceV2 {
 
   // sequence data source
-  private GlobalSortedSeriesDataSourceV2 seriesDataSource;
+  private GlobalSortedSeriesDataSourceV2 sequenceDataSource;
 
   // unSequence data source
-  private GlobalSortedSeriesDataSourceV2 overflowSeriesDataSource;
+  private GlobalSortedSeriesDataSourceV2 unSequenceDataSource;
 
-  public QueryDataSourceV2(GlobalSortedSeriesDataSourceV2 seriesDataSource,
-      GlobalSortedSeriesDataSourceV2 overflowSeriesDataSource) {
-    this.seriesDataSource = seriesDataSource;
-    this.overflowSeriesDataSource = overflowSeriesDataSource;
+  public QueryDataSourceV2(GlobalSortedSeriesDataSourceV2 sequenceDataSource,
+      GlobalSortedSeriesDataSourceV2 unSequenceDataSource) {
+    this.sequenceDataSource = sequenceDataSource;
+    this.unSequenceDataSource = unSequenceDataSource;
   }
 
   public GlobalSortedSeriesDataSourceV2 getSeqDataSource() {
-    return seriesDataSource;
+    return sequenceDataSource;
   }
 
-  public GlobalSortedSeriesDataSourceV2 getOverflowSeriesDataSource() {
-    return overflowSeriesDataSource;
+  public GlobalSortedSeriesDataSourceV2 getUnSequenceDataSource() {
+    return unSequenceDataSource;
   }
 }
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 3810b56..385b86e 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
@@ -20,27 +20,38 @@ package org.apache.iotdb.db.engine.filenodeV2;
 
 import java.util.Collections;
 import java.util.Map;
+import java.util.concurrent.ExecutionException;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.engine.MetadataManagerHelper;
+import org.apache.iotdb.db.engine.bufferwrite.BufferWriteProcessor;
+import org.apache.iotdb.db.engine.filenode.FileNodeProcessor;
+import org.apache.iotdb.db.engine.querycontext.QueryDataSourceV2;
+import org.apache.iotdb.db.exception.BufferWriteProcessorException;
+import org.apache.iotdb.db.exception.FileNodeProcessorException;
 import org.apache.iotdb.db.utils.EnvironmentUtils;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.write.record.TSRecord;
+import org.apache.iotdb.tsfile.write.record.datapoint.DataPoint;
 import org.junit.After;
+import org.junit.Assert;
 import org.junit.Before;
+import org.junit.Test;
 
 public class FileNodeProcessorV2Test {
 
-  private UnsealedTsFileProcessorV2 processor;
   private String storageGroup = "storage_group1";
   private String baseDir = "data";
   private String deviceId = "root.vehicle.d0";
   private String measurementId = "s0";
   private TSDataType dataType = TSDataType.INT32;
   private Map<String, String> props = Collections.emptyMap();
-
+  private FileNodeProcessorV2 processor;
 
   @Before
   public void setUp() throws Exception {
     MetadataManagerHelper.initMetadata();
     EnvironmentUtils.envSetUp();
+    processor = new FileNodeProcessorV2(baseDir, storageGroup);
   }
 
   @After
@@ -50,4 +61,25 @@ public class FileNodeProcessorV2Test {
   }
 
 
+  @Test
+  public void testAsyncClose()
+      throws FileNodeProcessorException, BufferWriteProcessorException, ExecutionException, InterruptedException {
+
+
+    for (int j = 1; j <= 10; j++) {
+      TSRecord record = new TSRecord(j, deviceId);
+      for (int i = 0; i < 10; i++) {
+        record.addTuple(DataPoint.getDataPoint(TSDataType.INT32, measurementId, String.valueOf(i)));
+      }
+      processor.insert(record);
+      processor.asyncForceClose();
+    }
+
+    QueryDataSourceV2 queryDataSource = processor.query(deviceId, measurementId);
+
+    Assert.assertEquals(queryDataSource.getSeqDataSource().getQueryTsFiles().size(), 10);
+
+  }
+
+
 }
\ No newline at end of file