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 2022/02/14 03:57:29 UTC

[iotdb] branch master updated: [IOTDB-2517] The count(sensor) query result is less than expected (#5043)

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/iotdb.git


The following commit(s) were added to refs/heads/master by this push:
     new 08c46e3  [IOTDB-2517] The count(sensor) query result is less than expected (#5043)
08c46e3 is described below

commit 08c46e3a2a45e1190d57ac151c6bdcb89a7b0107
Author: Liu Xuxin <37...@users.noreply.github.com>
AuthorDate: Mon Feb 14 11:56:45 2022 +0800

    [IOTDB-2517] The count(sensor) query result is less than expected (#5043)
---
 .../resources/conf/iotdb-engine.properties         |   2 +-
 .../java/org/apache/iotdb/db/conf/IoTDBConfig.java |   2 +-
 .../db/engine/storagegroup/TsFileManager.java      |  41 ++-
 .../db/engine/storagegroup/TsFileResourceList.java | 327 ++++++++-------------
 .../storagegroup/VirtualStorageGroupProcessor.java |   1 -
 5 files changed, 157 insertions(+), 216 deletions(-)

diff --git a/server/src/assembly/resources/conf/iotdb-engine.properties b/server/src/assembly/resources/conf/iotdb-engine.properties
index 97446fa..30bb052 100644
--- a/server/src/assembly/resources/conf/iotdb-engine.properties
+++ b/server/src/assembly/resources/conf/iotdb-engine.properties
@@ -417,7 +417,7 @@ timestamp_precision=ms
 
 # unsequence space compaction: only compact the unsequence files
 # Datatype: boolean
-# enable_unseq_space_compaction=false
+# enable_unseq_space_compaction=true
 
 # cross space compaction: compact the unsequence files into the overlapped sequence files
 # Datatype: boolean
diff --git a/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java b/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java
index 0bc6e99..05ede86 100644
--- a/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java
+++ b/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java
@@ -360,7 +360,7 @@ public class IoTDBConfig {
   private boolean enableSeqSpaceCompaction = true;
 
   /** Enable inner space copaction for unsequence files */
-  private boolean enableUnseqSpaceCompaction = false;
+  private boolean enableUnseqSpaceCompaction = true;
 
   /** Compact the unsequence files into the overlapped sequence files */
   private boolean enableCrossSpaceCompaction = true;
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileManager.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileManager.java
index e667792..c40149c 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileManager.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileManager.java
@@ -84,11 +84,21 @@ public class TsFileManager {
   }
 
   public TsFileResourceList getSequenceListByTimePartition(long timePartition) {
-    return sequenceFiles.computeIfAbsent(timePartition, l -> new TsFileResourceList());
+    readLock();
+    try {
+      return sequenceFiles.computeIfAbsent(timePartition, l -> new TsFileResourceList());
+    } finally {
+      readUnlock();
+    }
   }
 
   public TsFileResourceList getUnsequenceListByTimePartition(long timePartition) {
-    return unsequenceFiles.computeIfAbsent(timePartition, l -> new TsFileResourceList());
+    readLock();
+    try {
+      return unsequenceFiles.computeIfAbsent(timePartition, l -> new TsFileResourceList());
+    } finally {
+      readUnlock();
+    }
   }
 
   public Iterator<TsFileResource> getIterator(boolean sequence) {
@@ -101,7 +111,7 @@ public class TsFileManager {
   }
 
   public void remove(TsFileResource tsFileResource, boolean sequence) {
-    readLock();
+    writeLock("remove");
     try {
       Map<Long, TsFileResourceList> selectedMap = sequence ? sequenceFiles : unsequenceFiles;
       for (Map.Entry<Long, TsFileResourceList> entry : selectedMap.entrySet()) {
@@ -112,14 +122,19 @@ public class TsFileManager {
         }
       }
     } finally {
-      readUnlock();
+      writeUnlock();
     }
   }
 
   public void removeAll(List<TsFileResource> tsFileResourceList, boolean sequence) {
-    for (TsFileResource resource : tsFileResourceList) {
-      remove(resource, sequence);
-      TsFileResourceManager.getInstance().removeTsFileResource(resource);
+    writeLock("removeAll");
+    try {
+      for (TsFileResource resource : tsFileResourceList) {
+        remove(resource, sequence);
+        TsFileResourceManager.getInstance().removeTsFileResource(resource);
+      }
+    } finally {
+      writeLock("removeAll");
     }
   }
 
@@ -153,6 +168,18 @@ public class TsFileManager {
     }
   }
 
+  public void keepOrderInsert(TsFileResource tsFileResource, boolean sequence) throws IOException {
+    writeLock("keepOrderInsert");
+    try {
+      Map<Long, TsFileResourceList> selectedMap = sequence ? sequenceFiles : unsequenceFiles;
+      selectedMap
+          .computeIfAbsent(tsFileResource.getTimePartition(), o -> new TsFileResourceList())
+          .keepOrderInsert(tsFileResource);
+    } finally {
+      writeUnlock();
+    }
+  }
+
   public void addForRecover(TsFileResource tsFileResource, boolean sequence) {
     if (sequence) {
       sequenceRecoverTsFileResources.add(tsFileResource);
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResourceList.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResourceList.java
index 5ba13fc..3446cd6 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResourceList.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResourceList.java
@@ -19,7 +19,6 @@
 
 package org.apache.iotdb.db.engine.storagegroup;
 
-import org.apache.iotdb.db.exception.WriteLockFailedException;
 import org.apache.iotdb.db.utils.TestOnly;
 import org.apache.iotdb.tsfile.exception.NotImplementedException;
 
@@ -32,7 +31,6 @@ import java.util.Collection;
 import java.util.Iterator;
 import java.util.List;
 import java.util.ListIterator;
-import java.util.concurrent.TimeUnit;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 
 public class TsFileResourceList implements List<TsFileResource> {
@@ -42,44 +40,6 @@ public class TsFileResourceList implements List<TsFileResource> {
   private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
   private int count = 0;
 
-  public void readLock() {
-    lock.readLock().lock();
-  }
-
-  public void readUnlock() {
-    lock.readLock().unlock();
-  }
-
-  public void writeLock() {
-    lock.writeLock().lock();
-  }
-
-  public boolean tryWriteLock() {
-    return lock.writeLock().tryLock();
-  }
-
-  /**
-   * Acquire write lock with timeout, {@link WriteLockFailedException} will be thrown after timeout.
-   * The unit of timeout is ms.
-   */
-  public void writeLockWithTimeout(long timeout) throws WriteLockFailedException {
-    try {
-      if (lock.writeLock().tryLock(timeout, TimeUnit.MILLISECONDS)) {
-      } else {
-        throw new WriteLockFailedException(
-            String.format("cannot get write lock in %d ms", timeout));
-      }
-    } catch (InterruptedException e) {
-      LOGGER.warn(e.getMessage(), e);
-      Thread.interrupted();
-      throw new WriteLockFailedException("thread is interrupted");
-    }
-  }
-
-  public void writeUnlock() {
-    lock.writeLock().unlock();
-  }
-
   /**
    * Insert a new node before an existing node
    *
@@ -87,20 +47,15 @@ public class TsFileResourceList implements List<TsFileResource> {
    * @param newNode the file to insert
    */
   public void insertBefore(TsFileResource node, TsFileResource newNode) {
-    writeLock();
-    try {
-      newNode.prev = node.prev;
-      newNode.next = node;
-      if (node.prev == null) {
-        header = newNode;
-      } else {
-        node.prev.next = newNode;
-      }
-      node.prev = newNode;
-      count++;
-    } finally {
-      writeUnlock();
+    newNode.prev = node.prev;
+    newNode.next = node;
+    if (node.prev == null) {
+      header = newNode;
+    } else {
+      node.prev.next = newNode;
     }
+    node.prev = newNode;
+    count++;
   }
 
   /**
@@ -110,20 +65,15 @@ public class TsFileResourceList implements List<TsFileResource> {
    * @param newNode the file to insert
    */
   public void insertAfter(TsFileResource node, TsFileResource newNode) {
-    writeLock();
-    try {
-      newNode.prev = node;
-      newNode.next = node.next;
-      if (node.next == null) {
-        tail = newNode;
-      } else {
-        node.next.prev = newNode;
-      }
-      node.next = newNode;
-      count++;
-    } finally {
-      writeUnlock();
+    newNode.prev = node;
+    newNode.next = node.next;
+    if (node.next == null) {
+      tail = newNode;
+    } else {
+      node.next.prev = newNode;
     }
+    node.next = newNode;
+    count++;
   }
 
   @Override
@@ -138,24 +88,19 @@ public class TsFileResourceList implements List<TsFileResource> {
 
   @Override
   public boolean contains(Object o) {
-    readLock();
-    try {
-      if (!(o instanceof TsFileResource)) {
-        return false;
-      }
-      boolean contain = false;
-      TsFileResource current = header;
-      while (current != null) {
-        if (current.equals(o)) {
-          contain = true;
-          break;
-        }
-        current = current.next;
+    if (!(o instanceof TsFileResource)) {
+      return false;
+    }
+    boolean contain = false;
+    TsFileResource current = header;
+    while (current != null) {
+      if (current.equals(o)) {
+        contain = true;
+        break;
       }
-      return contain;
-    } finally {
-      readUnlock();
+      current = current.next;
     }
+    return contain;
   }
 
   @Override
@@ -170,23 +115,18 @@ public class TsFileResourceList implements List<TsFileResource> {
   /** Insert a new tsFileResource node to the end of List */
   @Override
   public boolean add(TsFileResource newNode) {
-    writeLock();
-    try {
-      if (newNode.prev != null || newNode.next != null) {
-        // this node already in a list
-        return false;
-      }
-      if (tail == null) {
-        header = newNode;
-        tail = newNode;
-        count++;
-      } else {
-        insertAfter(tail, newNode);
-      }
-      return true;
-    } finally {
-      writeUnlock();
+    if (newNode.prev != null || newNode.next != null) {
+      // this node already in a list
+      return false;
     }
+    if (tail == null) {
+      header = newNode;
+      tail = newNode;
+      count++;
+    } else {
+      insertAfter(tail, newNode);
+    }
+    return true;
   }
 
   /**
@@ -195,56 +135,51 @@ public class TsFileResourceList implements List<TsFileResource> {
    * node's, the new node will be inserted to the tail of the list.
    */
   public boolean keepOrderInsert(TsFileResource newNode) throws IOException {
-    writeLock();
-    try {
-      if (newNode.prev != null || newNode.next != null) {
-        // this node already in a list
-        return false;
-      }
-      if (tail == null) {
-        header = newNode;
-        tail = newNode;
-        count++;
+    if (newNode.prev != null || newNode.next != null) {
+      // this node already in a list
+      return false;
+    }
+    if (tail == null) {
+      header = newNode;
+      tail = newNode;
+      count++;
+    } else {
+      // find the position to insert of this node
+      // the list should be ordered by file timestamp
+      long timeOfNewNode =
+          TsFileNameGenerator.getTsFileName(newNode.getTsFile().getName()).getTime();
+
+      if (TsFileNameGenerator.getTsFileName(header.getTsFile().getName()).getTime()
+          > timeOfNewNode) {
+        // the timestamp of head node is greater than the new node
+        // insert it before the head
+        insertBefore(header, newNode);
+      } else if (TsFileNameGenerator.getTsFileName(tail.getTsFile().getName()).getTime()
+          < timeOfNewNode) {
+        // the timestamp of new node is greater than the tail node
+        // insert it after the tail
+        insertAfter(tail, newNode);
       } else {
-        // find the position to insert of this node
-        // the list should be ordered by file timestamp
-        long timeOfNewNode =
-            TsFileNameGenerator.getTsFileName(newNode.getTsFile().getName()).getTime();
-
-        if (TsFileNameGenerator.getTsFileName(header.getTsFile().getName()).getTime()
-            > timeOfNewNode) {
-          // the timestamp of head node is greater than the new node
-          // insert it before the head
-          insertBefore(header, newNode);
-        } else if (TsFileNameGenerator.getTsFileName(tail.getTsFile().getName()).getTime()
-            < timeOfNewNode) {
-          // the timestamp of new node is greater than the tail node
-          // insert it after the tail
-          insertAfter(tail, newNode);
-        } else {
-          // the timestamp of new node is between the timestamp of head and tail node
-          // find the first node whose timestamp is greater than new node
-          // and insert the new node before this node
-          TsFileResource currNode = header;
-          while (currNode.next != null) {
-            if (TsFileNameGenerator.getTsFileName(currNode.getTsFile().getName()).getTime()
-                > timeOfNewNode) {
-              break;
-            }
-            currNode = currNode.next;
-          }
+        // the timestamp of new node is between the timestamp of head and tail node
+        // find the first node whose timestamp is greater than new node
+        // and insert the new node before this node
+        TsFileResource currNode = header;
+        while (currNode.next != null) {
           if (TsFileNameGenerator.getTsFileName(currNode.getTsFile().getName()).getTime()
-              < timeOfNewNode) {
-            LOGGER.error("Cannot find an appropriate place to insert {}", newNode);
-          } else {
-            insertBefore(currNode, newNode);
+              > timeOfNewNode) {
+            break;
           }
+          currNode = currNode.next;
+        }
+        if (TsFileNameGenerator.getTsFileName(currNode.getTsFile().getName()).getTime()
+            < timeOfNewNode) {
+          LOGGER.error("Cannot find an appropriate place to insert {}", newNode);
+        } else {
+          insertBefore(currNode, newNode);
         }
       }
-      return true;
-    } finally {
-      writeUnlock();
     }
+    return true;
   }
 
   /**
@@ -253,42 +188,37 @@ public class TsFileResourceList implements List<TsFileResource> {
    */
   @Override
   public boolean remove(Object o) {
-    writeLock();
-    try {
-      TsFileResource tsFileResource = (TsFileResource) o;
-      if (!contains(o)) {
-        // the tsFileResource does not exist in this list
-        return false;
+    TsFileResource tsFileResource = (TsFileResource) o;
+    if (!contains(o)) {
+      // the tsFileResource does not exist in this list
+      return false;
+    }
+    if (tsFileResource.prev == null) {
+      // remove header
+      header = header.next;
+      if (header != null) {
+        header.prev = null;
+      } else {
+        // if list contains only one item, remove the header and the tail
+        tail = null;
       }
-      if (tsFileResource.prev == null) {
-        // remove header
-        header = header.next;
-        if (header != null) {
-          header.prev = null;
-        } else {
-          // if list contains only one item, remove the header and the tail
-          tail = null;
-        }
-      } else if (tsFileResource.next == null) {
-        // remove tail
-        tail = tail.prev;
-        if (tail != null) {
-          tail.next = null;
-        } else {
-          // if list contains only one item, remove the header and the tail
-          header = null;
-        }
+    } else if (tsFileResource.next == null) {
+      // remove tail
+      tail = tail.prev;
+      if (tail != null) {
+        tail.next = null;
       } else {
-        tsFileResource.prev.next = tsFileResource.next;
-        tsFileResource.next.prev = tsFileResource.prev;
+        // if list contains only one item, remove the header and the tail
+        header = null;
       }
-      tsFileResource.prev = null;
-      tsFileResource.next = null;
-      count--;
-      return true;
-    } finally {
-      writeUnlock();
+    } else {
+      tsFileResource.prev.next = tsFileResource.next;
+      tsFileResource.next.prev = tsFileResource.prev;
     }
+    tsFileResource.prev = null;
+    tsFileResource.next = null;
+    count--;
+    return true;
   }
 
   @Override
@@ -299,30 +229,20 @@ public class TsFileResourceList implements List<TsFileResource> {
   /** Only List type parameter is legal, because it is in order. */
   @Override
   public boolean addAll(Collection<? extends TsFileResource> c) {
-    writeLock();
-    try {
-      if (c instanceof List) {
-        for (TsFileResource resource : c) {
-          add(resource);
-        }
-        return true;
+    if (c instanceof List) {
+      for (TsFileResource resource : c) {
+        add(resource);
       }
-      throw new NotImplementedException();
-    } finally {
-      writeUnlock();
+      return true;
     }
+    throw new NotImplementedException();
   }
 
   @Override
   public void clear() {
-    writeLock();
-    try {
-      header = null;
-      tail = null;
-      count = 0;
-    } finally {
-      writeUnlock();
-    }
+    header = null;
+    tail = null;
+    count = 0;
   }
 
   @Override
@@ -433,22 +353,17 @@ public class TsFileResourceList implements List<TsFileResource> {
   }
 
   public List<TsFileResource> getArrayList() {
-    readLock();
-    try {
-      List<TsFileResource> list = new ArrayList<>();
-      if (header == null) {
-        return list;
-      }
-      TsFileResource current = header;
-      while (current.next != null) {
-        list.add(current);
-        current = current.next;
-      }
-      list.add(current);
+    List<TsFileResource> list = new ArrayList<>();
+    if (header == null) {
       return list;
-    } finally {
-      readUnlock();
     }
+    TsFileResource current = header;
+    while (current.next != null) {
+      list.add(current);
+      current = current.next;
+    }
+    list.add(current);
+    return list;
   }
 
   private class TsFileIterator implements Iterator<TsFileResource> {
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/VirtualStorageGroupProcessor.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/VirtualStorageGroupProcessor.java
index 5b07b32..8f07355 100755
--- a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/VirtualStorageGroupProcessor.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/VirtualStorageGroupProcessor.java
@@ -1771,7 +1771,6 @@ public class VirtualStorageGroupProcessor {
       Filter timeFilter)
       throws QueryProcessException {
     readLock();
-
     try {
       List<TsFileResource> seqResources =
           getFileResourceListForQuery(