You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iotdb.apache.org by ji...@apache.org on 2019/06/27 07:46:36 UTC

[incubator-iotdb] branch feature_async_close_tsfile updated: abstract TVList

This is an automated email from the ASF dual-hosted git repository.

jiangtian pushed a commit to branch feature_async_close_tsfile
in repository https://gitbox.apache.org/repos/asf/incubator-iotdb.git


The following commit(s) were added to refs/heads/feature_async_close_tsfile by this push:
     new bcaad93  abstract TVList
     new c34a32b  Merge branch 'feature_async_close_tsfile' of github.com:apache/incubator-iotdb into feature_async_close_tsfile
bcaad93 is described below

commit bcaad93d34f79dce6b0f92bb59072d11a7cd034a
Author: 江天 <jt...@163.com>
AuthorDate: Thu Jun 27 15:43:49 2019 +0800

    abstract TVList
---
 .../iotdb/db/utils/datastructure/LongTVList.java   | 138 +++++-------------
 .../iotdb/db/utils/datastructure/TVList.java       | 157 +++++++++++++++++++++
 2 files changed, 194 insertions(+), 101 deletions(-)

diff --git a/iotdb/src/main/java/org/apache/iotdb/db/utils/datastructure/LongTVList.java b/iotdb/src/main/java/org/apache/iotdb/db/utils/datastructure/LongTVList.java
index d8c1d1f..1fbcd07 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/utils/datastructure/LongTVList.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/utils/datastructure/LongTVList.java
@@ -21,30 +21,20 @@ package org.apache.iotdb.db.utils.datastructure;
 import java.util.ArrayList;
 import java.util.List;
 
