You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iotdb.apache.org by qi...@apache.org on 2019/06/28 09:06:49 UTC

[incubator-iotdb] 01/03: add other types of TVList

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

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

commit cd631fedede6b7566aed1bcde313bb218b9e17a6
Author: qiaojialin <64...@qq.com>
AuthorDate: Fri Jun 28 14:31:29 2019 +0800

    add other types of TVList
---
 .../iotdb/db/utils/datastructure/BinaryTVList.java | 148 +++++++++++++++++++++
 .../db/utils/datastructure/BooleanTVList.java      | 147 ++++++++++++++++++++
 .../iotdb/db/utils/datastructure/FloatTVList.java  | 147 ++++++++++++++++++++
 .../iotdb/db/utils/datastructure/IntTVList.java    | 147 ++++++++++++++++++++
 .../iotdb/db/utils/datastructure/TVList.java       |   4 +
 .../db/utils/datastructure/TVListAllocator.java    |   1 +
 .../filenodeV2/UnsealedTsFileProcessorV2Test.java  |   3 +
 7 files changed, 597 insertions(+)

diff --git a/iotdb/src/main/java/org/apache/iotdb/db/utils/datastructure/BinaryTVList.java b/iotdb/src/main/java/org/apache/iotdb/db/utils/datastructure/BinaryTVList.java
new file mode 100644
index 0000000..73cb711
--- /dev/null
+++ b/iotdb/src/main/java/org/apache/iotdb/db/utils/datastructure/BinaryTVList.java
@@ -0,0 +1,148 @@
+/**
+ * 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;
+import org.apache.iotdb.tsfile.utils.Binary;
+
+public class BinaryTVList extends TVList {
+
+  private List<Binary[]> values;
+
+  private Binary[] sortedValues;
+
+  private Binary pivotValue;
+
+  public BinaryTVList() {
+    super();
+    values = new ArrayList<>();
+  }
+
+  @Override
+  public void putBinary(long timestamp, Binary value) {
+    checkExpansion();
+    int arrayIndex = size / SINGLE_ARRAY_SIZE;
+    int elementIndex = size % SINGLE_ARRAY_SIZE;
+    timestamps.get(arrayIndex)[elementIndex] = timestamp;
+    values.get(arrayIndex)[elementIndex] = value;
+    size++;
+  }
+
+  @Override
+  public Binary getBinary(int index) {
+    if (index >= size) {
+      throw new ArrayIndexOutOfBoundsException(index);
+    }
+    if (!sorted) {
+      int arrayIndex = index / SINGLE_ARRAY_SIZE;
+      int elementIndex = index % SINGLE_ARRAY_SIZE;
+      return values.get(arrayIndex)[elementIndex];
+    } else {
+      return sortedValues[index];
+    }
+  }
+
+  public void set(int index, long timestamp, Binary value) {
+    if (index >= size) {
+      throw new ArrayIndexOutOfBoundsException(index);
+    }
+    int arrayIndex = index / SINGLE_ARRAY_SIZE;
+    int elementIndex = index % SINGLE_ARRAY_SIZE;
+    timestamps.get(arrayIndex)[elementIndex] = timestamp;
+    values.get(arrayIndex)[elementIndex] = value;
+  }
+
+  @Override
+  public BinaryTVList clone() {
+    BinaryTVList cloneList = new BinaryTVList();
+    cloneAs(cloneList);
+    if (!sorted) {
+      for (Binary[] valueArray : values) {
+        cloneList.values.add(cloneValue(valueArray));
+      }
+    } else {
+      cloneList.sortedValues = new Binary[size];
+      System.arraycopy(sortedValues, 0, cloneList.sortedValues, 0, size);
+    }
+    return cloneList;
+  }
+
+  private Binary[] cloneValue(Binary[] array) {
+    Binary[] cloneArray = new Binary[array.length];
+    System.arraycopy(array, 0, cloneArray, 0, array.length);
+    return cloneArray;
+  }
+
+  public void sort() {
+    if (sortedTimestamps == null || sortedTimestamps.length < size) {
+      sortedTimestamps = new long[size];
+    }
+    if (sortedValues == null || sortedValues.length < size) {
+      sortedValues = new Binary[size];
+    }
+    sort(0, size);
+    sorted = true;
+  }
+
+  @Override
+  protected void setFromSorted(int src, int dest) {
+    set(dest, sortedTimestamps[src], sortedValues[src]);
+  }
+
+  protected void set(int src, int dest) {
+    long srcT = getTime(src);
+    Binary srcV = getBinary(src);
+    set(dest, srcT, srcV);
+  }
+
+  protected void setToSorted(int src, int dest) {
+    sortedTimestamps[dest] = getTime(src);
+    sortedValues[dest] = getBinary(src);
+  }
+
+  protected void reverseRange(int lo, int hi) {
+    hi--;
+    while (lo < hi) {
+      long loT = getTime(lo);
+      Binary loV = getBinary(lo);
+      long hiT = getTime(hi);
+      Binary hiV = getBinary(hi);
+      set(lo++, hiT, hiV);
+      set(hi--, loT, loV);
+    }
+  }
+
+  @Override
+  protected void expandValues() {
+    values.add(new Binary[SINGLE_ARRAY_SIZE]);
+  }
+
+  @Override
+  protected void saveAsPivot(int pos) {
+    pivotTime = getTime(pos);
+    pivotValue = getBinary(pos);
+  }
+
+  @Override
+  protected void setPivotTo(int pos) {
+    set(pos, pivotTime, pivotValue);
+  }
+
+}
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/utils/datastructure/BooleanTVList.java b/iotdb/src/main/java/org/apache/iotdb/db/utils/datastructure/BooleanTVList.java
new file mode 100644
index 0000000..f0c1b40
--- /dev/null
+++ b/iotdb/src/main/java/org/apache/iotdb/db/utils/datastructure/BooleanTVList.java
@@ -0,0 +1,147 @@
+/**
+ * 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 class BooleanTVList extends TVList {
+
+  private List<boolean[]> values;
+
+  private boolean[] sortedValues;
+
+  private boolean pivotValue;
+
+  public BooleanTVList() {
+    super();
+    values = new ArrayList<>();
+  }
+
+  @Override
+  public void putBoolean(long timestamp, boolean value) {
+    checkExpansion();
+    int arrayIndex = size / SINGLE_ARRAY_SIZE;
+    int elementIndex = size % SINGLE_ARRAY_SIZE;
+    timestamps.get(arrayIndex)[elementIndex] = timestamp;
+    values.get(arrayIndex)[elementIndex] = value;
+    size++;
+  }
+
+  @Override
+  public boolean getBoolean(int index) {
+    if (index >= size) {
+      throw new ArrayIndexOutOfBoundsException(index);
+    }
+    if (!sorted) {
+      int arrayIndex = index / SINGLE_ARRAY_SIZE;
+      int elementIndex = index % SINGLE_ARRAY_SIZE;
+      return values.get(arrayIndex)[elementIndex];
+    } else {
+      return sortedValues[index];
+    }
+  }
+
+  public void set(int index, long timestamp, boolean value) {
+    if (index >= size) {
+      throw new ArrayIndexOutOfBoundsException(index);
+    }
+    int arrayIndex = index / SINGLE_ARRAY_SIZE;
+    int elementIndex = index % SINGLE_ARRAY_SIZE;
+    timestamps.get(arrayIndex)[elementIndex] = timestamp;
+    values.get(arrayIndex)[elementIndex] = value;
+  }
+
+  @Override
+  public BooleanTVList clone() {
+    BooleanTVList cloneList = new BooleanTVList();
+    cloneAs(cloneList);
+    if (!sorted) {
+      for (boolean[] valueArray : values) {
+        cloneList.values.add(cloneValue(valueArray));
+      }
+    } else {
+      cloneList.sortedValues = new boolean[size];
+      System.arraycopy(sortedValues, 0, cloneList.sortedValues, 0, size);
+    }
+    return cloneList;
+  }
+
+  private boolean[] cloneValue(boolean[] array) {
+    boolean[] cloneArray = new boolean[array.length];
+    System.arraycopy(array, 0, cloneArray, 0, array.length);
+    return cloneArray;
+  }
+
+  public void sort() {
+    if (sortedTimestamps == null || sortedTimestamps.length < size) {
+      sortedTimestamps = new long[size];
+    }
+    if (sortedValues == null || sortedValues.length < size) {
+      sortedValues = new boolean[size];
+    }
+    sort(0, size);
+    sorted = true;
+  }
+
+  @Override
+  protected void setFromSorted(int src, int dest) {
+    set(dest, sortedTimestamps[src], sortedValues[src]);
+  }
+
+  protected void set(int src, int dest) {
+    long srcT = getTime(src);
+    boolean srcV = getBoolean(src);
+    set(dest, srcT, srcV);
+  }
+
+  protected void setToSorted(int src, int dest) {
+    sortedTimestamps[dest] = getTime(src);
+    sortedValues[dest] = getBoolean(src);
+  }
+
+  protected void reverseRange(int lo, int hi) {
+    hi--;
+    while (lo < hi) {
+      long loT = getTime(lo);
+      boolean loV = getBoolean(lo);
+      long hiT = getTime(hi);
+      boolean hiV = getBoolean(hi);
+      set(lo++, hiT, hiV);
+      set(hi--, loT, loV);
+    }
+  }
+
+  @Override
+  protected void expandValues() {
+    values.add(new boolean[SINGLE_ARRAY_SIZE]);
+  }
+
+  @Override
+  protected void saveAsPivot(int pos) {
+    pivotTime = getTime(pos);
+    pivotValue = getBoolean(pos);
+  }
+
+  @Override
+  protected void setPivotTo(int pos) {
+    set(pos, pivotTime, pivotValue);
+  }
+
+}
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/utils/datastructure/FloatTVList.java b/iotdb/src/main/java/org/apache/iotdb/db/utils/datastructure/FloatTVList.java
new file mode 100644
index 0000000..e7083bf
--- /dev/null
+++ b/iotdb/src/main/java/org/apache/iotdb/db/utils/datastructure/FloatTVList.java
@@ -0,0 +1,147 @@
+/**
+ * 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 class FloatTVList extends TVList {
+
+  private List<float[]> values;
+
+  private float[] sortedValues;
+
+  private float pivotValue;
+
+  public FloatTVList() {
+    super();
+    values = new ArrayList<>();
+  }
+
+  @Override
+  public void putFloat(long timestamp, float value) {
+    checkExpansion();
+    int arrayIndex = size / SINGLE_ARRAY_SIZE;
+    int elementIndex = size % SINGLE_ARRAY_SIZE;
+    timestamps.get(arrayIndex)[elementIndex] = timestamp;
+    values.get(arrayIndex)[elementIndex] = value;
+    size++;
+  }
+
+  @Override
+  public float getFloat(int index) {
+    if (index >= size) {
+      throw new ArrayIndexOutOfBoundsException(index);
+    }
+    if (!sorted) {
+      int arrayIndex = index / SINGLE_ARRAY_SIZE;
+      int elementIndex = index % SINGLE_ARRAY_SIZE;
+      return values.get(arrayIndex)[elementIndex];
+    } else {
+      return sortedValues[index];
+    }
+  }
+
+  public void set(int index, long timestamp, float value) {
+    if (index >= size) {
+      throw new ArrayIndexOutOfBoundsException(index);
+    }
+    int arrayIndex = index / SINGLE_ARRAY_SIZE;
+    int elementIndex = index % SINGLE_ARRAY_SIZE;
+    timestamps.get(arrayIndex)[elementIndex] = timestamp;
+    values.get(arrayIndex)[elementIndex] = value;
+  }
+
+  @Override
+  public FloatTVList clone() {
+    FloatTVList cloneList = new FloatTVList();
+    cloneAs(cloneList);
+    if (!sorted) {
+      for (float[] valueArray : values) {
+        cloneList.values.add(cloneValue(valueArray));
+      }
+    } else {
+      cloneList.sortedValues = new float[size];
+      System.arraycopy(sortedValues, 0, cloneList.sortedValues, 0, size);
+    }
+    return cloneList;
+  }
+
+  private float[] cloneValue(float[] array) {
+    float[] cloneArray = new float[array.length];
+    System.arraycopy(array, 0, cloneArray, 0, array.length);
+    return cloneArray;
+  }
+
+  public void sort() {
+    if (sortedTimestamps == null || sortedTimestamps.length < size) {
+      sortedTimestamps = new long[size];
+    }
+    if (sortedValues == null || sortedValues.length < size) {
+      sortedValues = new float[size];
+    }
+    sort(0, size);
+    sorted = true;
+  }
+
+  @Override
+  protected void setFromSorted(int src, int dest) {
+    set(dest, sortedTimestamps[src], sortedValues[src]);
+  }
+
+  protected void set(int src, int dest) {
+    long srcT = getTime(src);
+    float srcV = getFloat(src);
+    set(dest, srcT, srcV);
+  }
+
+  protected void setToSorted(int src, int dest) {
+    sortedTimestamps[dest] = getTime(src);
+    sortedValues[dest] = getFloat(src);
+  }
+
+  protected void reverseRange(int lo, int hi) {
+    hi--;
+    while (lo < hi) {
+      long loT = getTime(lo);
+      float loV = getFloat(lo);
+      long hiT = getTime(hi);
+      float hiV = getFloat(hi);
+      set(lo++, hiT, hiV);
+      set(hi--, loT, loV);
+    }
+  }
+
+  @Override
+  protected void expandValues() {
+    values.add(new float[SINGLE_ARRAY_SIZE]);
+  }
+
+  @Override
+  protected void saveAsPivot(int pos) {
+    pivotTime = getTime(pos);
+    pivotValue = getFloat(pos);
+  }
+
+  @Override
+  protected void setPivotTo(int pos) {
+    set(pos, pivotTime, pivotValue);
+  }
+
+}
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/utils/datastructure/IntTVList.java b/iotdb/src/main/java/org/apache/iotdb/db/utils/datastructure/IntTVList.java
new file mode 100644
index 0000000..b2a5ad0
--- /dev/null
+++ b/iotdb/src/main/java/org/apache/iotdb/db/utils/datastructure/IntTVList.java
@@ -0,0 +1,147 @@
+/**
+ * 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 class IntTVList extends TVList {
+
+  private List<int[]> values;
+
+  private int[] sortedValues;
+
+  private int pivotValue;
+
+  public IntTVList() {
+    super();
+    values = new ArrayList<>();
+  }
+
+  @Override
+  public void putInt(long timestamp, int value) {
+    checkExpansion();
+    int arrayIndex = size / SINGLE_ARRAY_SIZE;
+    int elementIndex = size % SINGLE_ARRAY_SIZE;
+    timestamps.get(arrayIndex)[elementIndex] = timestamp;
+    values.get(arrayIndex)[elementIndex] = value;
+    size++;
+  }
+
+  @Override
+  public int getInt(int index) {
+    if (index >= size) {
+      throw new ArrayIndexOutOfBoundsException(index);
+    }
+    if (!sorted) {
+      int arrayIndex = index / SINGLE_ARRAY_SIZE;
+      int elementIndex = index % SINGLE_ARRAY_SIZE;
+      return values.get(arrayIndex)[elementIndex];
+    } else {
+      return sortedValues[index];
+    }
+  }
+
+  public void set(int index, long timestamp, int value) {
+    if (index >= size) {
+      throw new ArrayIndexOutOfBoundsException(index);
+    }
+    int arrayIndex = index / SINGLE_ARRAY_SIZE;
+    int elementIndex = index % SINGLE_ARRAY_SIZE;
+    timestamps.get(arrayIndex)[elementIndex] = timestamp;
+    values.get(arrayIndex)[elementIndex] = value;
+  }
+
+  @Override
+  public IntTVList clone() {
+    IntTVList cloneList = new IntTVList();
+    cloneAs(cloneList);
+    if (!sorted) {
+      for (int[] valueArray : values) {
+        cloneList.values.add(cloneValue(valueArray));
+      }
+    } else {
+      cloneList.sortedValues = new int[size];
+      System.arraycopy(sortedValues, 0, cloneList.sortedValues, 0, size);
+    }
+    return cloneList;
+  }
+
+  private int[] cloneValue(int[] array) {
+    int[] cloneArray = new int[array.length];
+    System.arraycopy(array, 0, cloneArray, 0, array.length);
+    return cloneArray;
+  }
+
+  public void sort() {
+    if (sortedTimestamps == null || sortedTimestamps.length < size) {
+      sortedTimestamps = new long[size];
+    }
+    if (sortedValues == null || sortedValues.length < size) {
+      sortedValues = new int[size];
+    }
+    sort(0, size);
+    sorted = true;
+  }
+
+  @Override
+  protected void setFromSorted(int src, int dest) {
+    set(dest, sortedTimestamps[src], sortedValues[src]);
+  }
+
+  protected void set(int src, int dest) {
+    long srcT = getTime(src);
+    int srcV = getInt(src);
+    set(dest, srcT, srcV);
+  }
+
+  protected void setToSorted(int src, int dest) {
+    sortedTimestamps[dest] = getTime(src);
+    sortedValues[dest] = getInt(src);
+  }
+
+  protected 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
+  protected void expandValues() {
+    values.add(new int[SINGLE_ARRAY_SIZE]);
+  }
+
+  @Override
+  protected void saveAsPivot(int pos) {
+    pivotTime = getTime(pos);
+    pivotValue = getInt(pos);
+  }
+
+  @Override
+  protected void setPivotTo(int pos) {
+    set(pos, pivotTime, pivotValue);
+  }
+
+}
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
index e394c64..cc7880f 100644
--- 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
@@ -199,13 +199,17 @@ public abstract class TVList {
   public static TVList newList(TSDataType dataType) {
     switch (dataType) {
       case TEXT:
+        return new BinaryTVList();
       case FLOAT:
+        return new FloatTVList();
       case INT32:
+        return new IntTVList();
       case INT64:
         return new LongTVList();
       case DOUBLE:
         return new DoubleTVList();
       case BOOLEAN:
+        return new BooleanTVList();
     }
     return null;
   }
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/utils/datastructure/TVListAllocator.java b/iotdb/src/main/java/org/apache/iotdb/db/utils/datastructure/TVListAllocator.java
index 3ddeffa..cdf387e 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/utils/datastructure/TVListAllocator.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/utils/datastructure/TVListAllocator.java
@@ -25,6 +25,7 @@ import java.util.concurrent.ConcurrentLinkedQueue;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 
 public class TVListAllocator {
+
   private Map<String, ConcurrentLinkedQueue<TVList>> tvListCache = new ConcurrentHashMap<>();
 
   public TVList allocate(String identifier, TSDataType dataType) {
diff --git a/iotdb/src/test/java/org/apache/iotdb/db/engine/filenodeV2/UnsealedTsFileProcessorV2Test.java b/iotdb/src/test/java/org/apache/iotdb/db/engine/filenodeV2/UnsealedTsFileProcessorV2Test.java
index bdd53dc..94f5f1b 100644
--- a/iotdb/src/test/java/org/apache/iotdb/db/engine/filenodeV2/UnsealedTsFileProcessorV2Test.java
+++ b/iotdb/src/test/java/org/apache/iotdb/db/engine/filenodeV2/UnsealedTsFileProcessorV2Test.java
@@ -66,6 +66,7 @@ public class UnsealedTsFileProcessorV2Test {
   @After
   public void tearDown() throws Exception {
     EnvironmentUtils.cleanEnv();
+    EnvironmentUtils.cleanDir("data");
   }
 
   @Test
@@ -111,6 +112,7 @@ public class UnsealedTsFileProcessorV2Test {
     assertEquals(1, right.size());
     assertEquals(measurementId, right.get(0).getMeasurementUid());
     assertEquals(dataType, right.get(0).getTsDataType());
+    processor.syncClose();
   }
 
 
@@ -145,6 +147,7 @@ public class UnsealedTsFileProcessorV2Test {
     assertEquals(10, right.size());
     assertEquals(measurementId, right.get(0).getMeasurementUid());
     assertEquals(dataType, right.get(0).getTsDataType());
+    processor.syncClose();
   }