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/23 08:59:40 UTC

[incubator-iotdb] 01/02: use modification file to filter chunk metadata when query unsealed file

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 1b55d6baafd1a7105fd0528c382008835e2533fb
Author: qiaojialin <64...@qq.com>
AuthorDate: Sun Jun 23 16:54:58 2019 +0800

    use modification file to filter chunk metadata when query unsealed file
---
 .../filenodeV2/UnsealedTsFileProcessorV2.java      | 19 ++++++++--
 .../UnsealedTsFileProcessorException.java          | 44 ++++++++++++++++++++++
 2 files changed, 59 insertions(+), 4 deletions(-)

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 6b594e1..53bd1a5 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
@@ -20,6 +20,7 @@ package org.apache.iotdb.db.engine.filenodeV2;
 
 import java.io.File;
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
 import java.util.Map;
@@ -34,11 +35,15 @@ import org.apache.iotdb.db.engine.memtable.IMemTable;
 import org.apache.iotdb.db.engine.memtable.MemSeriesLazyMerger;
 import org.apache.iotdb.db.engine.memtable.MemTableFlushTaskV2;
 import org.apache.iotdb.db.engine.memtable.MemTablePool;
+import org.apache.iotdb.db.engine.modification.Modification;
+import org.apache.iotdb.db.engine.modification.ModificationFile;
 import org.apache.iotdb.db.engine.querycontext.ReadOnlyMemChunk;
 import org.apache.iotdb.db.engine.version.VersionController;
 import org.apache.iotdb.db.exception.BufferWriteProcessorException;
+import org.apache.iotdb.db.exception.UnsealedTsFileProcessorException;
 import org.apache.iotdb.db.qp.constant.DatetimeUtils;
 import org.apache.iotdb.db.qp.physical.crud.InsertPlan;
+import org.apache.iotdb.db.utils.QueryUtils;
 import org.apache.iotdb.db.writelog.manager.MultiFileLogNodeManager;
 import org.apache.iotdb.db.writelog.node.WriteLogNode;
 import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor;
@@ -366,7 +371,8 @@ 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) {
+      String measurementId, TSDataType dataType, Map<String, String> props)
+      throws UnsealedTsFileProcessorException {
     flushQueryLock.readLock().lock();
     try {
       MemSeriesLazyMerger memSeriesLazyMerger = new MemSeriesLazyMerger();
@@ -385,10 +391,15 @@ public class UnsealedTsFileProcessorV2 {
       // so we do not need to handle it again in the following readOnlyMemChunk
       ReadOnlyMemChunk timeValuePairSorter = new ReadOnlyMemChunk(dataType, memSeriesLazyMerger,
           Collections.emptyMap());
-      return new Pair<>(timeValuePairSorter,
-          writer.getVisibleMetadatas(deviceId, measurementId, dataType));
-      //RL: TODO 后面查询处理时认为这里返回的List<ChunkMetaData>是已经被mod处理过的?
 
+      ModificationFile modificationFile = tsFileResource.getModFile();
+
+      List<ChunkMetaData> chunkMetaDataList = writer.getVisibleMetadatas(deviceId, measurementId, dataType);
+      QueryUtils.modifyChunkMetaData(chunkMetaDataList, (List<Modification>) modificationFile.getModifications());
+
+      return new Pair<>(timeValuePairSorter, chunkMetaDataList);
+    } catch (IOException e) {
+      throw new UnsealedTsFileProcessorException("read modification file failed", e);
     } finally {
       flushQueryLock.readLock().unlock();
     }
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/exception/UnsealedTsFileProcessorException.java b/iotdb/src/main/java/org/apache/iotdb/db/exception/UnsealedTsFileProcessorException.java
new file mode 100644
index 0000000..72374e9
--- /dev/null
+++ b/iotdb/src/main/java/org/apache/iotdb/db/exception/UnsealedTsFileProcessorException.java
@@ -0,0 +1,44 @@
+/**
+ * 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.iotdb.db.exception;
+
+public class UnsealedTsFileProcessorException extends ProcessorException {
+
+  private static final long serialVersionUID = 3749107630243950925L;
+
+  public UnsealedTsFileProcessorException() {
+    super();
+  }
+
+  public UnsealedTsFileProcessorException(Exception pathExcp) {
+    super(pathExcp.getMessage());
+  }
+
+  public UnsealedTsFileProcessorException(String msg) {
+    super(msg);
+  }
+
+  public UnsealedTsFileProcessorException(Throwable throwable) {
+    super(throwable);
+  }
+
+  public UnsealedTsFileProcessorException(String msg, Throwable e) {
+    super(msg, e);
+  }
+}