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 2022/09/06 02:48:44 UTC

[GitHub] [iotdb] HeimingZ commented on a diff in pull request #7217: [TVList]restructure TVList, add TimSort interface

HeimingZ commented on code in PR #7217:
URL: https://github.com/apache/iotdb/pull/7217#discussion_r963202393


##########
server/src/main/java/org/apache/iotdb/db/utils/datastructure/TimSort.java:
##########
@@ -0,0 +1,184 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iotdb.db.utils.datastructure;
+
+public interface TimSort {
+  int SMALL_ARRAY_LENGTH = 32;

Review Comment:
   It's better to add some comments for this sort interface.



##########
server/src/main/java/org/apache/iotdb/db/utils/datastructure/IntTVList.java:
##########
@@ -29,28 +29,34 @@
 import org.apache.iotdb.tsfile.utils.BitMap;
 import org.apache.iotdb.tsfile.utils.TsPrimitiveType;
 
-import java.io.DataInputStream;
-import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 
 import static org.apache.iotdb.db.rescon.PrimitiveArrayManager.ARRAY_SIZE;
 
-public class IntTVList extends TVList {
-
-  // list of primitive array, add 1 when expanded -> int primitive array
-  // index relation: arrayIndex -> elementIndex
-  private List<int[]> values;
-
-  private int[][] sortedValues;
-
-  private int pivotValue;
+public abstract class IntTVList extends TVList {
+  protected List<int[]> values;

Review Comment:
   Comments are lost
   ```
   // list of primitive array, add 1 when expanded -> int primitive array
   // index relation: arrayIndex -> elementIndex
   ```



##########
server/src/main/java/org/apache/iotdb/db/utils/datastructure/TimIntTVList.java:
##########
@@ -0,0 +1,151 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iotdb.db.utils.datastructure;
+
+import org.apache.iotdb.db.rescon.PrimitiveArrayManager;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+
+import java.io.DataInputStream;
+import java.io.IOException;
+
+import static org.apache.iotdb.db.rescon.PrimitiveArrayManager.ARRAY_SIZE;
+
+public class TimIntTVList extends IntTVList implements TimSort {
+
+  private long[][] sortedTimestamps;
+  private int[][] sortedValues;
+
+  private int pivotValue;
+  private long pivotTime;
+
+  @Override
+  public void sort() {
+    if (sortedTimestamps == null
+        || sortedTimestamps.length < PrimitiveArrayManager.getArrayRowCount(rowCount)) {
+      sortedTimestamps =
+          (long[][]) PrimitiveArrayManager.createDataListsByType(TSDataType.INT64, rowCount);
+    }
+    if (sortedValues == null
+        || sortedValues.length < PrimitiveArrayManager.getArrayRowCount(rowCount)) {
+      sortedValues =
+          (int[][]) PrimitiveArrayManager.createDataListsByType(TSDataType.INT32, rowCount);
+    }
+    if (!sorted) {
+      sort(0, rowCount);
+    }
+    clearSortedValue();
+    clearSortedTime();
+    sorted = true;
+  }
+
+  @Override
+  public void tim_set(int src, int dest) {
+    set(src, dest);
+  }
+
+  @Override
+  public void set(int src, int dest) {
+    long srcT = getTime(src);
+    int srcV = getInt(src);
+    set(dest, srcT, srcV);
+  }
+
+  @Override
+  public void setToSorted(int src, int dest) {
+    sortedTimestamps[dest / ARRAY_SIZE][dest % ARRAY_SIZE] = getTime(src);
+    sortedValues[dest / ARRAY_SIZE][dest % ARRAY_SIZE] = getInt(src);
+  }
+
+  @Override
+  public void saveAsPivot(int pos) {
+    pivotTime = getTime(pos);
+    pivotValue = getInt(pos);
+  }
+
+  @Override
+  public void setFromSorted(int src, int dest) {
+    set(
+        dest,
+        sortedTimestamps[src / ARRAY_SIZE][src % ARRAY_SIZE],
+        sortedValues[src / ARRAY_SIZE][src % ARRAY_SIZE]);
+  }
+
+  @Override
+  public void setPivotTo(int pos) {
+    set(pos, pivotTime, pivotValue);
+  }
+
+  @Override
+  public void clearSortedTime() {
+    if (sortedTimestamps != null) {
+      sortedTimestamps = null;
+    }
+  }
+
+  @Override
+  public void clearSortedValue() {
+    if (sortedValues != null) {
+      sortedValues = null;
+    }
+  }
+
+  @Override
+  public int compare(int idx1, int idx2) {
+    long t1 = getTime(idx1);
+    long t2 = getTime(idx2);
+    return Long.compare(t1, t2);
+  }
+
+  @Override
+  public void reverseRange(int lo, int hi) {
+    hi--;
+    while (lo < hi) {
+      long loT = getTime(lo);
+      int loV = getInt(lo);
+      long hiT = getTime(hi);
+      int hiV = getInt(hi);
+      set(lo++, hiT, hiV);
+      set(hi--, loT, loV);
+    }
+  }
+
+  @Override
+  public void clear() {
+    rowCount = 0;
+    sorted = true;
+    minTime = Long.MAX_VALUE;
+    clearTime();
+    clearSortedTime();
+    clearValue();
+    clearSortedValue();

Review Comment:
   ```
   super.clear();
   clearSortedTime();
   clearSortedValue();
   ```
   super.clear() can reuse clear method in TVList.



##########
server/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java:
##########
@@ -53,14 +53,11 @@ public abstract class TVList implements WALEntryValue {
   protected List<long[]> timestamps;
   protected int rowCount;
 
-  protected long[][] sortedTimestamps;
   protected boolean sorted = true;
   // record reference count of this tv list
   // currently this reference will only be increase because we can't know when to decrease it

Review Comment:
   These notes should be on top of referenceCount.



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

To unsubscribe, e-mail: reviews-unsubscribe@iotdb.apache.org

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