-public class LongTVList {
+public class LongTVList extends TVList {
 
-  private static final int SMALL_ARRAY_LENGTH = 32;
-  private static final int SINGLE_ARRAY_SIZE = 512;
-
-  private Class clazz;
   private List<long[]> values;
-  private List<long[]> timestamps;
-
-  private int size; // Total data number of all objects of current ArrayList
 
-  private long[] sortedTimestamps;
   private long[] sortedValues;
-  private boolean sorted = false;
 
-  public LongTVList(Class clazz) {
-    this.clazz = clazz;
+  public LongTVList() {
+    super();
     values = new ArrayList<>();
-    timestamps = new ArrayList<>();
-    size = 0;
 
   }
 
-  public void append(long timestamp, long value) {
+  @Override
+  public void putLong(long timestamp, long value) {
     if ((size % SINGLE_ARRAY_SIZE) == 0) {
       values.add(new long[SINGLE_ARRAY_SIZE]);
       timestamps.add(new long[SINGLE_ARRAY_SIZE]);
@@ -56,20 +46,8 @@ public class LongTVList {
     size++;
   }
 
-  public long getTimestamp(int index) {
-    if (index >= size) {
-      throw new ArrayIndexOutOfBoundsException(index);
-    }
-    if (!sorted) {
-      int arrayIndex = index / SINGLE_ARRAY_SIZE;
-      int elementIndex = index % SINGLE_ARRAY_SIZE;
-      return timestamps.get(arrayIndex)[elementIndex];
-    } else {
-      return sortedTimestamps[index];
-    }
-  }
-
-  public long getValue(int index) {
+  @Override
+  public long getLong(int index) {
     if (index >= size) {
       throw new ArrayIndexOutOfBoundsException(index);
     }
@@ -82,10 +60,6 @@ public class LongTVList {
     }
   }
 
-  public int size() {
-    return size;
-  }
-
   public void set(int index, long timestamp, long value) {
     if (index >= size) {
       throw new ArrayIndexOutOfBoundsException(index);
@@ -98,7 +72,7 @@ public class LongTVList {
 
   @Override
   public LongTVList clone() {
-    LongTVList cloneList = new LongTVList(clazz);
+    LongTVList cloneList = new LongTVList();
     if (!sorted) {
       for (long[] valueArray : values) {
         cloneList.values.add(cloneValue(valueArray));
@@ -118,12 +92,6 @@ public class LongTVList {
     return cloneList;
   }
 
-  private long[] cloneTime(long[] array) {
-    long[] cloneArray = new long[array.length];
-    System.arraycopy(array, 0, cloneArray, 0, array.length);
-    return cloneArray;
-  }
-
   private long[] cloneValue(long[] array) {
     long[] cloneArray = new long[array.length];
     System.arraycopy(array, 0, cloneArray, 0, array.length);
@@ -137,35 +105,19 @@ public class LongTVList {
   public void sort() {
     sortedTimestamps = new long[size];
     sortedValues = new long[size];
-
     sort(0, size);
   }
 
-  private void sort(int lo, int hi) {
-    if (hi - lo <= SMALL_ARRAY_LENGTH) {
-      int initRunLen = countRunAndMakeAscending(lo, hi);
-      binarySort(lo, hi, lo + initRunLen);
-      return;
-    }
-    int mid = (lo + hi) >>> 1;
-    sort(lo, mid);
-    sort(mid, hi);
-    merge(lo, mid, hi);
-    sorted = true;
-    values = null;
-    timestamps = null;
-  }
-
-  private void merge(int lo, int mid, int hi) {
+  protected void merge(int lo, int mid, int hi) {
     int tmpIdx = 0;
 
     int leftIdx = lo;
     int rightIdx = mid;
 
-    long leftFirstT = getTimestamp(leftIdx);
-    long leftFirstV = getValue(leftIdx);
-    long rightFirstT = getTimestamp(rightIdx);
-    long rightFirstV = getValue(rightIdx);
+    long leftFirstT = getTime(leftIdx);
+    long leftFirstV = getLong(leftIdx);
+    long rightFirstT = getTime(rightIdx);
+    long rightFirstV = getLong(rightIdx);
 
     int endSide = 0;
     while (endSide == 0) {
@@ -177,8 +129,8 @@ public class LongTVList {
         if (leftIdx == mid) {
           endSide = 1;
         } else {
-          leftFirstT = getTimestamp(leftIdx);
-          leftFirstV = getValue(leftIdx);
+          leftFirstT = getTime(leftIdx);
+          leftFirstV = getLong(leftIdx);
         }
       } else {
         sortedTimestamps[lo + tmpIdx] = rightFirstT;
@@ -188,23 +140,23 @@ public class LongTVList {
         if (rightIdx == hi) {
           endSide = 2;
         } else {
-          rightFirstT = getTimestamp(leftIdx);
-          rightFirstV = getValue(leftIdx);
+          rightFirstT = getTime(leftIdx);
+          rightFirstV = getLong(leftIdx);
         }
       }
     }
     if (endSide == 1) {
       for (; rightIdx < hi; rightIdx++) {
-        rightFirstT = getTimestamp(leftIdx);
-        rightFirstV = getValue(leftIdx);
+        rightFirstT = getTime(leftIdx);
+        rightFirstV = getLong(leftIdx);
         sortedTimestamps[lo + tmpIdx] = rightFirstT;
         sortedValues[lo + tmpIdx] = rightFirstV;
         tmpIdx ++;
       }
     } else {
       for(; leftIdx < mid; leftIdx++) {
-        leftFirstT = getTimestamp(leftIdx);
-        leftFirstV = getValue(leftIdx);
+        leftFirstT = getTime(leftIdx);
+        leftFirstV = getLong(leftIdx);
         sortedTimestamps[lo + tmpIdx] = leftFirstT;
         sortedValues[lo + tmpIdx] = leftFirstV;
         tmpIdx ++;
@@ -215,22 +167,27 @@ public class LongTVList {
     }
   }
 
+  @Override
+  protected void cleanAfterSort() {
+    values = null;
+  }
+
   private void set(int src, int dest) {
-    long srcT = getTimestamp(src);
-    long srcV = getTimestamp(src);
+    long srcT = getTime(src);
+    long srcV = getLong(src);
     set(dest, srcT, srcV);
   }
 
   /**
    * From TimSort.java
    */
-  private void binarySort(int lo, int hi, int start) {
+  protected void binarySort(int lo, int hi, int start) {
     assert lo <= start && start <= hi;
     if (start == lo)
       start++;
     for ( ; start < hi; start++) {
-      long pivotT = getTimestamp(start);
-      long pivotV = getValue(start);
+      long pivotT = getTime(start);
+      long pivotV = getLong(start);
 
       // Set left (and right) to the index where a[start] (pivot) belongs
       int left = lo;
@@ -243,7 +200,7 @@ public class LongTVList {
        */
       while (left < right) {
         int mid = (left + right) >>> 1;
-        if (pivotT < getTimestamp(mid))
+        if (pivotT < getTime(mid))
           right = mid;
         else
           left = mid + 1;
@@ -265,34 +222,13 @@ public class LongTVList {
     }
   }
 
-  private int countRunAndMakeAscending(int lo, int hi) {
-    assert lo < hi;
-    int runHi = lo + 1;
-    if (runHi == hi) {
-      return 1;
-    }
-
-    // Find end of run, and reverse range if descending
-    if (getTimestamp(runHi++) < getTimestamp(lo)) { // Descending
-      while (runHi < hi && getTimestamp(runHi) < getTimestamp(runHi - 1)) {
-        runHi++;
-      }
-      reverseRange(lo, runHi);
-    } else {                              // Ascending
-      while (runHi < hi &&getTimestamp(runHi) >= getTimestamp(runHi - 1))
-        runHi++;
-    }
-
-    return runHi - lo;
-  }
-
-  private void reverseRange(int lo, int hi) {
+  protected void reverseRange(int lo, int hi) {
     hi--;
     while (lo < hi) {
-      long loT = getTimestamp(lo);
-      long loV = getValue(lo);
-      long hiT = getTimestamp(hi);
-      long hiV = getValue(hi);
+      long loT = getTime(lo);
+      long loV = getLong(lo);
+      long hiT = getTime(hi);
+      long hiV = getLong(hi);
       set(lo++, hiT, hiV);
       set(hi--, loT, loV);
       lo++;
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java b/iotdb/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java
new file mode 100644
index 0000000..2d85fb0
--- /dev/null
+++ b/iotdb/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java
@@ -0,0 +1,157 @@
+/**
+ * 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 java.util.ArrayList;
+import java.util.List;
+
+public abstract class TVList {
+
+  protected static final int SMALL_ARRAY_LENGTH = 32;
+  protected static final int SINGLE_ARRAY_SIZE = 512;
+
+  protected List<long[]> timestamps;
+  protected int size;
+
+  protected long[] sortedTimestamps;
+  protected boolean sorted = false;
+
+  public TVList() {
+    timestamps = new ArrayList<>();
+    size = 0;
+  }
+
+  public int size() {
+    return size;
+  }
+
+  public long getTime(int index) {
+    if (index >= size) {
+      throw new ArrayIndexOutOfBoundsException(index);
+    }
+    if (!sorted) {
+      int arrayIndex = index / SINGLE_ARRAY_SIZE;
+      int elementIndex = index % SINGLE_ARRAY_SIZE;
+      return timestamps.get(arrayIndex)[elementIndex];
+    } else {
+      return sortedTimestamps[index];
+    }
+  }
+
+  public void putLong(long time, long value) {
+    throw new UnsupportedOperationException("DataType not consistent");
+  }
+
+  public void putInt(long time, int value) {
+    throw new UnsupportedOperationException("DataType not consistent");
+  }
+
+  public void putFloat(long time, float value) {
+    throw new UnsupportedOperationException("DataType not consistent");
+  }
+
+  public void putDouble(long time, double value) {
+    throw new UnsupportedOperationException("DataType not consistent");
+  }
+
+  public void putString(long time, String value) {
+    throw new UnsupportedOperationException("DataType not consistent");
+  }
+
+  public void putBoolean(long time, boolean value) {
+    throw new UnsupportedOperationException("DataType not consistent");
+  }
+
+  public long getLong(int index) {
+    throw new UnsupportedOperationException("DataType not consistent");
+  }
+
+  public int getInt(int index) {
+    throw new UnsupportedOperationException("DataType not consistent");
+  }
+
+  public float getFloat(int index) {
+    throw new UnsupportedOperationException("DataType not consistent");
+  }
+
+  public double getDouble(int index) {
+    throw new UnsupportedOperationException("DataType not consistent");
+  }
+
+  public String getString(int index) {
+    throw new UnsupportedOperationException("DataType not consistent");
+  }
+
+  public boolean getBoolean(int index) {
+    throw new UnsupportedOperationException("DataType not consistent");
+  }
+
+  public abstract void sort();
+
+  protected abstract void binarySort(int lo, int hi, int start);
+
+  protected abstract void merge(int lo, int mid, int hi);
+
+  protected abstract void cleanAfterSort();
+
+  protected abstract void reverseRange(int lo, int hi);
+
+  protected long[] cloneTime(long[] array) {
+    long[] cloneArray = new long[array.length];
+    System.arraycopy(array, 0, cloneArray, 0, array.length);
+    return cloneArray;
+  }
+
+  protected void sort(int lo, int hi) {
+    if (hi - lo <= SMALL_ARRAY_LENGTH) {
+      int initRunLen = countRunAndMakeAscending(lo, hi);
+      binarySort(lo, hi, lo + initRunLen);
+      return;
+    }
+    int mid = (lo + hi) >>> 1;
+    sort(lo, mid);
+    sort(mid, hi);
+    merge(lo, mid, hi);
+    sorted = true;
+    cleanAfterSort();
+    timestamps = null;
+  }
+
+  protected int countRunAndMakeAscending(int lo, int hi) {
+    assert lo < hi;
+    int runHi = lo + 1;
+    if (runHi == hi) {
+      return 1;
+    }
+
+    // Find end of run, and reverse range if descending
+    if (getTime(runHi++) < getTime(lo)) { // Descending
+      while (runHi < hi && getTime(runHi) < getTime(runHi - 1)) {
+        runHi++;
+      }
+      reverseRange(lo, runHi);
+    } else {                              // Ascending
+      while (runHi < hi &&getTime(runHi) >= getTime(runHi - 1))
+        runHi++;
+    }
+
+    return runHi - lo;
+  }
+}