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/06/21 16:29:23 UTC

[GitHub] [incubator-iotdb] wshao08 opened a new pull request #1400: [IOTDB-627]Support range deletion for timeseries

wshao08 opened a new pull request #1400:
URL: https://github.com/apache/incubator-iotdb/pull/1400


   JIRA issue: https://issues.apache.org/jira/browse/IOTDB-627
   
   Timeseries deletion support more SQL expressions:
   
   delete from root.sg.d1.s1 where time < 10
   delete from root.sg.d1.s1 where time <= 10
   delete from root.sg.d1.s1 where time < 20 and time > 10
   delete from root.sg.d1.s1 where time <= 20 and time >= 10
   delete from root.sg.d1.s1 where time > 20
   delete from root.sg.d1.s1 where time >= 20
   delete from root.sg.d1.s1 where time = 20


----------------------------------------------------------------
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



[GitHub] [incubator-iotdb] wshao08 commented on a change in pull request #1400: [IOTDB-627]Support range deletion for timeseries

Posted by GitBox <gi...@apache.org>.
wshao08 commented on a change in pull request #1400:
URL: https://github.com/apache/incubator-iotdb/pull/1400#discussion_r451430864



##########
File path: tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/page/PageReader.java
##########
@@ -159,12 +162,25 @@ public void setFilter(Filter filter) {
     this.filter = filter;
   }
 
