You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@iotdb.apache.org by GitBox <gi...@apache.org> on 2020/10/06 11:21:52 UTC

[GitHub] [iotdb] JackieTien97 commented on a change in pull request #1732: [IOTDB-829] Accelerate delete timeseries

JackieTien97 commented on a change in pull request #1732:
URL: https://github.com/apache/iotdb/pull/1732#discussion_r500139338



##########
File path: server/src/main/java/org/apache/iotdb/db/engine/merge/manage/MergeResource.java
##########
@@ -184,7 +184,7 @@ public IChunkWriter getChunkWriter(MeasurementSchema measurementSchema) {
     // each path is visited only once in a merge, so the modifications can be removed after visiting
     while (modificationIterator.hasNext()) {
       Modification modification = modificationIterator.next();
-      if (modification.getPath().equals(path)) {
+      if (modification.getPath().matchFullPath(path)) {
         pathModifications.add(modification);
         modificationIterator.remove();

Review comment:
       Before, the modication record corresponds to only one timeseries, so it's safe to delete it while occuring one matched time series. However. now, one modification record could correspond to many timeseries, it won't be right to delete it.

##########
File path: server/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessor.java
##########
@@ -1440,15 +1439,26 @@ private void logDeletion(long startTime, long endTime, PartialPath deviceId, Str
     }
   }
 
+  private boolean canSkipDelete(TsFileResource tsFileResource, Deletion deletion)
+      throws MetadataException {
+    for (PartialPath p : IoTDB.metaManager.getAllTimeseriesPath(deletion.getPath())) {
+      String deviceId = p.getDevice();
+      if (tsFileResource.containsDevice(deviceId) &&
+          (deletion.getStartTime() <= tsFileResource.getStartTime(deviceId)
+              && tsFileResource.getStartTime(deviceId) <= deletion.getEndTime())
+          || (deletion.getStartTime() >= tsFileResource.getStartTime(deviceId)
+          && deletion.getStartTime() <= tsFileResource.getOrDefaultEndTime(deviceId, Long.MAX_VALUE))) {

Review comment:
       The if condition is wrong. Maybe you can change a way, think about when it will return true.Like:
   !tsFileResource.containsDevice(deviceId) ||	      if (canSkipDelete(tsFileResource, deletion)) {
             deletion.getEndTime() < tsFileResource.getStartTime(deviceId) ||	
             deletion.getStartTime() > tsFileResource.getOrDefaultEndTime(deviceId, Long.MAX_VALUE)

##########
File path: server/src/main/java/org/apache/iotdb/db/qp/executor/PlanExecutor.java
##########
@@ -713,7 +713,9 @@ public void delete(DeletePlan deletePlan) throws QueryProcessException {
     try {
       Set<PartialPath> existingPaths = new HashSet<>();
       for (PartialPath p : deletePlan.getPaths()) {
-        existingPaths.addAll(getPathsName(p));
+        if (!getPathsName(p).isEmpty()) {

Review comment:
       It seems that you have get all the timeseries for each path here, there is no need for you to call it again somewhere else, you can pass it from here.

##########
File path: server/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessor.java
##########
@@ -1440,15 +1439,26 @@ private void logDeletion(long startTime, long endTime, PartialPath deviceId, Str
     }
   }
 
+  private boolean canSkipDelete(TsFileResource tsFileResource, Deletion deletion)
+      throws MetadataException {
+    for (PartialPath p : IoTDB.metaManager.getAllTimeseriesPath(deletion.getPath())) {
+      String deviceId = p.getDevice();
+      if (tsFileResource.containsDevice(deviceId) &&
+          (deletion.getStartTime() <= tsFileResource.getStartTime(deviceId)
+              && tsFileResource.getStartTime(deviceId) <= deletion.getEndTime())
+          || (deletion.getStartTime() >= tsFileResource.getStartTime(deviceId)
+          && deletion.getStartTime() <= tsFileResource.getOrDefaultEndTime(deviceId, Long.MAX_VALUE))) {
+        return false;
+      }
+    }
+    return true;
+  }
 
   private void deleteDataInFiles(Collection<TsFileResource> tsFileResourceList, Deletion deletion,
       List<ModificationFile> updatedModFiles)
-      throws IOException {
-    String deviceId = deletion.getDevice();
+          throws IOException, MetadataException {
     for (TsFileResource tsFileResource : tsFileResourceList) {
-      if (!tsFileResource.containsDevice(deviceId) ||
-          deletion.getEndTime() < tsFileResource.getStartTime(deviceId) ||
-          deletion.getStartTime() > tsFileResource.getOrDefaultEndTime(deviceId, Long.MAX_VALUE)) {
+      if (canSkipDelete(tsFileResource, deletion)) {

Review comment:
       The canSkipDelete function will can the IoTDB.metaManager.getAllTimeseriesPath(deletion.getPath()), it shouldn't be in the for loop, and actually, you already have it in the caller of this function.

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/PartialPath.java
##########
@@ -87,6 +93,7 @@ void concatPath(String[] otherNodes) {
     int len = nodes.length;
     this.nodes = Arrays.copyOf(nodes, nodes.length + otherNodes.length);
     System.arraycopy(otherNodes, 0, nodes, len, otherNodes.length);
+    fullPath = String.join(TsFileConstant.PATH_SEPARATOR, nodes);

Review comment:
       Same as above.

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/PartialPath.java
##########
@@ -50,11 +50,17 @@ public PartialPath(String path) throws IllegalPathException {
     this.fullPath = path;
   }
 
+  public PartialPath(String device, String measurement) throws IllegalPathException {
+    this.fullPath = device + TsFileConstant.PATH_SEPARATOR + measurement;
+    this.nodes = MetaUtils.splitPathToDetachedPath(fullPath);
+  }
+
   /**
    * @param partialNodes nodes of a time series path
    */
   public PartialPath(String[] partialNodes) {
-    nodes = partialNodes;
+    this.nodes = partialNodes;
+    this.fullPath = String.join(TsFileConstant.PATH_SEPARATOR, nodes);

Review comment:
       No need to get full path here.

##########
File path: server/src/main/java/org/apache/iotdb/db/engine/StorageEngine.java
##########
@@ -447,28 +447,32 @@ public void update(String deviceId, String measurementId, long startTime, long e
     // TODO
   }
 
-  /**
-   * delete data of timeseries "{deviceId}.{measurementId}" with time <= timestamp.
-   */
-  public void delete(PartialPath deviceId, String measurementId, long startTime, long endTime)
-      throws StorageEngineException {
-    StorageGroupProcessor storageGroupProcessor = getProcessor(deviceId);
+  public void delete(PartialPath path, long startTime, long endTime)
+          throws StorageEngineException {
     try {
-      storageGroupProcessor.delete(deviceId, measurementId, startTime, endTime);
-    } catch (IOException e) {
+      List<PartialPath> sgPaths = IoTDB.metaManager.searchAllRelatedStorageGroups(path);
+      for (PartialPath storageGroupPath : sgPaths) {
+        StorageGroupProcessor storageGroupProcessor = getProcessor(storageGroupPath);
+        PartialPath newPath = path.alterPrefixPath(storageGroupPath);
+        storageGroupProcessor.delete(newPath, startTime, endTime);
+      }
+    } catch (IOException | MetadataException e) {
       throw new StorageEngineException(e.getMessage());
     }
   }
 
   /**
    * delete data of timeseries "{deviceId}.{measurementId}"
    */
-  public void deleteTimeseries(PartialPath deviceId, String measurementId)
+  public void deleteTimeseries(PartialPath path)
       throws StorageEngineException {
-    StorageGroupProcessor storageGroupProcessor = getProcessor(deviceId);
     try {
-      storageGroupProcessor.delete(deviceId, measurementId, Long.MIN_VALUE, Long.MAX_VALUE);
-    } catch (IOException e) {
+      for (PartialPath storageGroupPath : IoTDB.metaManager.searchAllRelatedStorageGroups(path)) {
+        StorageGroupProcessor storageGroupProcessor = getProcessor(storageGroupPath);
+        PartialPath newPath = path.alterPrefixPath(storageGroupPath);
+        storageGroupProcessor.delete(newPath, Long.MIN_VALUE, Long.MAX_VALUE);
+      }
+    } catch (IOException | MetadataException e) {

Review comment:
       It seems that deleteTimeseries function can directly call the delete function to avoid code duplication.

##########
File path: server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileProcessor.java
##########
@@ -226,17 +229,18 @@ public void insertTablet(InsertTabletPlan insertTabletPlan, int start, int end,
    * <p>
    * Delete data in both working MemTable and flushing MemTables.
    */
-  public void deleteDataInMemory(Deletion deletion) {
+  public void deleteDataInMemory(Deletion deletion) throws MetadataException {
     flushQueryLock.writeLock().lock();
     if (logger.isDebugEnabled()) {
       logger
           .debug(FLUSH_QUERY_WRITE_LOCKED, storageGroupName, tsFileResource.getTsFile().getName());
     }
     try {
       if (workMemTable != null) {
-        workMemTable
-            .delete(deletion.getDevice(), deletion.getMeasurement(), deletion.getStartTime(),
-                deletion.getEndTime());
+        for (PartialPath p : IoTDB.metaManager.getAllTimeseriesPath(deletion.getPath())) {

Review comment:
       no need to call getAllTimeseriesPath again, you can pass it from the caller.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org