-  public void setDeletedAt(long deletedAt) {
-    this.deletedAt = deletedAt;
+  public void setDeleteIntervalList(List<TimeRange> list) {
+    this.deleteIntervalList = list;
   }
 
   @Override
   public boolean isModified() {
     return pageHeader.isModified();
   }
+
+  private boolean isDeleted(long timestamp, int deleteCursor) {

Review comment:
       Fixed, thanks!




----------------------------------------------------------------
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



[GitHub] [incubator-iotdb] wshao08 commented on a change in pull request #1400: [IOTDB-627]Support range deletion for timeseries

Posted by GitBox <gi...@apache.org>.
wshao08 commented on a change in pull request #1400:
URL: https://github.com/apache/incubator-iotdb/pull/1400#discussion_r447377269



##########
File path: tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/ChunkMetadata.java
##########
@@ -171,12 +174,22 @@ public void setVersion(long version) {
     this.version = version;
   }
 
-  public long getDeletedAt() {
-    return deletedAt;
+  public List<Pair<Long, Long>> getDeleteRangeList() {
+    if (deleteRangeList == null) {

Review comment:
       Fixed




----------------------------------------------------------------
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



[GitHub] [incubator-iotdb] jixuan1989 commented on a change in pull request #1400: [IOTDB-627]Support range deletion for timeseries

Posted by GitBox <gi...@apache.org>.
jixuan1989 commented on a change in pull request #1400:
URL: https://github.com/apache/incubator-iotdb/pull/1400#discussion_r446758281



##########
File path: server/src/main/java/org/apache/iotdb/db/engine/memtable/AbstractMemTable.java
##########
@@ -203,38 +204,38 @@ public ReadOnlyMemChunk query(String deviceId, String measurement, TSDataType da
     if (!checkPath(deviceId, measurement)) {
       return null;
     }
-    long undeletedTime = findUndeletedTime(deviceId, measurement, timeLowerBound);
+    List<Pair<Long, Long>> deletionList = findUndeletedTime(deviceId, measurement, timeLowerBound);
     IWritableMemChunk memChunk = memTableMap.get(deviceId).get(measurement);
     TVList chunkCopy = memChunk.getTVList().clone();
 
-    chunkCopy.setTimeOffset(undeletedTime);
+    chunkCopy.setDeletionList(deletionList);
     return new ReadOnlyMemChunk(measurement, dataType, encoding, chunkCopy, props, getVersion());
   }
 
 
-  private long findUndeletedTime(String deviceId, String measurement, long timeLowerBound) {
-    long undeletedTime = Long.MIN_VALUE;
+  private List<Pair<Long, Long>> findUndeletedTime(String deviceId, String measurement,
+      long timeLowerBound) {
+    List<Pair<Long, Long>> deletionList = new ArrayList<>();
     for (Modification modification : modifications) {
-      if (modification instanceof Deletion) {
-        Deletion deletion = (Deletion) modification;
-        if (deletion.getDevice().equals(deviceId) && deletion.getMeasurement().equals(measurement)
-            && deletion.getTimestamp() > undeletedTime) {
-          undeletedTime = deletion.getTimestamp();
-        }
+      Deletion deletion = (Deletion) modification;

Review comment:
       if (modification instanceof Deletion)

##########
File path: server/src/main/java/org/apache/iotdb/db/engine/modification/io/LocalTextModificationAccessor.java
##########
@@ -124,28 +124,42 @@ private static Modification decodeModification(String src) throws IOException {
   private static String encodeDeletion(Deletion del) {
     return del.getType().toString() + SEPARATOR + del.getPathString()
         + SEPARATOR + del.getVersionNum() + SEPARATOR
-        + del.getTimestamp();
+        + del.getStartTime() + SEPARATOR + del.getEndTime();
   }
 
   private static Deletion decodeDeletion(String[] fields) throws IOException {
-    if (fields.length != 4) {
+    if (fields.length != 4 && fields.length != 5) {

Review comment:
       !=5 first, then !=4 

##########
File path: server/src/main/java/org/apache/iotdb/db/utils/QueryUtils.java
##########
@@ -45,46 +49,52 @@ private QueryUtils() {
    */
   public static void modifyChunkMetaData(List<ChunkMetadata> chunkMetaData,
                                          List<Modification> modifications) {
-    int modIndex = 0;
+    List<Modification> sortedModifications = sortModifications(modifications);
 
     for (int metaIndex = 0; metaIndex < chunkMetaData.size(); metaIndex++) {
       ChunkMetadata metaData = chunkMetaData.get(metaIndex);
-      for (int j = modIndex; j < modifications.size(); j++) {
-        // iterate each modification to find the max deletion time
-        Modification modification = modifications.get(j);
+      for (Modification modification : sortedModifications) {
         if (modification.getVersionNum() > metaData.getVersion()) {
-          // this modification is after the Chunk, try modifying the chunk
-          // if this modification succeeds, update modIndex so in the next loop the previous
-          // modifications will not be examined
-          modIndex = doModifyChunkMetaData(modification, metaData)? j : modIndex;
-        } else {
-          // skip old modifications for next metadata
-          modIndex++;
+          doModifyChunkMetaData(modification, metaData);
         }
       }
     }
     // remove chunks that are completely deleted
     chunkMetaData.removeIf(metaData -> {
-      if (metaData.getDeletedAt() >= metaData.getEndTime()) {
-        return true;
-      } else {
-        if (metaData.getDeletedAt() >= metaData.getStartTime()) {
+      long lower = metaData.getStartTime();
+      long upper = metaData.getEndTime();
+      for (Pair<Long, Long> range : metaData.getDeleteRangeList()) {
+        if (upper < range.left) {
+          break;
+        }
+        if (range.left <= lower && lower <= range.right) {
+          metaData.setModified(true);
+          if (upper <= range.right) {
+            return true;
+          }
+          lower = range.right;
+        } else if (lower < range.left) {
           metaData.setModified(true);
+          break;
         }
-        return false;
       }
+      return false;
     });
   }
 
-  private static boolean doModifyChunkMetaData(Modification modification, ChunkMetadata metaData) {
+  private static LinkedList<Modification> sortModifications(List<Modification> modifications) {
+    return modifications.stream()
+        .sorted(

Review comment:
       .filter( x -> x instanceof Deletion)

##########
File path: server/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java
##########
@@ -46,7 +47,7 @@
   /**
    * this field is effective only in the Tvlist in a RealOnlyMemChunk.
    */
-  private long timeOffset = Long.MIN_VALUE;
+  private List<Pair<Long, Long>> deletionList = new ArrayList<>();

Review comment:
       Deletion operation is rare.
   Put `new ArrayList()` to `delete()`, add check whether the list == null before using it. 

##########
File path: tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/page/PageReader.java
##########
@@ -61,9 +64,9 @@
   private Filter filter;
 
   /**
-   * Data whose timestamp <= deletedAt should be considered deleted(not be returned).
+   * A list of deleted intervals.
    */
-  private long deletedAt = Long.MIN_VALUE;
+  private List<Pair<Long, Long>> deleteRangeList = new ArrayList<>();

Review comment:
       similar, deleteRangeList usually be null.

##########
File path: server/src/main/java/org/apache/iotdb/db/engine/memtable/AbstractMemTable.java
##########
@@ -203,38 +204,38 @@ public ReadOnlyMemChunk query(String deviceId, String measurement, TSDataType da
     if (!checkPath(deviceId, measurement)) {
       return null;
     }
-    long undeletedTime = findUndeletedTime(deviceId, measurement, timeLowerBound);
+    List<Pair<Long, Long>> deletionList = findUndeletedTime(deviceId, measurement, timeLowerBound);
     IWritableMemChunk memChunk = memTableMap.get(deviceId).get(measurement);
     TVList chunkCopy = memChunk.getTVList().clone();
 
-    chunkCopy.setTimeOffset(undeletedTime);
+    chunkCopy.setDeletionList(deletionList);
     return new ReadOnlyMemChunk(measurement, dataType, encoding, chunkCopy, props, getVersion());
   }
 
 
-  private long findUndeletedTime(String deviceId, String measurement, long timeLowerBound) {
-    long undeletedTime = Long.MIN_VALUE;
+  private List<Pair<Long, Long>> findUndeletedTime(String deviceId, String measurement,

Review comment:
       the method name should be changed.




----------------------------------------------------------------
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



[GitHub] [incubator-iotdb] wshao08 commented on a change in pull request #1400: [IOTDB-627]Support range deletion for timeseries

Posted by GitBox <gi...@apache.org>.
wshao08 commented on a change in pull request #1400:
URL: https://github.com/apache/incubator-iotdb/pull/1400#discussion_r451430217



##########
File path: server/src/main/java/org/apache/iotdb/db/utils/QueryUtils.java
##########
@@ -19,6 +19,9 @@
 
 package org.apache.iotdb.db.utils;
 
+import java.util.Comparator;
+import java.util.LinkedList;
+import java.util.stream.Collectors;

Review comment:
       Fixed

##########
File path: server/src/test/java/org/apache/iotdb/db/integration/IoTDBDeletionIT.java
##########
@@ -164,6 +164,43 @@ public void testDelAfterFlush() throws SQLException {
     }
   }
 
+  @Test
+  public void testRangeDelete() throws SQLException {
+    prepareData();
+    try (Connection connection = DriverManager
+        .getConnection(Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root",
+            "root");
+        Statement statement = connection.createStatement()) {
+
+      statement.execute("DELETE FROM root.vehicle.d0.s0  WHERE time <= 300");

Review comment:
       Fixed

##########
File path: tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/TimeRange.java
##########
@@ -103,6 +103,8 @@ public boolean contains(long min, long max) {
     return this.min <= min && this.max >= max;
   }
 
+  public boolean contains(long time) {return this.min <= time && time <= this.max;}

Review comment:
       Fixed

##########
File path: server/src/test/java/org/apache/iotdb/db/qp/plan/LogicalPlanSmallTest.java
##########
@@ -237,5 +239,100 @@ public void testChineseCharacter() {
     Assert.assertEquals(paths, ((QueryOperator) operator).getSelectedPaths());
   }
 
+  @Test
+  public void testRangeDelete() {
+    String sql1 = "delete from root.d1.s1 where time>=1 and time < 3";
+    Operator op = parseDriver.parse(sql1, IoTDBDescriptor.getInstance().getConfig().getZoneID());
+    Assert.assertEquals(DeleteDataOperator.class, op.getClass());
+    ArrayList<Path> paths = new ArrayList<>();
+    paths.add(new Path("root.d1.s1"));
+    Assert.assertEquals(paths, ((DeleteDataOperator) op).getSelectedPaths());
+    Assert.assertEquals(1, ((DeleteDataOperator) op).getStartTime());
+    Assert.assertEquals(2, ((DeleteDataOperator) op).getEndTime());
+
+    String sql2 = "delete from root.d1.s1 where time>=1";
+    op = parseDriver.parse(sql2, IoTDBDescriptor.getInstance().getConfig().getZoneID());
+    Assert.assertEquals(paths, ((DeleteDataOperator) op).getSelectedPaths());
+    Assert.assertEquals(1, ((DeleteDataOperator) op).getStartTime());
+    Assert.assertEquals(Long.MAX_VALUE, ((DeleteDataOperator) op).getEndTime());
+
+    String sql3 = "delete from root.d1.s1 where time>1";
+    op = parseDriver.parse(sql3, IoTDBDescriptor.getInstance().getConfig().getZoneID());
+    Assert.assertEquals(paths, ((DeleteDataOperator) op).getSelectedPaths());
+    Assert.assertEquals(2, ((DeleteDataOperator) op).getStartTime());
+    Assert.assertEquals(Long.MAX_VALUE, ((DeleteDataOperator) op).getEndTime());
+
+    String sql4 = "delete from root.d1.s1 where time <= 1";
+    op = parseDriver.parse(sql4, IoTDBDescriptor.getInstance().getConfig().getZoneID());
+    Assert.assertEquals(paths, ((DeleteDataOperator) op).getSelectedPaths());
+    Assert.assertEquals(Long.MIN_VALUE, ((DeleteDataOperator) op).getStartTime());
+    Assert.assertEquals(1, ((DeleteDataOperator) op).getEndTime());
+
+    String sql5 = "delete from root.d1.s1 where time<1";
+    op = parseDriver.parse(sql5, IoTDBDescriptor.getInstance().getConfig().getZoneID());
+    Assert.assertEquals(paths, ((DeleteDataOperator) op).getSelectedPaths());
+    Assert.assertEquals(Long.MIN_VALUE, ((DeleteDataOperator) op).getStartTime());
+    Assert.assertEquals(0, ((DeleteDataOperator) op).getEndTime());
+
+    String sql6 = "delete from root.d1.s1 where time = 3";
+    op = parseDriver.parse(sql6, IoTDBDescriptor.getInstance().getConfig().getZoneID());
+    Assert.assertEquals(paths, ((DeleteDataOperator) op).getSelectedPaths());
+    Assert.assertEquals(3, ((DeleteDataOperator) op).getStartTime());
+    Assert.assertEquals(3, ((DeleteDataOperator) op).getEndTime());
 
+    String sql7 = "delete from root.d1.s1 where time = 1 and time < -1";
+    String errorMsg = null;
+    try {
+      op = parseDriver.parse(sql7, IoTDBDescriptor.getInstance().getConfig().getZoneID());
+    } catch (RuntimeException e) {
+      errorMsg = e.getMessage();
+    }
+    Assert.assertEquals(errorMsg, "Invalid delete range: [1, -2]");
+
+    String sql8 = "delete from root.d1.s1 where time > 5 and time <= 0";
+    try {
+      op = parseDriver.parse(sql8, IoTDBDescriptor.getInstance().getConfig().getZoneID());
+    } catch (RuntimeException e) {
+      errorMsg = e.getMessage();
+    }
+    Assert.assertEquals(errorMsg, "Invalid delete range: [6, 0]");
+  }
+
+  @Test
+  public void testErrorDeleteRange() {
+    String sql = "delete from root.d1.s1 where time>=1 and time < 3 or time >1";
+    String errorMsg = null;
+    try {
+      parseDriver.parse(sql, IoTDBDescriptor.getInstance().getConfig().getZoneID());
+    } catch (SQLParserException e) {
+      errorMsg = e.getMessage();
+    }
+    Assert.assertEquals(
+        "For delete command, where clause can only contain atomic expressions like : "
+            + "time > XXX, time <= XXX, or And with two atomic expressions",
+        errorMsg);
+
+    sql = "delete from root.d1.s1 where time>=1 or time < 3";
+    errorMsg = null;
+    try {
+      parseDriver.parse(sql, IoTDBDescriptor.getInstance().getConfig().getZoneID());
+    } catch (SQLParserException e) {
+      errorMsg = e.getMessage();
+    }
+    Assert.assertEquals(
+        "For delete command, where clause can only contain atomic expressions like : "
+            + "time > XXX, time <= XXX, or And with two atomic expressions",
+        errorMsg);
+
+    /*
+    sql = "delete from root.d1.s1 where time<=1 and time > 3";
+    errorMsg = null;
+    try {
+      parseDriver.parse(sql, IoTDBDescriptor.getInstance().getConfig().getZoneID());
+    } catch (SQLParserException e) {
+      errorMsg = e.getMessage();
+    }
+    Assert.assertEquals("Unreachable deleted time interval", errorMsg);
+     */

Review comment:
       Fixed




----------------------------------------------------------------
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



[GitHub] [incubator-iotdb] wshao08 commented on a change in pull request #1400: [IOTDB-627]Support range deletion for timeseries

Posted by GitBox <gi...@apache.org>.
wshao08 commented on a change in pull request #1400:
URL: https://github.com/apache/incubator-iotdb/pull/1400#discussion_r450587979



##########
File path: session/src/main/java/org/apache/iotdb/session/Session.java
##########
@@ -747,14 +747,37 @@ public void deleteData(String path, long time)
    * delete data <= time in multiple timeseries
    *
    * @param paths data in which time series to delete
-   * @param time  data with time stamp less than or equal to time will be deleted
+   * @param endTime data with time stamp less than or equal to time will be deleted
    */
-  public void deleteData(List<String> paths, long time)
+  public void deleteData(List<String> paths, long endTime)
       throws IoTDBConnectionException, StatementExecutionException {
     TSDeleteDataReq request = new TSDeleteDataReq();
     request.setSessionId(sessionId);
     request.setPaths(paths);
-    request.setTimestamp(time);
+    request.setStartTime(Long.MIN_VALUE);
+    request.setEndTime(endTime);
+

Review comment:
       Fixed




----------------------------------------------------------------
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



[GitHub] [incubator-iotdb] wshao08 commented on a change in pull request #1400: [IOTDB-627]Support range deletion for timeseries

Posted by GitBox <gi...@apache.org>.
wshao08 commented on a change in pull request #1400:
URL: https://github.com/apache/incubator-iotdb/pull/1400#discussion_r451306376



##########
File path: server/src/main/java/org/apache/iotdb/db/utils/QueryUtils.java
##########
@@ -45,46 +49,52 @@ private QueryUtils() {
    */
   public static void modifyChunkMetaData(List<ChunkMetadata> chunkMetaData,
                                          List<Modification> modifications) {
-    int modIndex = 0;
+    List<Modification> sortedModifications = sortModifications(modifications);
 
     for (int metaIndex = 0; metaIndex < chunkMetaData.size(); metaIndex++) {
       ChunkMetadata metaData = chunkMetaData.get(metaIndex);
-      for (int j = modIndex; j < modifications.size(); j++) {
-        // iterate each modification to find the max deletion time
-        Modification modification = modifications.get(j);
+      for (Modification modification : sortedModifications) {
         if (modification.getVersionNum() > metaData.getVersion()) {
-          // this modification is after the Chunk, try modifying the chunk
-          // if this modification succeeds, update modIndex so in the next loop the previous
-          // modifications will not be examined
-          modIndex = doModifyChunkMetaData(modification, metaData)? j : modIndex;
-        } else {
-          // skip old modifications for next metadata
-          modIndex++;
+          doModifyChunkMetaData(modification, metaData);
         }
       }
     }
     // remove chunks that are completely deleted
     chunkMetaData.removeIf(metaData -> {
-      if (metaData.getDeletedAt() >= metaData.getEndTime()) {
-        return true;
-      } else {
-        if (metaData.getDeletedAt() >= metaData.getStartTime()) {
-          metaData.setModified(true);
+      long lower = metaData.getStartTime();
+      long upper = metaData.getEndTime();
+      if (metaData.getDeleteIntervalList() != null) {
+        for (Pair<Long, Long> range : metaData.getDeleteIntervalList()) {
+          if (upper < range.left) {
+            break;
+          }
+          if (range.left <= lower && lower <= range.right) {
+            metaData.setModified(true);
+            if (upper <= range.right) {
+              return true;
+            }
+            lower = range.right;
+          } else if (lower < range.left) {
+            metaData.setModified(true);
+            break;
+          }
         }
-        return false;
       }
+      return false;
     });
   }
 
-  private static boolean doModifyChunkMetaData(Modification modification, ChunkMetadata metaData) {
+  private static LinkedList<Modification> sortModifications(List<Modification> modifications) {

Review comment:
       Change to sort and merge intervals when added into ChunkMetadata




----------------------------------------------------------------
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



[GitHub] [incubator-iotdb] wshao08 commented on a change in pull request #1400: [IOTDB-627]Support range deletion for timeseries

Posted by GitBox <gi...@apache.org>.
wshao08 commented on a change in pull request #1400:
URL: https://github.com/apache/incubator-iotdb/pull/1400#discussion_r451304867



##########
File path: server/src/main/java/org/apache/iotdb/db/engine/modification/io/LocalTextModificationAccessor.java
##########
@@ -124,28 +124,42 @@ private static Modification decodeModification(String src) throws IOException {
   private static String encodeDeletion(Deletion del) {
     return del.getType().toString() + SEPARATOR + del.getPathString()
         + SEPARATOR + del.getVersionNum() + SEPARATOR
-        + del.getTimestamp();
+        + del.getStartTime() + SEPARATOR + del.getEndTime();
   }
 
   private static Deletion decodeDeletion(String[] fields) throws IOException {
-    if (fields.length != 4) {
+    if (fields.length != 5 && fields.length != 4) {
       throw new IOException("Incorrect deletion fields number: " + fields.length);
     }
 
     String path = fields[1];
     long versionNum;
-    long timestamp;
+    long startTimestamp = Long.MIN_VALUE;
+    long endTimestamp;
     try {
       versionNum = Long.parseLong(fields[2]);
     } catch (NumberFormatException e) {
       throw new IOException("Invalid version number: " + fields[2]);
     }
-    try {
-      timestamp = Long.parseLong(fields[3]);
-    } catch (NumberFormatException e) {
-      throw new IOException("Invalid timestamp: " + fields[3]);
+    if (fields.length == 4) {
+      try {
+        endTimestamp = Long.parseLong(fields[3]);
+      } catch (NumberFormatException e) {
+        throw new IOException("Invalid timestamp: " + fields[3]);
+      }
+    } else {
+      try {
+        startTimestamp = Long.parseLong(fields[3]);
+      } catch (NumberFormatException e) {
+        throw new IOException("Invalid timestamp: " + fields[3]);
+      }
+      try {
+        endTimestamp = Long.parseLong(fields[4]);
+      } catch (NumberFormatException e) {
+        throw new IOException("Invalid timestamp: " + fields[4]);
+      }

Review comment:
       Fixed




----------------------------------------------------------------
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



[GitHub] [incubator-iotdb] wshao08 commented on a change in pull request #1400: [IOTDB-627]Support range deletion for timeseries

Posted by GitBox <gi...@apache.org>.
wshao08 commented on a change in pull request #1400:
URL: https://github.com/apache/incubator-iotdb/pull/1400#discussion_r451304753



##########
File path: server/src/main/java/org/apache/iotdb/db/engine/memtable/AbstractMemTable.java
##########
@@ -203,38 +204,40 @@ public ReadOnlyMemChunk query(String deviceId, String measurement, TSDataType da
     if (!checkPath(deviceId, measurement)) {
       return null;
     }
-    long undeletedTime = findUndeletedTime(deviceId, measurement, timeLowerBound);
+    List<Pair<Long, Long>> deletionList = constructDeletionList(deviceId, measurement, timeLowerBound);
     IWritableMemChunk memChunk = memTableMap.get(deviceId).get(measurement);
     TVList chunkCopy = memChunk.getTVList().clone();
 
-    chunkCopy.setTimeOffset(undeletedTime);
+    chunkCopy.setDeletionList(deletionList);
     return new ReadOnlyMemChunk(measurement, dataType, encoding, chunkCopy, props, getVersion());
   }
 
 
-  private long findUndeletedTime(String deviceId, String measurement, long timeLowerBound) {
-    long undeletedTime = Long.MIN_VALUE;
+  private List<Pair<Long, Long>> constructDeletionList(String deviceId, String measurement,
+      long timeLowerBound) {
+    List<Pair<Long, Long>> deletionList = new ArrayList<>();
     for (Modification modification : modifications) {
       if (modification instanceof Deletion) {
         Deletion deletion = (Deletion) modification;
         if (deletion.getDevice().equals(deviceId) && deletion.getMeasurement().equals(measurement)
-            && deletion.getTimestamp() > undeletedTime) {
-          undeletedTime = deletion.getTimestamp();
+            && deletion.getEndTime() > timeLowerBound) {
+          long lowerBound = Math.max(deletion.getStartTime(), timeLowerBound);
+          deletionList.add(new Pair<>(lowerBound, deletion.getEndTime()));

Review comment:
       Fixed

##########
File path: server/src/main/java/org/apache/iotdb/db/engine/modification/Deletion.java
##########
@@ -28,21 +28,36 @@
 public class Deletion extends Modification {
 
   /**
-   * data whose timestamp <= this field are to be deleted.
+   * data within the interval [startTime, endTime] are to be deleted.
    */
-  private long timestamp;
+  private long startTime;
+  private long endTime;
 
-  public Deletion(Path path, long versionNum, long timestamp) {
+  public Deletion(Path path, long versionNum, long endTime) {
     super(Type.DELETION, path, versionNum);
-    this.timestamp = timestamp;
+    this.endTime = endTime;

Review comment:
       Fixed




----------------------------------------------------------------
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



[GitHub] [incubator-iotdb] wshao08 commented on a change in pull request #1400: [IOTDB-627]Support range deletion for timeseries

Posted by GitBox <gi...@apache.org>.
wshao08 commented on a change in pull request #1400:
URL: https://github.com/apache/incubator-iotdb/pull/1400#discussion_r451305062



##########
File path: tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/chunk/ChunkReader.java
##########
@@ -131,10 +132,25 @@ private void skipBytesInStreamByLength(long length) {
   }
 
   public boolean pageSatisfied(PageHeader pageHeader) {
-    if (pageHeader.getEndTime() <= deletedAt) {
-      return false;
-    } else if (pageHeader.getStartTime() <= deletedAt) {
-      pageHeader.setModified(true);
+    long lower = pageHeader.getStartTime();
+    long upper = pageHeader.getEndTime();
+    // deleteIntervalList is sorted in terms of startTime
+    if (deleteIntervalList != null) {
+      for (Pair<Long, Long> range : deleteIntervalList) {
+        if (upper < range.left) {
+          break;
+        }
+        if (range.left <= lower && lower <= range.right) {
+          pageHeader.setModified(true);
+          if (upper <= range.right) {
+            return true;

Review comment:
       Fixed

##########
File path: tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/chunk/ChunkReaderByTimestamp.java
##########
@@ -35,7 +35,10 @@ public ChunkReaderByTimestamp(Chunk chunk) throws IOException {
   public boolean pageSatisfied(PageHeader pageHeader) {
     long maxTimestamp = pageHeader.getEndTime();
     // if maxTimestamp > currentTimestamp, this page should NOT be skipped
-    return maxTimestamp >= currentTimestamp && maxTimestamp > deletedAt;
+    // return maxTimestamp >= currentTimestamp && super.pageSatisfied(pageHeader);

Review comment:
       Done




----------------------------------------------------------------
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



[GitHub] [incubator-iotdb] wshao08 commented on a change in pull request #1400: [IOTDB-627]Support range deletion for timeseries

Posted by GitBox <gi...@apache.org>.
wshao08 commented on a change in pull request #1400:
URL: https://github.com/apache/incubator-iotdb/pull/1400#discussion_r447356854



##########
File path: server/src/main/java/org/apache/iotdb/db/engine/memtable/AbstractMemTable.java
##########
@@ -203,38 +204,38 @@ public ReadOnlyMemChunk query(String deviceId, String measurement, TSDataType da
     if (!checkPath(deviceId, measurement)) {
       return null;
     }
-    long undeletedTime = findUndeletedTime(deviceId, measurement, timeLowerBound);
+    List<Pair<Long, Long>> deletionList = findUndeletedTime(deviceId, measurement, timeLowerBound);
     IWritableMemChunk memChunk = memTableMap.get(deviceId).get(measurement);
     TVList chunkCopy = memChunk.getTVList().clone();
 
-    chunkCopy.setTimeOffset(undeletedTime);
+    chunkCopy.setDeletionList(deletionList);
     return new ReadOnlyMemChunk(measurement, dataType, encoding, chunkCopy, props, getVersion());
   }
 
 
-  private long findUndeletedTime(String deviceId, String measurement, long timeLowerBound) {
-    long undeletedTime = Long.MIN_VALUE;
+  private List<Pair<Long, Long>> findUndeletedTime(String deviceId, String measurement,
+      long timeLowerBound) {
+    List<Pair<Long, Long>> deletionList = new ArrayList<>();
     for (Modification modification : modifications) {
-      if (modification instanceof Deletion) {
-        Deletion deletion = (Deletion) modification;
-        if (deletion.getDevice().equals(deviceId) && deletion.getMeasurement().equals(measurement)
-            && deletion.getTimestamp() > undeletedTime) {
-          undeletedTime = deletion.getTimestamp();
-        }
+      Deletion deletion = (Deletion) modification;

Review comment:
       Fixed




----------------------------------------------------------------
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



[GitHub] [incubator-iotdb] qiaojialin merged pull request #1400: [IOTDB-627]Support range deletion for timeseries

Posted by GitBox <gi...@apache.org>.
qiaojialin merged pull request #1400:
URL: https://github.com/apache/incubator-iotdb/pull/1400


   


----------------------------------------------------------------
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



[GitHub] [incubator-iotdb] jixuan1989 commented on a change in pull request #1400: [IOTDB-627]Support range deletion for timeseries

Posted by GitBox <gi...@apache.org>.
jixuan1989 commented on a change in pull request #1400:
URL: https://github.com/apache/incubator-iotdb/pull/1400#discussion_r447064513



##########
File path: tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/ChunkMetadata.java
##########
@@ -171,12 +174,22 @@ public void setVersion(long version) {
     this.version = version;
   }
 
-  public long getDeletedAt() {
-    return deletedAt;
+  public List<Pair<Long, Long>> getDeleteRangeList() {
+    if (deleteRangeList == null) {

Review comment:
       em... if so, you still generate many ArrayList instances ...
   If you do not want to check whether the list is null, use `Collections.emptyList` instead
   




----------------------------------------------------------------
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



[GitHub] [incubator-iotdb] wshao08 commented on a change in pull request #1400: [IOTDB-627]Support range deletion for timeseries

Posted by GitBox <gi...@apache.org>.
wshao08 commented on a change in pull request #1400:
URL: https://github.com/apache/incubator-iotdb/pull/1400#discussion_r451306475



##########
File path: server/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java
##########
@@ -519,7 +518,7 @@ public boolean hasNextTimeValuePair() {
 
       while (cur < size) {
         long time = getTime(cur);
-        if (time < getTimeOffset() || (cur + 1 < size() && (time == getTime(cur + 1)))) {
+        if (isPointDeleted(time) || (cur + 1 < size() && (time == getTime(cur + 1)))) {

Review comment:
       Fixed




----------------------------------------------------------------
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



[GitHub] [incubator-iotdb] wshao08 commented on a change in pull request #1400: [IOTDB-627]Support range deletion for timeseries

Posted by GitBox <gi...@apache.org>.
wshao08 commented on a change in pull request #1400:
URL: https://github.com/apache/incubator-iotdb/pull/1400#discussion_r451430687



##########
File path: tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/ChunkMetadata.java
##########
@@ -198,15 +222,14 @@ public boolean equals(Object o) {
     ChunkMetadata that = (ChunkMetadata) o;
     return offsetOfChunkHeader == that.offsetOfChunkHeader &&
         version == that.version &&
-        deletedAt == that.deletedAt &&

Review comment:
       Added




----------------------------------------------------------------
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



[GitHub] [incubator-iotdb] wshao08 commented on a change in pull request #1400: [IOTDB-627]Support range deletion for timeseries

Posted by GitBox <gi...@apache.org>.
wshao08 commented on a change in pull request #1400:
URL: https://github.com/apache/incubator-iotdb/pull/1400#discussion_r450587930



##########
File path: server/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java
##########
@@ -531,6 +530,17 @@ public boolean hasNextTimeValuePair() {
       return hasCachedPair;
     }
 
+    private boolean isPointDeleted(long timestamp) {
+      if (deletionList != null) {
+        for (Pair<Long, Long> del : deletionList) {
+          if (del.left <= timestamp && timestamp <= del.right) {
+            return true;
+          }
+        }

Review comment:
       Fixed

##########
File path: server/src/test/java/org/apache/iotdb/db/engine/merge/MergeTaskTest.java
##########
@@ -218,7 +218,7 @@ public void testPartialMerge3() throws Exception {
   public void mergeWithDeletionTest() throws Exception {
     try {
       seqResources.get(0).getModFile().write(new Deletion(new Path(deviceIds[0],
-          measurementSchemas[0].getMeasurementId()), 10000, 49));
+          measurementSchemas[0].getMeasurementId()), 10000, 0,49));

Review comment:
       Fixed




----------------------------------------------------------------
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



[GitHub] [incubator-iotdb] samperson1997 commented on a change in pull request #1400: [IOTDB-627]Support range deletion for timeseries

Posted by GitBox <gi...@apache.org>.
samperson1997 commented on a change in pull request #1400:
URL: https://github.com/apache/incubator-iotdb/pull/1400#discussion_r451343367



##########
File path: server/src/test/java/org/apache/iotdb/db/integration/IoTDBDeletionIT.java
##########
@@ -164,6 +164,43 @@ public void testDelAfterFlush() throws SQLException {
     }
   }
 
+  @Test
+  public void testRangeDelete() throws SQLException {
+    prepareData();
+    try (Connection connection = DriverManager
+        .getConnection(Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root",
+            "root");
+        Statement statement = connection.createStatement()) {
+
+      statement.execute("DELETE FROM root.vehicle.d0.s0  WHERE time <= 300");

Review comment:
       ```suggestion
         statement.execute("DELETE FROM root.vehicle.d0.s0 WHERE time <= 300");
   ```

##########
File path: server/src/main/java/org/apache/iotdb/db/utils/QueryUtils.java
##########
@@ -19,6 +19,9 @@
 
 package org.apache.iotdb.db.utils;
 
+import java.util.Comparator;
+import java.util.LinkedList;
+import java.util.stream.Collectors;

Review comment:
       Remove unused import




----------------------------------------------------------------
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



[GitHub] [incubator-iotdb] qiaojialin commented on a change in pull request #1400: [IOTDB-627]Support range deletion for timeseries

Posted by GitBox <gi...@apache.org>.
qiaojialin commented on a change in pull request #1400:
URL: https://github.com/apache/incubator-iotdb/pull/1400#discussion_r448842280



##########
File path: server/src/main/java/org/apache/iotdb/db/engine/modification/io/LocalTextModificationAccessor.java
##########
@@ -124,28 +124,42 @@ private static Modification decodeModification(String src) throws IOException {
   private static String encodeDeletion(Deletion del) {
     return del.getType().toString() + SEPARATOR + del.getPathString()
         + SEPARATOR + del.getVersionNum() + SEPARATOR
-        + del.getTimestamp();
+        + del.getStartTime() + SEPARATOR + del.getEndTime();
   }
 
   private static Deletion decodeDeletion(String[] fields) throws IOException {
-    if (fields.length != 4) {
+    if (fields.length != 5 && fields.length != 4) {
       throw new IOException("Incorrect deletion fields number: " + fields.length);
     }
 
     String path = fields[1];
     long versionNum;
-    long timestamp;
+    long startTimestamp = Long.MIN_VALUE;
+    long endTimestamp;
     try {
       versionNum = Long.parseLong(fields[2]);
     } catch (NumberFormatException e) {
       throw new IOException("Invalid version number: " + fields[2]);
     }
-    try {
-      timestamp = Long.parseLong(fields[3]);
-    } catch (NumberFormatException e) {
-      throw new IOException("Invalid timestamp: " + fields[3]);
+    if (fields.length == 4) {
+      try {
+        endTimestamp = Long.parseLong(fields[3]);
+      } catch (NumberFormatException e) {
+        throw new IOException("Invalid timestamp: " + fields[3]);
+      }
+    } else {
+      try {
+        startTimestamp = Long.parseLong(fields[3]);
+      } catch (NumberFormatException e) {
+        throw new IOException("Invalid timestamp: " + fields[3]);
+      }
+      try {
+        endTimestamp = Long.parseLong(fields[4]);
+      } catch (NumberFormatException e) {
+        throw new IOException("Invalid timestamp: " + fields[4]);
+      }

Review comment:
       You can combine the two parsing and put the whole deletion line into IOException

##########
File path: server/src/main/java/org/apache/iotdb/db/qp/physical/crud/DeletePlan.java
##########
@@ -45,7 +46,22 @@ public DeletePlan() {
    */
   public DeletePlan(long deleteTime, Path path) {
     super(false, Operator.OperatorType.DELETE);
-    this.deleteTime = deleteTime;
+    this.deleteStartTime = Long.MIN_VALUE;
+    this.deleteEndTime = deleteTime;
+    this.paths.add(path);
+  }

Review comment:
       It's better to remove this constructor, it may misleading others that this means delete time = deleteTime.

##########
File path: docs/UserGuide/Operation Manual/DML Data Manipulation Language.md
##########
@@ -783,6 +783,12 @@ The wf02 plant's wt02 device has many segments of errors in its power supply sta
 delete from root.ln.wf02.wt02.status where time<=2017-11-01T16:26:00;
 ```
 
+In case we hope to merely delete the data before 2017-11-01 16:26:00 in the year of 2017, The SQL statement is:
+```
+delete from root.ln.wf02.wt02.status where time>=2017-01-01T00:00:00 and time<=2017-11-01T16:26:00;

Review comment:
       give more examples
   put the sqls in pr here

##########
File path: server/src/main/java/org/apache/iotdb/db/utils/QueryUtils.java
##########
@@ -45,46 +49,52 @@ private QueryUtils() {
    */
   public static void modifyChunkMetaData(List<ChunkMetadata> chunkMetaData,
                                          List<Modification> modifications) {
-    int modIndex = 0;
+    List<Modification> sortedModifications = sortModifications(modifications);
 
     for (int metaIndex = 0; metaIndex < chunkMetaData.size(); metaIndex++) {
       ChunkMetadata metaData = chunkMetaData.get(metaIndex);
-      for (int j = modIndex; j < modifications.size(); j++) {
-        // iterate each modification to find the max deletion time
-        Modification modification = modifications.get(j);
+      for (Modification modification : sortedModifications) {
         if (modification.getVersionNum() > metaData.getVersion()) {
-          // this modification is after the Chunk, try modifying the chunk
-          // if this modification succeeds, update modIndex so in the next loop the previous
-          // modifications will not be examined
-          modIndex = doModifyChunkMetaData(modification, metaData)? j : modIndex;
-        } else {
-          // skip old modifications for next metadata
-          modIndex++;
+          doModifyChunkMetaData(modification, metaData);
         }
       }
     }
     // remove chunks that are completely deleted
     chunkMetaData.removeIf(metaData -> {
-      if (metaData.getDeletedAt() >= metaData.getEndTime()) {
-        return true;
-      } else {
-        if (metaData.getDeletedAt() >= metaData.getStartTime()) {
-          metaData.setModified(true);
+      long lower = metaData.getStartTime();
+      long upper = metaData.getEndTime();
+      if (metaData.getDeleteIntervalList() != null) {
+        for (Pair<Long, Long> range : metaData.getDeleteIntervalList()) {
+          if (upper < range.left) {
+            break;
+          }
+          if (range.left <= lower && lower <= range.right) {
+            metaData.setModified(true);
+            if (upper <= range.right) {
+              return true;
+            }
+            lower = range.right;
+          } else if (lower < range.left) {
+            metaData.setModified(true);
+            break;
+          }
         }
-        return false;
       }
+      return false;
     });
   }
 
-  private static boolean doModifyChunkMetaData(Modification modification, ChunkMetadata metaData) {
+  private static LinkedList<Modification> sortModifications(List<Modification> modifications) {

Review comment:
       This should be sortAndMerge
   
   Suppose we have [1,10], [5,12], [15,20], [16,21]
   
   It's better to merge them to [1,12] and [15,21]

##########
File path: server/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java
##########
@@ -519,7 +518,7 @@ public boolean hasNextTimeValuePair() {
 
       while (cur < size) {
         long time = getTime(cur);
-        if (time < getTimeOffset() || (cur + 1 < size() && (time == getTime(cur + 1)))) {
+        if (isPointDeleted(time) || (cur + 1 < size() && (time == getTime(cur + 1)))) {

Review comment:
       you record the deletion pair in the TVList, when iterating the TVList, it's better to sortAndMerge all deletions.
   When calling the Ite, the TVList should be sorted already, you can store the deletions inside the Ite and record and index of current checked deletion. Then you could use a more efficient way to check which point is deleted.

##########
File path: tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/chunk/ChunkReaderByTimestamp.java
##########
@@ -35,7 +35,10 @@ public ChunkReaderByTimestamp(Chunk chunk) throws IOException {
   public boolean pageSatisfied(PageHeader pageHeader) {
     long maxTimestamp = pageHeader.getEndTime();
     // if maxTimestamp > currentTimestamp, this page should NOT be skipped
-    return maxTimestamp >= currentTimestamp && maxTimestamp > deletedAt;
+    // return maxTimestamp >= currentTimestamp && super.pageSatisfied(pageHeader);

Review comment:
       remove this

##########
File path: server/src/main/java/org/apache/iotdb/db/qp/strategy/LogicalGenerator.java
##########
@@ -1550,18 +1552,52 @@ long parseTimeFormat(String timestampStr) throws SQLParserException {
    *
    * @param operator delete logical plan
    */
-  private long parseDeleteTimeFilter(DeleteDataOperator operator) {
+  private Pair<Long, Long> parseDeleteTimeRange(DeleteDataOperator operator) {
     FilterOperator filterOperator = operator.getFilterOperator();
-    if (filterOperator.getTokenIntType() != SQLConstant.LESSTHAN
-        && filterOperator.getTokenIntType() != SQLConstant.LESSTHANOREQUALTO) {
+    if (!filterOperator.isLeaf() && filterOperator.getTokenIntType() != SQLConstant.KW_AND) {
       throw new SQLParserException(
-          "For delete command, where clause must be like : time < XXX or time <= XXX");
+          "For delete command, where clause can only contain atomic expressions like : "
+              + "time > XXX, time <= XXX, or And with two atomic expressions");
     }
+
+    if (filterOperator.isLeaf()) {
+      return calcOperatorRange(filterOperator);
+    }
+
+    List<FilterOperator> children = filterOperator.getChildren();
+    FilterOperator lOperator = children.get(0);

Review comment:
       add more check for BinaryOperator, it must be a AndOperator and the  lower bound must <= the upper bound

##########
File path: server/src/main/java/org/apache/iotdb/db/engine/modification/Deletion.java
##########
@@ -28,21 +28,36 @@
 public class Deletion extends Modification {
 
   /**
-   * data whose timestamp <= this field are to be deleted.
+   * data within the interval [startTime, endTime] are to be deleted.
    */
-  private long timestamp;
+  private long startTime;
+  private long endTime;
 
-  public Deletion(Path path, long versionNum, long timestamp) {
+  public Deletion(Path path, long versionNum, long endTime) {
     super(Type.DELETION, path, versionNum);
-    this.timestamp = timestamp;
+    this.endTime = endTime;

Review comment:
       this.startTime = Long.MIN_VALUE;

##########
File path: server/src/main/java/org/apache/iotdb/db/engine/memtable/AbstractMemTable.java
##########
@@ -203,38 +204,40 @@ public ReadOnlyMemChunk query(String deviceId, String measurement, TSDataType da
     if (!checkPath(deviceId, measurement)) {
       return null;
     }
-    long undeletedTime = findUndeletedTime(deviceId, measurement, timeLowerBound);
+    List<Pair<Long, Long>> deletionList = constructDeletionList(deviceId, measurement, timeLowerBound);
     IWritableMemChunk memChunk = memTableMap.get(deviceId).get(measurement);
     TVList chunkCopy = memChunk.getTVList().clone();
 
-    chunkCopy.setTimeOffset(undeletedTime);
+    chunkCopy.setDeletionList(deletionList);
     return new ReadOnlyMemChunk(measurement, dataType, encoding, chunkCopy, props, getVersion());
   }
 
 
-  private long findUndeletedTime(String deviceId, String measurement, long timeLowerBound) {
-    long undeletedTime = Long.MIN_VALUE;
+  private List<Pair<Long, Long>> constructDeletionList(String deviceId, String measurement,
+      long timeLowerBound) {
+    List<Pair<Long, Long>> deletionList = new ArrayList<>();
     for (Modification modification : modifications) {
       if (modification instanceof Deletion) {
         Deletion deletion = (Deletion) modification;
         if (deletion.getDevice().equals(deviceId) && deletion.getMeasurement().equals(measurement)
-            && deletion.getTimestamp() > undeletedTime) {
-          undeletedTime = deletion.getTimestamp();
+            && deletion.getEndTime() > timeLowerBound) {
+          long lowerBound = Math.max(deletion.getStartTime(), timeLowerBound);
+          deletionList.add(new Pair<>(lowerBound, deletion.getEndTime()));

Review comment:
       Don't we need to add a [-infinity, lowerBound] into deletionList?

##########
File path: server/src/main/java/org/apache/iotdb/db/qp/physical/crud/DeletePlan.java
##########
@@ -57,16 +73,39 @@ public DeletePlan(long deleteTime, Path path) {
    */
   public DeletePlan(long deleteTime, List<Path> paths) {
     super(false, Operator.OperatorType.DELETE);
-    this.deleteTime = deleteTime;
+    this.deleteStartTime = Long.MIN_VALUE;
+    this.deleteEndTime = deleteTime;
+    this.paths = paths;
+  }

Review comment:
       also delete this 

##########
File path: tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/chunk/ChunkReader.java
##########
@@ -131,10 +132,25 @@ private void skipBytesInStreamByLength(long length) {
   }
 
   public boolean pageSatisfied(PageHeader pageHeader) {
-    if (pageHeader.getEndTime() <= deletedAt) {
-      return false;
-    } else if (pageHeader.getStartTime() <= deletedAt) {
-      pageHeader.setModified(true);
+    long lower = pageHeader.getStartTime();
+    long upper = pageHeader.getEndTime();
+    // deleteIntervalList is sorted in terms of startTime
+    if (deleteIntervalList != null) {
+      for (Pair<Long, Long> range : deleteIntervalList) {
+        if (upper < range.left) {
+          break;
+        }
+        if (range.left <= lower && lower <= range.right) {
+          pageHeader.setModified(true);
+          if (upper <= range.right) {
+            return true;

Review comment:
       if the page is totally deleted, this page is not satisfied, return false.

##########
File path: tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/page/PageReader.java
##########
@@ -159,12 +161,23 @@ public void setFilter(Filter filter) {
     this.filter = filter;
   }
 
-  public void setDeletedAt(long deletedAt) {
-    this.deletedAt = deletedAt;
+  public void setDeleteIntervalList(List<Pair<Long, Long>> list) {
+    this.deleteIntervalList = list;
   }
 
   @Override
   public boolean isModified() {
     return pageHeader.isModified();
   }
+
+  private boolean isDeleted(long timestamp) {

Review comment:
       also, the Deletions should be sortAndMerge, then optimize this check




----------------------------------------------------------------
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



[GitHub] [incubator-iotdb] jt2594838 commented on a change in pull request #1400: [IOTDB-627]Support range deletion for timeseries

Posted by GitBox <gi...@apache.org>.
jt2594838 commented on a change in pull request #1400:
URL: https://github.com/apache/incubator-iotdb/pull/1400#discussion_r448861552



##########
File path: server/src/test/java/org/apache/iotdb/db/engine/merge/MergeTaskTest.java
##########
@@ -218,7 +218,7 @@ public void testPartialMerge3() throws Exception {
   public void mergeWithDeletionTest() throws Exception {
     try {
       seqResources.get(0).getModFile().write(new Deletion(new Path(deviceIds[0],
-          measurementSchemas[0].getMeasurementId()), 10000, 49));
+          measurementSchemas[0].getMeasurementId()), 10000, 0,49));

Review comment:
       Please mind the format.

##########
File path: server/src/main/java/org/apache/iotdb/db/utils/QueryUtils.java
##########
@@ -45,46 +49,52 @@ private QueryUtils() {
    */
   public static void modifyChunkMetaData(List<ChunkMetadata> chunkMetaData,
                                          List<Modification> modifications) {
-    int modIndex = 0;
+    List<Modification> sortedModifications = sortModifications(modifications);
 
     for (int metaIndex = 0; metaIndex < chunkMetaData.size(); metaIndex++) {
       ChunkMetadata metaData = chunkMetaData.get(metaIndex);
-      for (int j = modIndex; j < modifications.size(); j++) {
-        // iterate each modification to find the max deletion time
-        Modification modification = modifications.get(j);
+      for (Modification modification : sortedModifications) {
         if (modification.getVersionNum() > metaData.getVersion()) {
-          // this modification is after the Chunk, try modifying the chunk
-          // if this modification succeeds, update modIndex so in the next loop the previous
-          // modifications will not be examined
-          modIndex = doModifyChunkMetaData(modification, metaData)? j : modIndex;
-        } else {
-          // skip old modifications for next metadata
-          modIndex++;
+          doModifyChunkMetaData(modification, metaData);
         }
       }
     }
     // remove chunks that are completely deleted
     chunkMetaData.removeIf(metaData -> {
-      if (metaData.getDeletedAt() >= metaData.getEndTime()) {
-        return true;
-      } else {
-        if (metaData.getDeletedAt() >= metaData.getStartTime()) {
-          metaData.setModified(true);
+      long lower = metaData.getStartTime();
+      long upper = metaData.getEndTime();
+      if (metaData.getDeleteIntervalList() != null) {
+        for (Pair<Long, Long> range : metaData.getDeleteIntervalList()) {
+          if (upper < range.left) {
+            break;
+          }
+          if (range.left <= lower && lower <= range.right) {
+            metaData.setModified(true);
+            if (upper <= range.right) {
+              return true;
+            }
+            lower = range.right;
+          } else if (lower < range.left) {
+            metaData.setModified(true);
+            break;
+          }
         }
-        return false;
       }
+      return false;
     });
   }
 
-  private static boolean doModifyChunkMetaData(Modification modification, ChunkMetadata metaData) {
+  private static LinkedList<Modification> sortModifications(List<Modification> modifications) {
+    return modifications.stream().filter(x -> x instanceof Deletion)
+        .sorted(Comparator.comparingLong(mods -> ((Deletion) mods).getStartTime()))
+        .collect(Collectors.toCollection(LinkedList::new));
+  }

Review comment:
       I wonder why LinkedList is preferred.

##########
File path: session/src/main/java/org/apache/iotdb/session/Session.java
##########
@@ -747,14 +747,37 @@ public void deleteData(String path, long time)
    * delete data <= time in multiple timeseries
    *
    * @param paths data in which time series to delete
-   * @param time  data with time stamp less than or equal to time will be deleted
+   * @param endTime data with time stamp less than or equal to time will be deleted
    */
-  public void deleteData(List<String> paths, long time)
+  public void deleteData(List<String> paths, long endTime)
       throws IoTDBConnectionException, StatementExecutionException {
     TSDeleteDataReq request = new TSDeleteDataReq();
     request.setSessionId(sessionId);
     request.setPaths(paths);
-    request.setTimestamp(time);
+    request.setStartTime(Long.MIN_VALUE);
+    request.setEndTime(endTime);
+

Review comment:
       It may simply be changed to a call of `deleteData(paths, Long.MIN_VALUE, endtime)`.

##########
File path: server/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java
##########
@@ -531,6 +530,17 @@ public boolean hasNextTimeValuePair() {
       return hasCachedPair;
     }
 
+    private boolean isPointDeleted(long timestamp) {
+      if (deletionList != null) {
+        for (Pair<Long, Long> del : deletionList) {
+          if (del.left <= timestamp && timestamp <= del.right) {
+            return true;
+          }
+        }

Review comment:
       I think it is not necessary to start from the first of the deletions at each call of `isPointDeleted()`, as this method is called with ordered timestamps, if one timestamp is after the range of one deletion, all remaining timestamps will either.




----------------------------------------------------------------
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



[GitHub] [incubator-iotdb] wshao08 commented on a change in pull request #1400: [IOTDB-627]Support range deletion for timeseries

Posted by GitBox <gi...@apache.org>.
wshao08 commented on a change in pull request #1400:
URL: https://github.com/apache/incubator-iotdb/pull/1400#discussion_r451430617



##########
File path: tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/ChunkMetadata.java
##########
@@ -91,8 +94,8 @@ public ChunkMetadata(String measurementUid, TSDataType tsDataType, long fileOffs
 
   @Override
   public String toString() {
-    return String.format("measurementId: %s, datatype: %s, version: %d, deletedAt: %d, "
-        + "Statistics: %s", measurementUid, tsDataType, version, deletedAt, statistics);
+    return String.format("measurementId: %s, datatype: %s, version: %d, "

Review comment:
       Yes, added in toString and hash




----------------------------------------------------------------
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



[GitHub] [incubator-iotdb] samperson1997 commented on a change in pull request #1400: [IOTDB-627]Support range deletion for timeseries

Posted by GitBox <gi...@apache.org>.
samperson1997 commented on a change in pull request #1400:
URL: https://github.com/apache/incubator-iotdb/pull/1400#discussion_r451314328



##########
File path: tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/ChunkMetadata.java
##########
@@ -91,8 +94,8 @@ public ChunkMetadata(String measurementUid, TSDataType tsDataType, long fileOffs
 
   @Override
   public String toString() {
-    return String.format("measurementId: %s, datatype: %s, version: %d, deletedAt: %d, "
-        + "Statistics: %s", measurementUid, tsDataType, version, deletedAt, statistics);
+    return String.format("measurementId: %s, datatype: %s, version: %d, "

Review comment:
       I think it's better to add the `deleteInterval` in string and hash code. What do you think?

##########
File path: server/src/test/java/org/apache/iotdb/db/qp/plan/LogicalPlanSmallTest.java
##########
@@ -237,5 +239,100 @@ public void testChineseCharacter() {
     Assert.assertEquals(paths, ((QueryOperator) operator).getSelectedPaths());
   }
 
+  @Test
+  public void testRangeDelete() {
+    String sql1 = "delete from root.d1.s1 where time>=1 and time < 3";
+    Operator op = parseDriver.parse(sql1, IoTDBDescriptor.getInstance().getConfig().getZoneID());
+    Assert.assertEquals(DeleteDataOperator.class, op.getClass());
+    ArrayList<Path> paths = new ArrayList<>();
+    paths.add(new Path("root.d1.s1"));
+    Assert.assertEquals(paths, ((DeleteDataOperator) op).getSelectedPaths());
+    Assert.assertEquals(1, ((DeleteDataOperator) op).getStartTime());
+    Assert.assertEquals(2, ((DeleteDataOperator) op).getEndTime());
+
+    String sql2 = "delete from root.d1.s1 where time>=1";
+    op = parseDriver.parse(sql2, IoTDBDescriptor.getInstance().getConfig().getZoneID());
+    Assert.assertEquals(paths, ((DeleteDataOperator) op).getSelectedPaths());
+    Assert.assertEquals(1, ((DeleteDataOperator) op).getStartTime());
+    Assert.assertEquals(Long.MAX_VALUE, ((DeleteDataOperator) op).getEndTime());
+
+    String sql3 = "delete from root.d1.s1 where time>1";
+    op = parseDriver.parse(sql3, IoTDBDescriptor.getInstance().getConfig().getZoneID());
+    Assert.assertEquals(paths, ((DeleteDataOperator) op).getSelectedPaths());
+    Assert.assertEquals(2, ((DeleteDataOperator) op).getStartTime());
+    Assert.assertEquals(Long.MAX_VALUE, ((DeleteDataOperator) op).getEndTime());
+
+    String sql4 = "delete from root.d1.s1 where time <= 1";
+    op = parseDriver.parse(sql4, IoTDBDescriptor.getInstance().getConfig().getZoneID());
+    Assert.assertEquals(paths, ((DeleteDataOperator) op).getSelectedPaths());
+    Assert.assertEquals(Long.MIN_VALUE, ((DeleteDataOperator) op).getStartTime());
+    Assert.assertEquals(1, ((DeleteDataOperator) op).getEndTime());
+
+    String sql5 = "delete from root.d1.s1 where time<1";
+    op = parseDriver.parse(sql5, IoTDBDescriptor.getInstance().getConfig().getZoneID());
+    Assert.assertEquals(paths, ((DeleteDataOperator) op).getSelectedPaths());
+    Assert.assertEquals(Long.MIN_VALUE, ((DeleteDataOperator) op).getStartTime());
+    Assert.assertEquals(0, ((DeleteDataOperator) op).getEndTime());
+
+    String sql6 = "delete from root.d1.s1 where time = 3";
+    op = parseDriver.parse(sql6, IoTDBDescriptor.getInstance().getConfig().getZoneID());
+    Assert.assertEquals(paths, ((DeleteDataOperator) op).getSelectedPaths());
+    Assert.assertEquals(3, ((DeleteDataOperator) op).getStartTime());
+    Assert.assertEquals(3, ((DeleteDataOperator) op).getEndTime());
 
+    String sql7 = "delete from root.d1.s1 where time = 1 and time < -1";
+    String errorMsg = null;
+    try {
+      op = parseDriver.parse(sql7, IoTDBDescriptor.getInstance().getConfig().getZoneID());
+    } catch (RuntimeException e) {
+      errorMsg = e.getMessage();
+    }
+    Assert.assertEquals(errorMsg, "Invalid delete range: [1, -2]");
+
+    String sql8 = "delete from root.d1.s1 where time > 5 and time <= 0";
+    try {
+      op = parseDriver.parse(sql8, IoTDBDescriptor.getInstance().getConfig().getZoneID());
+    } catch (RuntimeException e) {
+      errorMsg = e.getMessage();
+    }
+    Assert.assertEquals(errorMsg, "Invalid delete range: [6, 0]");
+  }
+
+  @Test
+  public void testErrorDeleteRange() {
+    String sql = "delete from root.d1.s1 where time>=1 and time < 3 or time >1";
+    String errorMsg = null;
+    try {
+      parseDriver.parse(sql, IoTDBDescriptor.getInstance().getConfig().getZoneID());
+    } catch (SQLParserException e) {
+      errorMsg = e.getMessage();
+    }
+    Assert.assertEquals(
+        "For delete command, where clause can only contain atomic expressions like : "
+            + "time > XXX, time <= XXX, or And with two atomic expressions",
+        errorMsg);
+
+    sql = "delete from root.d1.s1 where time>=1 or time < 3";
+    errorMsg = null;
+    try {
+      parseDriver.parse(sql, IoTDBDescriptor.getInstance().getConfig().getZoneID());
+    } catch (SQLParserException e) {
+      errorMsg = e.getMessage();
+    }
+    Assert.assertEquals(
+        "For delete command, where clause can only contain atomic expressions like : "
+            + "time > XXX, time <= XXX, or And with two atomic expressions",
+        errorMsg);
+
+    /*
+    sql = "delete from root.d1.s1 where time<=1 and time > 3";
+    errorMsg = null;
+    try {
+      parseDriver.parse(sql, IoTDBDescriptor.getInstance().getConfig().getZoneID());
+    } catch (SQLParserException e) {
+      errorMsg = e.getMessage();
+    }
+    Assert.assertEquals("Unreachable deleted time interval", errorMsg);
+     */

Review comment:
       Why is this commented?

##########
File path: server/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessor.java
##########
@@ -1350,14 +1353,13 @@ public void delete(String deviceId, String measurementId, long timestamp) throws
         return;
       }
 
-      // time partition to divide storage group
-      long timePartitionId = StorageEngine.getTimePartition(timestamp);
       // write log to impacted working TsFileProcessors
-      logDeletion(timestamp, deviceId, measurementId, timePartitionId);
+      logDeletion(startTime, endTime, deviceId, measurementId);
+      // delete Last cache record if necessary
+      tryToDeleteLastCache(deviceId, measurementId, startTime, endTime);
 
       Path fullPath = new Path(deviceId, measurementId);
-      Deletion deletion = new Deletion(fullPath,
-          getVersionControllerByTimePartitionId(timePartitionId).nextVersion(), timestamp);
+      Deletion deletion = new Deletion(fullPath, 1, startTime, endTime);

Review comment:
       Why 1?

##########
File path: tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/ChunkMetadata.java
##########
@@ -198,15 +222,14 @@ public boolean equals(Object o) {
     ChunkMetadata that = (ChunkMetadata) o;
     return offsetOfChunkHeader == that.offsetOfChunkHeader &&
         version == that.version &&
-        deletedAt == that.deletedAt &&

Review comment:
       I think `deleteInterval` should be compared as well

##########
File path: tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/page/PageReader.java
##########
@@ -159,12 +162,25 @@ public void setFilter(Filter filter) {
     this.filter = filter;
   }
 
-  public void setDeletedAt(long deletedAt) {
-    this.deletedAt = deletedAt;
+  public void setDeleteIntervalList(List<TimeRange> list) {
+    this.deleteIntervalList = list;
   }
 
   @Override
   public boolean isModified() {
     return pageHeader.isModified();
   }
+
+  private boolean isDeleted(long timestamp, int deleteCursor) {

Review comment:
       Could you please recheck the logic? `deleteCursor` will not be changed after calling this method, since you didn't return it... When you call this method, `deleteCursor` keeps value 0.

##########
File path: tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/TimeRange.java
##########
@@ -103,6 +103,8 @@ public boolean contains(long min, long max) {
     return this.min <= min && this.max >= max;
   }
 
+  public boolean contains(long time) {return this.min <= time && time <= this.max;}

Review comment:
       Format this line to 3 lines : D




----------------------------------------------------------------
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



[GitHub] [incubator-iotdb] wshao08 commented on a change in pull request #1400: [IOTDB-627]Support range deletion for timeseries

Posted by GitBox <gi...@apache.org>.
wshao08 commented on a change in pull request #1400:
URL: https://github.com/apache/incubator-iotdb/pull/1400#discussion_r447356953



##########
File path: server/src/main/java/org/apache/iotdb/db/utils/QueryUtils.java
##########
@@ -45,46 +49,52 @@ private QueryUtils() {
    */
   public static void modifyChunkMetaData(List<ChunkMetadata> chunkMetaData,
                                          List<Modification> modifications) {
-    int modIndex = 0;
+    List<Modification> sortedModifications = sortModifications(modifications);
 
     for (int metaIndex = 0; metaIndex < chunkMetaData.size(); metaIndex++) {
       ChunkMetadata metaData = chunkMetaData.get(metaIndex);
-      for (int j = modIndex; j < modifications.size(); j++) {
-        // iterate each modification to find the max deletion time
-        Modification modification = modifications.get(j);
+      for (Modification modification : sortedModifications) {
         if (modification.getVersionNum() > metaData.getVersion()) {
-          // this modification is after the Chunk, try modifying the chunk
-          // if this modification succeeds, update modIndex so in the next loop the previous
-          // modifications will not be examined
-          modIndex = doModifyChunkMetaData(modification, metaData)? j : modIndex;
-        } else {
-          // skip old modifications for next metadata
-          modIndex++;
+          doModifyChunkMetaData(modification, metaData);
         }
       }
     }
     // remove chunks that are completely deleted
     chunkMetaData.removeIf(metaData -> {
-      if (metaData.getDeletedAt() >= metaData.getEndTime()) {
-        return true;
-      } else {
-        if (metaData.getDeletedAt() >= metaData.getStartTime()) {
+      long lower = metaData.getStartTime();
+      long upper = metaData.getEndTime();
+      for (Pair<Long, Long> range : metaData.getDeleteRangeList()) {
+        if (upper < range.left) {
+          break;
+        }
+        if (range.left <= lower && lower <= range.right) {
+          metaData.setModified(true);
+          if (upper <= range.right) {
+            return true;
+          }
+          lower = range.right;
+        } else if (lower < range.left) {
           metaData.setModified(true);
+          break;
         }
-        return false;
       }
+      return false;
     });
   }
 
-  private static boolean doModifyChunkMetaData(Modification modification, ChunkMetadata metaData) {
+  private static LinkedList<Modification> sortModifications(List<Modification> modifications) {
+    return modifications.stream()
+        .sorted(

Review comment:
       Added




----------------------------------------------------------------
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



[GitHub] [incubator-iotdb] wshao08 commented on a change in pull request #1400: [IOTDB-627]Support range deletion for timeseries

Posted by GitBox <gi...@apache.org>.
wshao08 commented on a change in pull request #1400:
URL: https://github.com/apache/incubator-iotdb/pull/1400#discussion_r447356892



##########
File path: server/src/main/java/org/apache/iotdb/db/engine/memtable/AbstractMemTable.java
##########
@@ -203,38 +204,38 @@ public ReadOnlyMemChunk query(String deviceId, String measurement, TSDataType da
     if (!checkPath(deviceId, measurement)) {
       return null;
     }
-    long undeletedTime = findUndeletedTime(deviceId, measurement, timeLowerBound);
+    List<Pair<Long, Long>> deletionList = findUndeletedTime(deviceId, measurement, timeLowerBound);
     IWritableMemChunk memChunk = memTableMap.get(deviceId).get(measurement);
     TVList chunkCopy = memChunk.getTVList().clone();
 
-    chunkCopy.setTimeOffset(undeletedTime);
+    chunkCopy.setDeletionList(deletionList);
     return new ReadOnlyMemChunk(measurement, dataType, encoding, chunkCopy, props, getVersion());
   }
 
 
-  private long findUndeletedTime(String deviceId, String measurement, long timeLowerBound) {
-    long undeletedTime = Long.MIN_VALUE;
+  private List<Pair<Long, Long>> findUndeletedTime(String deviceId, String measurement,

Review comment:
       Fixed

##########
File path: server/src/main/java/org/apache/iotdb/db/engine/modification/io/LocalTextModificationAccessor.java
##########
@@ -124,28 +124,42 @@ private static Modification decodeModification(String src) throws IOException {
   private static String encodeDeletion(Deletion del) {
     return del.getType().toString() + SEPARATOR + del.getPathString()
         + SEPARATOR + del.getVersionNum() + SEPARATOR
-        + del.getTimestamp();
+        + del.getStartTime() + SEPARATOR + del.getEndTime();
   }
 
   private static Deletion decodeDeletion(String[] fields) throws IOException {
-    if (fields.length != 4) {
+    if (fields.length != 4 && fields.length != 5) {

Review comment:
       Fixed




----------------------------------------------------------------
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



[GitHub] [incubator-iotdb] wshao08 commented on a change in pull request #1400: [IOTDB-627]Support range deletion for timeseries

Posted by GitBox <gi...@apache.org>.
wshao08 commented on a change in pull request #1400:
URL: https://github.com/apache/incubator-iotdb/pull/1400#discussion_r451304936



##########
File path: server/src/main/java/org/apache/iotdb/db/qp/physical/crud/DeletePlan.java
##########
@@ -57,16 +73,39 @@ public DeletePlan(long deleteTime, Path path) {
    */
   public DeletePlan(long deleteTime, List<Path> paths) {
     super(false, Operator.OperatorType.DELETE);
-    this.deleteTime = deleteTime;
+    this.deleteStartTime = Long.MIN_VALUE;
+    this.deleteEndTime = deleteTime;
+    this.paths = paths;
+  }

Review comment:
       Done




----------------------------------------------------------------
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