From ef5de534a47a35396f2788ef4c9ead9d77b87587 Mon Sep 17 00:00:00 2001 From: shizy Date: Wed, 27 Nov 2024 14:07:54 +0800 Subject: [PATCH 01/54] null bitmap for int tvlist --- .../db/utils/datastructure/AlignedTVList.java | 34 ----- .../utils/datastructure/BackBinaryTVList.java | 13 +- .../datastructure/BackBooleanTVList.java | 12 +- .../utils/datastructure/BackDoubleTVList.java | 12 +- .../utils/datastructure/BackFloatTVList.java | 12 +- .../db/utils/datastructure/BackIntTVList.java | 12 +- .../utils/datastructure/BackLongTVList.java | 12 +- .../db/utils/datastructure/BinaryTVList.java | 71 ++++----- .../db/utils/datastructure/BooleanTVList.java | 36 +++-- .../db/utils/datastructure/DoubleTVList.java | 37 +++-- .../db/utils/datastructure/FloatTVList.java | 37 +++-- .../db/utils/datastructure/IntTVList.java | 37 +++-- .../db/utils/datastructure/LongTVList.java | 37 +++-- .../datastructure/QuickBinaryTVList.java | 8 +- .../datastructure/QuickBooleanTVList.java | 6 +- .../datastructure/QuickDoubleTVList.java | 6 +- .../utils/datastructure/QuickFloatTVList.java | 6 +- .../utils/datastructure/QuickIntTVList.java | 6 +- .../utils/datastructure/QuickLongTVList.java | 6 +- .../iotdb/db/utils/datastructure/TVList.java | 138 +++++++++++++++--- .../utils/datastructure/TimBinaryTVList.java | 34 ++--- .../utils/datastructure/TimBooleanTVList.java | 32 ++-- .../utils/datastructure/TimDoubleTVList.java | 32 ++-- .../utils/datastructure/TimFloatTVList.java | 33 ++--- .../db/utils/datastructure/TimIntTVList.java | 29 ++-- .../db/utils/datastructure/TimLongTVList.java | 32 ++-- 26 files changed, 402 insertions(+), 328 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/AlignedTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/AlignedTVList.java index f0a645bf3ecd9..442fe84b37838 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/AlignedTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/AlignedTVList.java @@ -68,11 +68,6 @@ public abstract class AlignedTVList extends TVList { // Index relation: columnIndex(dataTypeIndex) -> arrayIndex -> elementIndex protected List> values; - // List of index array, add 1 when expanded -> data point index array - // Index relation: arrayIndex -> elementIndex - // Used in sort method, sort only changes indices - protected List indices; - // Data type list -> list of BitMap, add 1 when expanded -> BitMap(maybe null), marked means the // Value is null // Index relation: columnIndex(dataTypeIndex) -> arrayIndex -> elementIndex @@ -615,20 +610,6 @@ public void deleteColumn(int columnIndex) { bitMaps.remove(columnIndex); } - protected void set(int index, long timestamp, int value) { - int arrayIndex = index / ARRAY_SIZE; - int elementIndex = index % ARRAY_SIZE; - timestamps.get(arrayIndex)[elementIndex] = timestamp; - indices.get(arrayIndex)[elementIndex] = value; - } - - @SuppressWarnings("squid:S3776") // Suppress high Cognitive Complexity warning - protected int[] cloneIndex(int[] array) { - int[] cloneArray = new int[array.length]; - System.arraycopy(array, 0, cloneArray, 0, array.length); - return cloneArray; - } - protected Object cloneValue(TSDataType type, Object value) { switch (type) { case TEXT: @@ -720,21 +701,6 @@ private void markRowNull(int i) { timeColDeletedMap.mark(getValueIndex(i)); } - /** - * Get the row index value in index column. - * - * @param index row index - */ - @Override - public int getValueIndex(int index) { - if (index >= rowCount) { - throw new ArrayIndexOutOfBoundsException(index); - } - int arrayIndex = index / ARRAY_SIZE; - int elementIndex = index % ARRAY_SIZE; - return indices.get(arrayIndex)[elementIndex]; - } - /** * Get the valid original row index in a column by a given time duplicated original row index * list. diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BackBinaryTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BackBinaryTVList.java index 58ab8f5452b5e..9a72499376b4c 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BackBinaryTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BackBinaryTVList.java @@ -21,7 +21,6 @@ import org.apache.iotdb.db.storageengine.rescon.memory.PrimitiveArrayManager; import org.apache.tsfile.enums.TSDataType; -import org.apache.tsfile.utils.Binary; import java.util.ArrayList; import java.util.List; @@ -30,7 +29,7 @@ public class BackBinaryTVList extends QuickBinaryTVList implements BackwardSort { private final List tmpTimestamps = new ArrayList<>(); - private final List tmpValues = new ArrayList<>(); + private final List tmpIndices = new ArrayList<>(); private int tmpLength = 0; @Override @@ -47,13 +46,13 @@ public void setFromTmp(int src, int dest) { set( dest, tmpTimestamps.get(src / ARRAY_SIZE)[src % ARRAY_SIZE], - tmpValues.get(src / ARRAY_SIZE)[src % ARRAY_SIZE]); + tmpIndices.get(src / ARRAY_SIZE)[src % ARRAY_SIZE]); } @Override public void setToTmp(int src, int dest) { tmpTimestamps.get(dest / ARRAY_SIZE)[dest % ARRAY_SIZE] = getTime(src); - tmpValues.get(dest / ARRAY_SIZE)[dest % ARRAY_SIZE] = getBinary(src); + tmpIndices.get(dest / ARRAY_SIZE)[dest % ARRAY_SIZE] = getValueIndex(src); } @Override @@ -72,7 +71,7 @@ public int compareTmp(int idx, int tmpIdx) { public void checkTmpLength(int len) { while (len > tmpLength) { tmpTimestamps.add((long[]) getPrimitiveArraysByType(TSDataType.INT64)); - tmpValues.add((Binary[]) getPrimitiveArraysByType(TSDataType.TEXT)); + tmpIndices.add((int[]) getPrimitiveArraysByType(TSDataType.INT32)); tmpLength += ARRAY_SIZE; } } @@ -83,9 +82,9 @@ public void clearTmp() { PrimitiveArrayManager.release(dataArray); } tmpTimestamps.clear(); - for (Binary[] dataArray : tmpValues) { + for (int[] dataArray : tmpIndices) { PrimitiveArrayManager.release(dataArray); } - tmpValues.clear(); + tmpIndices.clear(); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BackBooleanTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BackBooleanTVList.java index 2b2a4720dc21e..c7b39a388ee91 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BackBooleanTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BackBooleanTVList.java @@ -30,7 +30,7 @@ public class BackBooleanTVList extends QuickBooleanTVList implements BackwardSort { private final List tmpTimestamps = new ArrayList<>(); - private final List tmpValues = new ArrayList<>(); + private final List tmpIndices = new ArrayList<>(); private int tmpLength = 0; @Override @@ -47,13 +47,13 @@ public void setFromTmp(int src, int dest) { set( dest, tmpTimestamps.get(src / ARRAY_SIZE)[src % ARRAY_SIZE], - tmpValues.get(src / ARRAY_SIZE)[src % ARRAY_SIZE]); + tmpIndices.get(src / ARRAY_SIZE)[src % ARRAY_SIZE]); } @Override public void setToTmp(int src, int dest) { tmpTimestamps.get(dest / ARRAY_SIZE)[dest % ARRAY_SIZE] = getTime(src); - tmpValues.get(dest / ARRAY_SIZE)[dest % ARRAY_SIZE] = getBoolean(src); + tmpIndices.get(dest / ARRAY_SIZE)[dest % ARRAY_SIZE] = getValueIndex(src); } @Override @@ -72,7 +72,7 @@ public int compareTmp(int idx, int tmpIdx) { public void checkTmpLength(int len) { while (len > tmpLength) { tmpTimestamps.add((long[]) getPrimitiveArraysByType(TSDataType.INT64)); - tmpValues.add((boolean[]) getPrimitiveArraysByType(TSDataType.BOOLEAN)); + tmpIndices.add((int[]) getPrimitiveArraysByType(TSDataType.INT32)); tmpLength += ARRAY_SIZE; } } @@ -83,9 +83,9 @@ public void clearTmp() { PrimitiveArrayManager.release(dataArray); } tmpTimestamps.clear(); - for (boolean[] dataArray : tmpValues) { + for (int[] dataArray : tmpIndices) { PrimitiveArrayManager.release(dataArray); } - tmpValues.clear(); + tmpIndices.clear(); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BackDoubleTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BackDoubleTVList.java index 2c83c48e67b28..48c80d1a3ab40 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BackDoubleTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BackDoubleTVList.java @@ -30,7 +30,7 @@ public class BackDoubleTVList extends QuickDoubleTVList implements BackwardSort { private final List tmpTimestamps = new ArrayList<>(); - private final List tmpValues = new ArrayList<>(); + private final List tmpIndices = new ArrayList<>(); private int tmpLength = 0; @Override @@ -47,13 +47,13 @@ public void setFromTmp(int src, int dest) { set( dest, tmpTimestamps.get(src / ARRAY_SIZE)[src % ARRAY_SIZE], - tmpValues.get(src / ARRAY_SIZE)[src % ARRAY_SIZE]); + tmpIndices.get(src / ARRAY_SIZE)[src % ARRAY_SIZE]); } @Override public void setToTmp(int src, int dest) { tmpTimestamps.get(dest / ARRAY_SIZE)[dest % ARRAY_SIZE] = getTime(src); - tmpValues.get(dest / ARRAY_SIZE)[dest % ARRAY_SIZE] = getDouble(src); + tmpIndices.get(dest / ARRAY_SIZE)[dest % ARRAY_SIZE] = getValueIndex(src); } @Override @@ -72,7 +72,7 @@ public int compareTmp(int idx, int tmpIdx) { public void checkTmpLength(int len) { while (len > tmpLength) { tmpTimestamps.add((long[]) getPrimitiveArraysByType(TSDataType.INT64)); - tmpValues.add((double[]) getPrimitiveArraysByType(TSDataType.DOUBLE)); + tmpIndices.add((int[]) getPrimitiveArraysByType(TSDataType.INT32)); tmpLength += ARRAY_SIZE; } } @@ -83,9 +83,9 @@ public void clearTmp() { PrimitiveArrayManager.release(dataArray); } tmpTimestamps.clear(); - for (double[] dataArray : tmpValues) { + for (int[] dataArray : tmpIndices) { PrimitiveArrayManager.release(dataArray); } - tmpValues.clear(); + tmpIndices.clear(); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BackFloatTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BackFloatTVList.java index e274cfe47b58d..79cf6d2bc06c3 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BackFloatTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BackFloatTVList.java @@ -30,7 +30,7 @@ public class BackFloatTVList extends QuickFloatTVList implements BackwardSort { private final List tmpTimestamps = new ArrayList<>(); - private final List tmpValues = new ArrayList<>(); + private final List tmpIndices = new ArrayList<>(); private int tmpLength = 0; @Override @@ -47,13 +47,13 @@ public void setFromTmp(int src, int dest) { set( dest, tmpTimestamps.get(src / ARRAY_SIZE)[src % ARRAY_SIZE], - tmpValues.get(src / ARRAY_SIZE)[src % ARRAY_SIZE]); + tmpIndices.get(src / ARRAY_SIZE)[src % ARRAY_SIZE]); } @Override public void setToTmp(int src, int dest) { tmpTimestamps.get(dest / ARRAY_SIZE)[dest % ARRAY_SIZE] = getTime(src); - tmpValues.get(dest / ARRAY_SIZE)[dest % ARRAY_SIZE] = getFloat(src); + tmpIndices.get(dest / ARRAY_SIZE)[dest % ARRAY_SIZE] = getValueIndex(src); } @Override @@ -72,7 +72,7 @@ public int compareTmp(int idx, int tmpIdx) { public void checkTmpLength(int len) { while (len > tmpLength) { tmpTimestamps.add((long[]) getPrimitiveArraysByType(TSDataType.INT64)); - tmpValues.add((float[]) getPrimitiveArraysByType(TSDataType.FLOAT)); + tmpIndices.add((int[]) getPrimitiveArraysByType(TSDataType.INT32)); tmpLength += ARRAY_SIZE; } } @@ -83,9 +83,9 @@ public void clearTmp() { PrimitiveArrayManager.release(dataArray); } tmpTimestamps.clear(); - for (float[] dataArray : tmpValues) { + for (int[] dataArray : tmpIndices) { PrimitiveArrayManager.release(dataArray); } - tmpValues.clear(); + tmpIndices.clear(); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BackIntTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BackIntTVList.java index 9ef7436c7e89b..093af0d175b51 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BackIntTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BackIntTVList.java @@ -31,7 +31,7 @@ public class BackIntTVList extends QuickIntTVList implements BackwardSort { private final List tmpTimestamps = new ArrayList<>(); - private final List tmpValues = new ArrayList<>(); + private final List tmpIndices = new ArrayList<>(); private int tmpLength = 0; @Override @@ -48,13 +48,13 @@ public void setFromTmp(int src, int dest) { set( dest, tmpTimestamps.get(src / ARRAY_SIZE)[src % ARRAY_SIZE], - tmpValues.get(src / ARRAY_SIZE)[src % ARRAY_SIZE]); + tmpIndices.get(src / ARRAY_SIZE)[src % ARRAY_SIZE]); } @Override public void setToTmp(int src, int dest) { tmpTimestamps.get(dest / ARRAY_SIZE)[dest % ARRAY_SIZE] = getTime(src); - tmpValues.get(dest / ARRAY_SIZE)[dest % ARRAY_SIZE] = getInt(src); + tmpIndices.get(dest / ARRAY_SIZE)[dest % ARRAY_SIZE] = getValueIndex(src); } @Override @@ -73,7 +73,7 @@ public int compareTmp(int idx, int tmpIdx) { public void checkTmpLength(int len) { while (len > tmpLength) { tmpTimestamps.add((long[]) getPrimitiveArraysByType(TSDataType.INT64)); - tmpValues.add((int[]) getPrimitiveArraysByType(TSDataType.INT32)); + tmpIndices.add((int[]) getPrimitiveArraysByType(TSDataType.INT32)); tmpLength += ARRAY_SIZE; } } @@ -84,9 +84,9 @@ public void clearTmp() { PrimitiveArrayManager.release(dataArray); } tmpTimestamps.clear(); - for (int[] dataArray : tmpValues) { + for (int[] dataArray : tmpIndices) { PrimitiveArrayManager.release(dataArray); } - tmpValues.clear(); + tmpIndices.clear(); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BackLongTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BackLongTVList.java index c24473f299538..cc28e8f09e641 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BackLongTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BackLongTVList.java @@ -29,7 +29,7 @@ public class BackLongTVList extends QuickLongTVList implements BackwardSort { private final List tmpTimestamps = new ArrayList<>(); - private final List tmpValues = new ArrayList<>(); + private final List tmpIndices = new ArrayList<>(); private int tmpLength = 0; @Override @@ -46,13 +46,13 @@ public void setFromTmp(int src, int dest) { set( dest, tmpTimestamps.get(src / ARRAY_SIZE)[src % ARRAY_SIZE], - tmpValues.get(src / ARRAY_SIZE)[src % ARRAY_SIZE]); + tmpIndices.get(src / ARRAY_SIZE)[src % ARRAY_SIZE]); } @Override public void setToTmp(int src, int dest) { tmpTimestamps.get(dest / ARRAY_SIZE)[dest % ARRAY_SIZE] = getTime(src); - tmpValues.get(dest / ARRAY_SIZE)[dest % ARRAY_SIZE] = getLong(src); + tmpIndices.get(dest / ARRAY_SIZE)[dest % ARRAY_SIZE] = getValueIndex(src); } @Override @@ -71,7 +71,7 @@ public int compareTmp(int idx, int tmpIdx) { public void checkTmpLength(int len) { while (len > tmpLength) { tmpTimestamps.add((long[]) getPrimitiveArraysByType(TSDataType.INT64)); - tmpValues.add((long[]) getPrimitiveArraysByType(TSDataType.INT64)); + tmpIndices.add((int[]) getPrimitiveArraysByType(TSDataType.INT32)); tmpLength += ARRAY_SIZE; } } @@ -82,9 +82,9 @@ public void clearTmp() { PrimitiveArrayManager.release(dataArray); } tmpTimestamps.clear(); - for (long[] dataArray : tmpValues) { + for (int[] dataArray : tmpIndices) { PrimitiveArrayManager.release(dataArray); } - tmpValues.clear(); + tmpIndices.clear(); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BinaryTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BinaryTVList.java index bb8cd6e513cc9..0e112d7815976 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BinaryTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BinaryTVList.java @@ -36,6 +36,7 @@ import java.io.IOException; import java.util.ArrayList; import java.util.List; +import java.util.stream.IntStream; import static org.apache.iotdb.db.storageengine.rescon.memory.PrimitiveArrayManager.ARRAY_SIZE; import static org.apache.iotdb.db.storageengine.rescon.memory.PrimitiveArrayManager.TVLIST_SORT_ALGORITHM; @@ -71,6 +72,7 @@ public static BinaryTVList newList() { public TimBinaryTVList clone() { TimBinaryTVList cloneList = new TimBinaryTVList(); cloneAs(cloneList); + cloneSlicesAndBitMap(cloneList); cloneList.memoryBinaryChunkSize = memoryBinaryChunkSize; for (Binary[] valueArray : values) { cloneList.values.add(cloneValue(valueArray)); @@ -92,6 +94,7 @@ public void putBinary(long timestamp, Binary value) { maxTime = Math.max(maxTime, timestamp); timestamps.get(arrayIndex)[elementIndex] = timestamp; values.get(arrayIndex)[elementIndex] = value; + indices.get(arrayIndex)[elementIndex] = rowCount; rowCount++; if (sorted && rowCount > 1 && timestamp < getTime(rowCount - 2)) { sorted = false; @@ -104,54 +107,17 @@ public boolean reachChunkSizeOrPointNumThreshold() { return memoryBinaryChunkSize >= TARGET_CHUNK_SIZE || rowCount >= MAX_SERIES_POINT_NUMBER; } - @Override - public int delete(long lowerBound, long upperBound) { - int newSize = 0; - maxTime = Long.MIN_VALUE; - for (int i = 0; i < rowCount; i++) { - long time = getTime(i); - if (time < lowerBound || time > upperBound) { - set(i, newSize++); - maxTime = Math.max(maxTime, time); - } else { - memoryBinaryChunkSize -= getBinarySize(getBinary(i)); - } - } - int deletedNumber = rowCount - newSize; - rowCount = newSize; - // release primitive arrays that are empty - int newArrayNum = newSize / ARRAY_SIZE; - if (newSize % ARRAY_SIZE != 0) { - newArrayNum++; - } - int oldArrayNum = timestamps.size(); - for (int releaseIdx = newArrayNum; releaseIdx < oldArrayNum; releaseIdx++) { - releaseLastTimeArray(); - releaseLastValueArray(); - } - return deletedNumber; - } - @Override public Binary getBinary(int index) { if (index >= rowCount) { throw new ArrayIndexOutOfBoundsException(index); } - int arrayIndex = index / ARRAY_SIZE; - int elementIndex = index % ARRAY_SIZE; + int valueIndex = getValueIndex(index); + int arrayIndex = valueIndex / ARRAY_SIZE; + int elementIndex = valueIndex % ARRAY_SIZE; return values.get(arrayIndex)[elementIndex]; } - protected void set(int index, long timestamp, Binary value) { - if (index >= rowCount) { - throw new ArrayIndexOutOfBoundsException(index); - } - int arrayIndex = index / ARRAY_SIZE; - int elementIndex = index % ARRAY_SIZE; - timestamps.get(arrayIndex)[elementIndex] = timestamp; - values.get(arrayIndex)[elementIndex] = value; - } - @Override void clearValue() { if (values != null) { @@ -160,12 +126,14 @@ void clearValue() { } values.clear(); } + clearSlicesAndBitMap(); memoryBinaryChunkSize = 0; } @Override protected void expandValues() { values.add((Binary[]) getPrimitiveArraysByType(TSDataType.TEXT)); + expandSlicesAndBitMap(); } @Override @@ -188,7 +156,8 @@ protected void writeValidValuesIntoTsBlock( List deletionList) { int[] deleteCursor = {0}; for (int i = 0; i < rowCount; i++) { - if (!isPointDeleted(getTime(i), deletionList, deleteCursor) + if (!isNullValue(i) + && !isPointDeleted(getTime(i), deletionList, deleteCursor) && (i == rowCount - 1 || getTime(i) != getTime(i + 1))) { builder.getTimeColumnBuilder().writeLong(getTime(i)); builder.getColumnBuilder(0).writeBinary(getBinary(i)); @@ -238,6 +207,8 @@ public void putBinaries(long[] time, Binary[] value, BitMap bitMap, int start, i System.arraycopy( time, idx - timeIdxOffset, timestamps.get(arrayIdx), elementIdx, inputRemaining); System.arraycopy(value, idx, values.get(arrayIdx), elementIdx, inputRemaining); + int[] indexes = IntStream.range(rowCount, rowCount + inputRemaining).toArray(); + System.arraycopy(indexes, 0, indices.get(arrayIdx), elementIdx, inputRemaining); rowCount += inputRemaining; break; } else { @@ -246,6 +217,8 @@ public void putBinaries(long[] time, Binary[] value, BitMap bitMap, int start, i System.arraycopy( time, idx - timeIdxOffset, timestamps.get(arrayIdx), elementIdx, internalRemaining); System.arraycopy(value, idx, values.get(arrayIdx), elementIdx, internalRemaining); + int[] indexes = IntStream.range(rowCount, rowCount + internalRemaining).toArray(); + System.arraycopy(indexes, 0, indices.get(arrayIdx), elementIdx, internalRemaining); idx += internalRemaining; rowCount += internalRemaining; checkExpansion(); @@ -291,7 +264,7 @@ public TSDataType getDataType() { @Override public int serializedSize() { - int size = Byte.BYTES + Integer.BYTES + rowCount * Long.BYTES; + int size = Byte.BYTES + Integer.BYTES + rowCount * (Long.BYTES + Byte.BYTES); for (int rowIdx = 0; rowIdx < rowCount; ++rowIdx) { size += ReadWriteIOUtils.sizeToWrite(getBinary(rowIdx)); } @@ -304,7 +277,13 @@ public void serializeToWAL(IWALByteBufferView buffer) { buffer.putInt(rowCount); for (int rowIdx = 0; rowIdx < rowCount; ++rowIdx) { buffer.putLong(getTime(rowIdx)); - WALWriteUtils.write(getBinary(rowIdx), buffer); + Binary valueT = getBinary(rowIdx); + if (valueT != null) { + WALWriteUtils.write(getBinary(rowIdx), buffer); + } else { + WALWriteUtils.write(new Binary(new byte[0]), buffer); + } + WALWriteUtils.write(isNullValue(rowIdx), buffer); } } @@ -313,11 +292,15 @@ public static BinaryTVList deserialize(DataInputStream stream) throws IOExceptio int rowCount = stream.readInt(); long[] times = new long[rowCount]; Binary[] values = new Binary[rowCount]; + BitMap bitMap = new BitMap(rowCount); for (int rowIdx = 0; rowIdx < rowCount; ++rowIdx) { times[rowIdx] = stream.readLong(); values[rowIdx] = ReadWriteIOUtils.readBinary(stream); + if (ReadWriteIOUtils.readBool(stream)) { + bitMap.mark(rowIdx); + } } - tvList.putBinaries(times, values, null, 0, rowCount); + tvList.putBinaries(times, values, bitMap, 0, rowCount); return tvList; } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BooleanTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BooleanTVList.java index 00b0c3de8fd67..4673083c78ab8 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BooleanTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BooleanTVList.java @@ -35,6 +35,7 @@ import java.io.IOException; import java.util.ArrayList; import java.util.List; +import java.util.stream.IntStream; import static org.apache.iotdb.db.storageengine.rescon.memory.PrimitiveArrayManager.ARRAY_SIZE; import static org.apache.iotdb.db.storageengine.rescon.memory.PrimitiveArrayManager.TVLIST_SORT_ALGORITHM; @@ -65,6 +66,7 @@ public static BooleanTVList newList() { public BooleanTVList clone() { BooleanTVList cloneList = BooleanTVList.newList(); cloneAs(cloneList); + cloneSlicesAndBitMap(cloneList); for (boolean[] valueArray : values) { cloneList.values.add(cloneValue(valueArray)); } @@ -85,6 +87,7 @@ public void putBoolean(long timestamp, boolean value) { maxTime = Math.max(maxTime, timestamp); timestamps.get(arrayIndex)[elementIndex] = timestamp; values.get(arrayIndex)[elementIndex] = value; + indices.get(arrayIndex)[elementIndex] = rowCount; rowCount++; if (sorted && rowCount > 1 && timestamp < getTime(rowCount - 2)) { sorted = false; @@ -96,21 +99,12 @@ public boolean getBoolean(int index) { if (index >= rowCount) { throw new ArrayIndexOutOfBoundsException(index); } - int arrayIndex = index / ARRAY_SIZE; - int elementIndex = index % ARRAY_SIZE; + int valueIndex = getValueIndex(index); + int arrayIndex = valueIndex / ARRAY_SIZE; + int elementIndex = valueIndex % ARRAY_SIZE; return values.get(arrayIndex)[elementIndex]; } - protected void set(int index, long timestamp, boolean value) { - if (index >= rowCount) { - throw new ArrayIndexOutOfBoundsException(index); - } - int arrayIndex = index / ARRAY_SIZE; - int elementIndex = index % ARRAY_SIZE; - timestamps.get(arrayIndex)[elementIndex] = timestamp; - values.get(arrayIndex)[elementIndex] = value; - } - @Override void clearValue() { if (values != null) { @@ -119,11 +113,13 @@ void clearValue() { } values.clear(); } + clearSlicesAndBitMap(); } @Override protected void expandValues() { values.add((boolean[]) getPrimitiveArraysByType(TSDataType.BOOLEAN)); + expandSlicesAndBitMap(); } @Override @@ -147,7 +143,8 @@ protected void writeValidValuesIntoTsBlock( List deletionList) { int[] deleteCursor = {0}; for (int i = 0; i < rowCount; i++) { - if (!isPointDeleted(getTime(i), deletionList, deleteCursor) + if (!isNullValue(i) + && !isPointDeleted(getTime(i), deletionList, deleteCursor) && (i == rowCount - 1 || getTime(i) != getTime(i + 1))) { builder.getTimeColumnBuilder().writeLong(getTime(i)); builder.getColumnBuilder(0).writeBoolean(getBoolean(i)); @@ -192,6 +189,8 @@ public void putBooleans(long[] time, boolean[] value, BitMap bitMap, int start, System.arraycopy( time, idx - timeIdxOffset, timestamps.get(arrayIdx), elementIdx, inputRemaining); System.arraycopy(value, idx, values.get(arrayIdx), elementIdx, inputRemaining); + int[] indexes = IntStream.range(rowCount, rowCount + inputRemaining).toArray(); + System.arraycopy(indexes, 0, indices.get(arrayIdx), elementIdx, inputRemaining); rowCount += inputRemaining; break; } else { @@ -200,6 +199,8 @@ public void putBooleans(long[] time, boolean[] value, BitMap bitMap, int start, System.arraycopy( time, idx - timeIdxOffset, timestamps.get(arrayIdx), elementIdx, internalRemaining); System.arraycopy(value, idx, values.get(arrayIdx), elementIdx, internalRemaining); + int[] indexes = IntStream.range(rowCount, rowCount + internalRemaining).toArray(); + System.arraycopy(indexes, 0, indices.get(arrayIdx), elementIdx, internalRemaining); idx += internalRemaining; rowCount += internalRemaining; checkExpansion(); @@ -245,7 +246,7 @@ public TSDataType getDataType() { @Override public int serializedSize() { - return Byte.BYTES + Integer.BYTES + rowCount * (Long.BYTES + Byte.BYTES); + return Byte.BYTES + Integer.BYTES + rowCount * (Long.BYTES + Byte.BYTES + Byte.BYTES); } @Override @@ -255,6 +256,7 @@ public void serializeToWAL(IWALByteBufferView buffer) { for (int rowIdx = 0; rowIdx < rowCount; ++rowIdx) { buffer.putLong(getTime(rowIdx)); WALWriteUtils.write(getBoolean(rowIdx), buffer); + WALWriteUtils.write(isNullValue(rowIdx), buffer); } } @@ -263,11 +265,15 @@ public static BooleanTVList deserialize(DataInputStream stream) throws IOExcepti int rowCount = stream.readInt(); long[] times = new long[rowCount]; boolean[] values = new boolean[rowCount]; + BitMap bitMap = new BitMap(rowCount); for (int rowIdx = 0; rowIdx < rowCount; ++rowIdx) { times[rowIdx] = stream.readLong(); values[rowIdx] = ReadWriteIOUtils.readBool(stream); + if (ReadWriteIOUtils.readBool(stream)) { + bitMap.mark(rowIdx); + } } - tvList.putBooleans(times, values, null, 0, rowCount); + tvList.putBooleans(times, values, bitMap, 0, rowCount); return tvList; } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/DoubleTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/DoubleTVList.java index b7cc3336d10a0..e9dc4298a5550 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/DoubleTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/DoubleTVList.java @@ -29,12 +29,14 @@ import org.apache.tsfile.read.common.TimeRange; import org.apache.tsfile.read.common.block.TsBlockBuilder; import org.apache.tsfile.utils.BitMap; +import org.apache.tsfile.utils.ReadWriteIOUtils; import org.apache.tsfile.utils.TsPrimitiveType; import java.io.DataInputStream; import java.io.IOException; import java.util.ArrayList; import java.util.List; +import java.util.stream.IntStream; import static org.apache.iotdb.db.storageengine.rescon.memory.PrimitiveArrayManager.ARRAY_SIZE; import static org.apache.iotdb.db.storageengine.rescon.memory.PrimitiveArrayManager.TVLIST_SORT_ALGORITHM; @@ -65,6 +67,7 @@ public static DoubleTVList newList() { public DoubleTVList clone() { DoubleTVList cloneList = DoubleTVList.newList(); cloneAs(cloneList); + cloneSlicesAndBitMap(cloneList); for (double[] valueArray : values) { cloneList.values.add(cloneValue(valueArray)); } @@ -85,6 +88,7 @@ public void putDouble(long timestamp, double value) { maxTime = Math.max(maxTime, timestamp); timestamps.get(arrayIndex)[elementIndex] = timestamp; values.get(arrayIndex)[elementIndex] = value; + indices.get(arrayIndex)[elementIndex] = rowCount; rowCount++; if (sorted && rowCount > 1 && timestamp < getTime(rowCount - 2)) { sorted = false; @@ -96,21 +100,12 @@ public double getDouble(int index) { if (index >= rowCount) { throw new ArrayIndexOutOfBoundsException(index); } - int arrayIndex = index / ARRAY_SIZE; - int elementIndex = index % ARRAY_SIZE; + int valueIndex = getValueIndex(index); + int arrayIndex = valueIndex / ARRAY_SIZE; + int elementIndex = valueIndex % ARRAY_SIZE; return values.get(arrayIndex)[elementIndex]; } - protected void set(int index, long timestamp, double value) { - if (index >= rowCount) { - throw new ArrayIndexOutOfBoundsException(index); - } - int arrayIndex = index / ARRAY_SIZE; - int elementIndex = index % ARRAY_SIZE; - timestamps.get(arrayIndex)[elementIndex] = timestamp; - values.get(arrayIndex)[elementIndex] = value; - } - @Override void clearValue() { if (values != null) { @@ -119,11 +114,13 @@ void clearValue() { } values.clear(); } + clearSlicesAndBitMap(); } @Override protected void expandValues() { values.add((double[]) getPrimitiveArraysByType(TSDataType.DOUBLE)); + expandSlicesAndBitMap(); } @Override @@ -150,7 +147,8 @@ protected void writeValidValuesIntoTsBlock( List deletionList) { int[] deleteCursor = {0}; for (int i = 0; i < rowCount; i++) { - if (!isPointDeleted(getTime(i), deletionList, deleteCursor) + if (!isNullValue(i) + && !isPointDeleted(getTime(i), deletionList, deleteCursor) && (i == rowCount - 1 || getTime(i) != getTime(i + 1))) { builder.getTimeColumnBuilder().writeLong(getTime(i)); builder @@ -197,6 +195,8 @@ public void putDoubles(long[] time, double[] value, BitMap bitMap, int start, in System.arraycopy( time, idx - timeIdxOffset, timestamps.get(arrayIdx), elementIdx, inputRemaining); System.arraycopy(value, idx, values.get(arrayIdx), elementIdx, inputRemaining); + int[] indexes = IntStream.range(rowCount, rowCount + inputRemaining).toArray(); + System.arraycopy(indexes, 0, indices.get(arrayIdx), elementIdx, inputRemaining); rowCount += inputRemaining; break; } else { @@ -205,6 +205,8 @@ public void putDoubles(long[] time, double[] value, BitMap bitMap, int start, in System.arraycopy( time, idx - timeIdxOffset, timestamps.get(arrayIdx), elementIdx, internalRemaining); System.arraycopy(value, idx, values.get(arrayIdx), elementIdx, internalRemaining); + int[] indexes = IntStream.range(rowCount, rowCount + internalRemaining).toArray(); + System.arraycopy(indexes, 0, indices.get(arrayIdx), elementIdx, internalRemaining); idx += internalRemaining; rowCount += internalRemaining; checkExpansion(); @@ -250,7 +252,7 @@ public TSDataType getDataType() { @Override public int serializedSize() { - return Byte.BYTES + Integer.BYTES + rowCount * (Long.BYTES + Double.BYTES); + return Byte.BYTES + Integer.BYTES + rowCount * (Long.BYTES + Double.BYTES + Byte.BYTES); } @Override @@ -260,6 +262,7 @@ public void serializeToWAL(IWALByteBufferView buffer) { for (int rowIdx = 0; rowIdx < rowCount; ++rowIdx) { buffer.putLong(getTime(rowIdx)); buffer.putDouble(getDouble(rowIdx)); + WALWriteUtils.write(isNullValue(rowIdx), buffer); } } @@ -268,11 +271,15 @@ public static DoubleTVList deserialize(DataInputStream stream) throws IOExceptio int rowCount = stream.readInt(); long[] times = new long[rowCount]; double[] values = new double[rowCount]; + BitMap bitMap = new BitMap(rowCount); for (int rowIdx = 0; rowIdx < rowCount; ++rowIdx) { times[rowIdx] = stream.readLong(); values[rowIdx] = stream.readDouble(); + if (ReadWriteIOUtils.readBool(stream)) { + bitMap.mark(rowIdx); + } } - tvList.putDoubles(times, values, null, 0, rowCount); + tvList.putDoubles(times, values, bitMap, 0, rowCount); return tvList; } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/FloatTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/FloatTVList.java index 43a208ecccfd8..1165aec71ed84 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/FloatTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/FloatTVList.java @@ -29,12 +29,14 @@ import org.apache.tsfile.read.common.TimeRange; import org.apache.tsfile.read.common.block.TsBlockBuilder; import org.apache.tsfile.utils.BitMap; +import org.apache.tsfile.utils.ReadWriteIOUtils; import org.apache.tsfile.utils.TsPrimitiveType; import java.io.DataInputStream; import java.io.IOException; import java.util.ArrayList; import java.util.List; +import java.util.stream.IntStream; import static org.apache.iotdb.db.storageengine.rescon.memory.PrimitiveArrayManager.ARRAY_SIZE; import static org.apache.iotdb.db.storageengine.rescon.memory.PrimitiveArrayManager.TVLIST_SORT_ALGORITHM; @@ -65,6 +67,7 @@ public static FloatTVList newList() { public FloatTVList clone() { FloatTVList cloneList = FloatTVList.newList(); cloneAs(cloneList); + cloneSlicesAndBitMap(cloneList); for (float[] valueArray : values) { cloneList.values.add(cloneValue(valueArray)); } @@ -85,6 +88,7 @@ public void putFloat(long timestamp, float value) { maxTime = Math.max(maxTime, timestamp); timestamps.get(arrayIndex)[elementIndex] = timestamp; values.get(arrayIndex)[elementIndex] = value; + indices.get(arrayIndex)[elementIndex] = rowCount; rowCount++; if (sorted && rowCount > 1 && timestamp < getTime(rowCount - 2)) { sorted = false; @@ -96,21 +100,12 @@ public float getFloat(int index) { if (index >= rowCount) { throw new ArrayIndexOutOfBoundsException(index); } - int arrayIndex = index / ARRAY_SIZE; - int elementIndex = index % ARRAY_SIZE; + int valueIndex = getValueIndex(index); + int arrayIndex = valueIndex / ARRAY_SIZE; + int elementIndex = valueIndex % ARRAY_SIZE; return values.get(arrayIndex)[elementIndex]; } - protected void set(int index, long timestamp, float value) { - if (index >= rowCount) { - throw new ArrayIndexOutOfBoundsException(index); - } - int arrayIndex = index / ARRAY_SIZE; - int elementIndex = index % ARRAY_SIZE; - timestamps.get(arrayIndex)[elementIndex] = timestamp; - values.get(arrayIndex)[elementIndex] = value; - } - @Override void clearValue() { if (values != null) { @@ -119,11 +114,13 @@ void clearValue() { } values.clear(); } + clearSlicesAndBitMap(); } @Override protected void expandValues() { values.add((float[]) getPrimitiveArraysByType(TSDataType.FLOAT)); + expandSlicesAndBitMap(); } @Override @@ -150,7 +147,8 @@ protected void writeValidValuesIntoTsBlock( List deletionList) { int[] deleteCursor = {0}; for (int i = 0; i < rowCount; i++) { - if (!isPointDeleted(getTime(i), deletionList, deleteCursor) + if (!isNullValue(i) + && !isPointDeleted(getTime(i), deletionList, deleteCursor) && (i == rowCount - 1 || getTime(i) != getTime(i + 1))) { builder.getTimeColumnBuilder().writeLong(getTime(i)); builder @@ -197,6 +195,8 @@ public void putFloats(long[] time, float[] value, BitMap bitMap, int start, int System.arraycopy( time, idx - timeIdxOffset, timestamps.get(arrayIdx), elementIdx, inputRemaining); System.arraycopy(value, idx, values.get(arrayIdx), elementIdx, inputRemaining); + int[] indexes = IntStream.range(rowCount, rowCount + inputRemaining).toArray(); + System.arraycopy(indexes, 0, indices.get(arrayIdx), elementIdx, inputRemaining); rowCount += inputRemaining; break; } else { @@ -205,6 +205,8 @@ public void putFloats(long[] time, float[] value, BitMap bitMap, int start, int System.arraycopy( time, idx - timeIdxOffset, timestamps.get(arrayIdx), elementIdx, internalRemaining); System.arraycopy(value, idx, values.get(arrayIdx), elementIdx, internalRemaining); + int[] indexes = IntStream.range(rowCount, rowCount + internalRemaining).toArray(); + System.arraycopy(indexes, 0, indices.get(arrayIdx), elementIdx, internalRemaining); idx += internalRemaining; rowCount += internalRemaining; checkExpansion(); @@ -250,7 +252,7 @@ public TSDataType getDataType() { @Override public int serializedSize() { - return Byte.BYTES + Integer.BYTES + rowCount * (Long.BYTES + Float.BYTES); + return Byte.BYTES + Integer.BYTES + rowCount * (Long.BYTES + Float.BYTES + Byte.BYTES); } @Override @@ -260,6 +262,7 @@ public void serializeToWAL(IWALByteBufferView buffer) { for (int rowIdx = 0; rowIdx < rowCount; ++rowIdx) { buffer.putLong(getTime(rowIdx)); buffer.putFloat(getFloat(rowIdx)); + WALWriteUtils.write(isNullValue(rowIdx), buffer); } } @@ -268,11 +271,15 @@ public static FloatTVList deserialize(DataInputStream stream) throws IOException int rowCount = stream.readInt(); long[] times = new long[rowCount]; float[] values = new float[rowCount]; + BitMap bitMap = new BitMap(rowCount); for (int rowIdx = 0; rowIdx < rowCount; ++rowIdx) { times[rowIdx] = stream.readLong(); values[rowIdx] = stream.readFloat(); + if (ReadWriteIOUtils.readBool(stream)) { + bitMap.mark(rowIdx); + } } - tvList.putFloats(times, values, null, 0, rowCount); + tvList.putFloats(times, values, bitMap, 0, rowCount); return tvList; } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/IntTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/IntTVList.java index 162f1fd1112b5..3af718817d449 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/IntTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/IntTVList.java @@ -28,12 +28,14 @@ import org.apache.tsfile.read.common.TimeRange; import org.apache.tsfile.read.common.block.TsBlockBuilder; import org.apache.tsfile.utils.BitMap; +import org.apache.tsfile.utils.ReadWriteIOUtils; import org.apache.tsfile.utils.TsPrimitiveType; import java.io.DataInputStream; import java.io.IOException; import java.util.ArrayList; import java.util.List; +import java.util.stream.IntStream; import static org.apache.iotdb.db.storageengine.rescon.memory.PrimitiveArrayManager.ARRAY_SIZE; import static org.apache.iotdb.db.storageengine.rescon.memory.PrimitiveArrayManager.TVLIST_SORT_ALGORITHM; @@ -64,6 +66,7 @@ public static IntTVList newList() { public IntTVList clone() { IntTVList cloneList = IntTVList.newList(); cloneAs(cloneList); + cloneSlicesAndBitMap(cloneList); for (int[] valueArray : values) { cloneList.values.add(cloneValue(valueArray)); } @@ -84,6 +87,7 @@ public void putInt(long timestamp, int value) { maxTime = Math.max(maxTime, timestamp); timestamps.get(arrayIndex)[elementIndex] = timestamp; values.get(arrayIndex)[elementIndex] = value; + indices.get(arrayIndex)[elementIndex] = rowCount; rowCount++; if (sorted && rowCount > 1 && timestamp < getTime(rowCount - 2)) { sorted = false; @@ -95,21 +99,12 @@ public int getInt(int index) { if (index >= rowCount) { throw new ArrayIndexOutOfBoundsException(index); } - int arrayIndex = index / ARRAY_SIZE; - int elementIndex = index % ARRAY_SIZE; + int valueIndex = getValueIndex(index); + int arrayIndex = valueIndex / ARRAY_SIZE; + int elementIndex = valueIndex % ARRAY_SIZE; return values.get(arrayIndex)[elementIndex]; } - protected void set(int index, long timestamp, int value) { - if (index >= rowCount) { - throw new ArrayIndexOutOfBoundsException(index); - } - int arrayIndex = index / ARRAY_SIZE; - int elementIndex = index % ARRAY_SIZE; - timestamps.get(arrayIndex)[elementIndex] = timestamp; - values.get(arrayIndex)[elementIndex] = value; - } - @Override void clearValue() { if (values != null) { @@ -118,11 +113,13 @@ void clearValue() { } values.clear(); } + clearSlicesAndBitMap(); } @Override protected void expandValues() { values.add((int[]) getPrimitiveArraysByType(TSDataType.INT32)); + expandSlicesAndBitMap(); } @Override @@ -145,7 +142,8 @@ protected void writeValidValuesIntoTsBlock( List deletionList) { int[] deleteCursor = {0}; for (int i = 0; i < rowCount; i++) { - if (!isPointDeleted(getTime(i), deletionList, deleteCursor) + if (!isNullValue(i) + && !isPointDeleted(getTime(i), deletionList, deleteCursor) && (i == rowCount - 1 || getTime(i) != getTime(i + 1))) { builder.getTimeColumnBuilder().writeLong(getTime(i)); builder.getColumnBuilder(0).writeInt(getInt(i)); @@ -190,6 +188,8 @@ public void putInts(long[] time, int[] value, BitMap bitMap, int start, int end) System.arraycopy( time, idx - timeIdxOffset, timestamps.get(arrayIdx), elementIdx, inputRemaining); System.arraycopy(value, idx, values.get(arrayIdx), elementIdx, inputRemaining); + int[] indexes = IntStream.range(rowCount, rowCount + inputRemaining).toArray(); + System.arraycopy(indexes, 0, indices.get(arrayIdx), elementIdx, inputRemaining); rowCount += inputRemaining; break; } else { @@ -198,6 +198,8 @@ public void putInts(long[] time, int[] value, BitMap bitMap, int start, int end) System.arraycopy( time, idx - timeIdxOffset, timestamps.get(arrayIdx), elementIdx, internalRemaining); System.arraycopy(value, idx, values.get(arrayIdx), elementIdx, internalRemaining); + int[] indexes = IntStream.range(rowCount, rowCount + internalRemaining).toArray(); + System.arraycopy(indexes, 0, indices.get(arrayIdx), elementIdx, internalRemaining); idx += internalRemaining; rowCount += internalRemaining; checkExpansion(); @@ -243,7 +245,7 @@ public TSDataType getDataType() { @Override public int serializedSize() { - return Byte.BYTES + Integer.BYTES + rowCount * (Long.BYTES + Integer.BYTES); + return Byte.BYTES + Integer.BYTES + rowCount * (Long.BYTES + Integer.BYTES + Byte.BYTES); } @Override @@ -253,6 +255,7 @@ public void serializeToWAL(IWALByteBufferView buffer) { for (int rowIdx = 0; rowIdx < rowCount; ++rowIdx) { buffer.putLong(getTime(rowIdx)); buffer.putInt(getInt(rowIdx)); + WALWriteUtils.write(isNullValue(rowIdx), buffer); } } @@ -261,11 +264,15 @@ public static IntTVList deserialize(DataInputStream stream) throws IOException { int rowCount = stream.readInt(); long[] times = new long[rowCount]; int[] values = new int[rowCount]; + BitMap bitMap = new BitMap(rowCount); for (int rowIdx = 0; rowIdx < rowCount; ++rowIdx) { times[rowIdx] = stream.readLong(); values[rowIdx] = stream.readInt(); + if (ReadWriteIOUtils.readBool(stream)) { + bitMap.mark(rowIdx); + } } - tvList.putInts(times, values, null, 0, rowCount); + tvList.putInts(times, values, bitMap, 0, rowCount); return tvList; } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/LongTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/LongTVList.java index 4c8f4ccea9e7b..557acdcb95480 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/LongTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/LongTVList.java @@ -28,12 +28,14 @@ import org.apache.tsfile.read.common.TimeRange; import org.apache.tsfile.read.common.block.TsBlockBuilder; import org.apache.tsfile.utils.BitMap; +import org.apache.tsfile.utils.ReadWriteIOUtils; import org.apache.tsfile.utils.TsPrimitiveType; import java.io.DataInputStream; import java.io.IOException; import java.util.ArrayList; import java.util.List; +import java.util.stream.IntStream; import static org.apache.iotdb.db.storageengine.rescon.memory.PrimitiveArrayManager.ARRAY_SIZE; import static org.apache.iotdb.db.storageengine.rescon.memory.PrimitiveArrayManager.TVLIST_SORT_ALGORITHM; @@ -64,6 +66,7 @@ public static LongTVList newList() { public LongTVList clone() { LongTVList cloneList = LongTVList.newList(); cloneAs(cloneList); + cloneSlicesAndBitMap(cloneList); for (long[] valueArray : values) { cloneList.values.add(cloneValue(valueArray)); } @@ -84,6 +87,7 @@ public void putLong(long timestamp, long value) { maxTime = Math.max(maxTime, timestamp); timestamps.get(arrayIndex)[elementIndex] = timestamp; values.get(arrayIndex)[elementIndex] = value; + indices.get(arrayIndex)[elementIndex] = rowCount; rowCount++; if (sorted && rowCount > 1 && timestamp < getTime(rowCount - 2)) { sorted = false; @@ -95,21 +99,12 @@ public long getLong(int index) { if (index >= rowCount) { throw new ArrayIndexOutOfBoundsException(index); } - int arrayIndex = index / ARRAY_SIZE; - int elementIndex = index % ARRAY_SIZE; + int valueIndex = getValueIndex(index); + int arrayIndex = valueIndex / ARRAY_SIZE; + int elementIndex = valueIndex % ARRAY_SIZE; return values.get(arrayIndex)[elementIndex]; } - protected void set(int index, long timestamp, long value) { - if (index >= rowCount) { - throw new ArrayIndexOutOfBoundsException(index); - } - int arrayIndex = index / ARRAY_SIZE; - int elementIndex = index % ARRAY_SIZE; - timestamps.get(arrayIndex)[elementIndex] = timestamp; - values.get(arrayIndex)[elementIndex] = value; - } - @Override void clearValue() { if (values != null) { @@ -118,11 +113,13 @@ void clearValue() { } values.clear(); } + clearSlicesAndBitMap(); } @Override protected void expandValues() { values.add((long[]) getPrimitiveArraysByType(TSDataType.INT64)); + expandSlicesAndBitMap(); } @Override @@ -145,7 +142,8 @@ protected void writeValidValuesIntoTsBlock( List deletionList) { int[] deleteCursor = {0}; for (int i = 0; i < rowCount; i++) { - if (!isPointDeleted(getTime(i), deletionList, deleteCursor) + if (!isNullValue(i) + && !isPointDeleted(getTime(i), deletionList, deleteCursor) && (i == rowCount - 1 || getTime(i) != getTime(i + 1))) { builder.getTimeColumnBuilder().writeLong(getTime(i)); builder.getColumnBuilder(0).writeLong(getLong(i)); @@ -190,6 +188,8 @@ public void putLongs(long[] time, long[] value, BitMap bitMap, int start, int en System.arraycopy( time, idx - timeIdxOffset, timestamps.get(arrayIdx), elementIdx, inputRemaining); System.arraycopy(value, idx, values.get(arrayIdx), elementIdx, inputRemaining); + int[] indexes = IntStream.range(rowCount, rowCount + inputRemaining).toArray(); + System.arraycopy(indexes, 0, indices.get(arrayIdx), elementIdx, inputRemaining); rowCount += inputRemaining; break; } else { @@ -198,6 +198,8 @@ public void putLongs(long[] time, long[] value, BitMap bitMap, int start, int en System.arraycopy( time, idx - timeIdxOffset, timestamps.get(arrayIdx), elementIdx, internalRemaining); System.arraycopy(value, idx, values.get(arrayIdx), elementIdx, internalRemaining); + int[] indexes = IntStream.range(rowCount, rowCount + internalRemaining).toArray(); + System.arraycopy(indexes, 0, indices.get(arrayIdx), elementIdx, internalRemaining); idx += internalRemaining; rowCount += internalRemaining; checkExpansion(); @@ -243,7 +245,7 @@ public TSDataType getDataType() { @Override public int serializedSize() { - return Byte.BYTES + Integer.BYTES + rowCount * 2 * Long.BYTES; + return Byte.BYTES + Integer.BYTES + rowCount * (2 * Long.BYTES + Byte.BYTES); } @Override @@ -253,6 +255,7 @@ public void serializeToWAL(IWALByteBufferView buffer) { for (int rowIdx = 0; rowIdx < rowCount; ++rowIdx) { buffer.putLong(getTime(rowIdx)); buffer.putLong(getLong(rowIdx)); + WALWriteUtils.write(isNullValue(rowIdx), buffer); } } @@ -261,11 +264,15 @@ public static LongTVList deserialize(DataInputStream stream) throws IOException int rowCount = stream.readInt(); long[] times = new long[rowCount]; long[] values = new long[rowCount]; + BitMap bitMap = new BitMap(rowCount); for (int rowIdx = 0; rowIdx < rowCount; ++rowIdx) { times[rowIdx] = stream.readLong(); values[rowIdx] = stream.readLong(); + if (ReadWriteIOUtils.readBool(stream)) { + bitMap.mark(rowIdx); + } } - tvList.putLongs(times, values, null, 0, rowCount); + tvList.putLongs(times, values, bitMap, 0, rowCount); return tvList; } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/QuickBinaryTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/QuickBinaryTVList.java index d0a228495ee32..7a524a1489880 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/QuickBinaryTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/QuickBinaryTVList.java @@ -18,8 +18,6 @@ */ package org.apache.iotdb.db.utils.datastructure; -import org.apache.tsfile.utils.Binary; - public class QuickBinaryTVList extends BinaryTVList implements QuickSort { @Override @@ -31,9 +29,9 @@ public int compare(int idx1, int idx2) { @Override public void swap(int p, int q) { - Binary valueP = getBinary(p); + int valueP = getValueIndex(p); long timeP = getTime(p); - Binary valueQ = getBinary(q); + int valueQ = getValueIndex(q); long timeQ = getTime(q); set(p, timeQ, valueQ); set(q, timeP, valueP); @@ -50,7 +48,7 @@ public void sort() { @Override protected void set(int src, int dest) { long srcT = getTime(src); - Binary srcV = getBinary(src); + int srcV = getValueIndex(src); set(dest, srcT, srcV); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/QuickBooleanTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/QuickBooleanTVList.java index e4fbfaefca25d..cbbf332fa996f 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/QuickBooleanTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/QuickBooleanTVList.java @@ -28,9 +28,9 @@ public int compare(int idx1, int idx2) { @Override public void swap(int p, int q) { - boolean valueP = getBoolean(p); + int valueP = getValueIndex(p); long timeP = getTime(p); - boolean valueQ = getBoolean(q); + int valueQ = getValueIndex(q); long timeQ = getTime(q); set(p, timeQ, valueQ); set(q, timeP, valueP); @@ -47,7 +47,7 @@ public void sort() { @Override protected void set(int src, int dest) { long srcT = getTime(src); - boolean srcV = getBoolean(src); + int srcV = getValueIndex(src); set(dest, srcT, srcV); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/QuickDoubleTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/QuickDoubleTVList.java index 852844f923164..2356d68f8a360 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/QuickDoubleTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/QuickDoubleTVList.java @@ -28,9 +28,9 @@ public int compare(int idx1, int idx2) { @Override public void swap(int p, int q) { - double valueP = getDouble(p); + int valueP = getValueIndex(p); long timeP = getTime(p); - double valueQ = getDouble(q); + int valueQ = getValueIndex(q); long timeQ = getTime(q); set(p, timeQ, valueQ); set(q, timeP, valueP); @@ -47,7 +47,7 @@ public void sort() { @Override protected void set(int src, int dest) { long srcT = getTime(src); - double srcV = getDouble(src); + int srcV = getValueIndex(src); set(dest, srcT, srcV); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/QuickFloatTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/QuickFloatTVList.java index 409a00a093ea2..a67003664bb91 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/QuickFloatTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/QuickFloatTVList.java @@ -28,9 +28,9 @@ public int compare(int idx1, int idx2) { @Override public void swap(int p, int q) { - float valueP = getFloat(p); + int valueP = getValueIndex(p); long timeP = getTime(p); - float valueQ = getFloat(q); + int valueQ = getValueIndex(q); long timeQ = getTime(q); set(p, timeQ, valueQ); set(q, timeP, valueP); @@ -47,7 +47,7 @@ public void sort() { @Override protected void set(int src, int dest) { long srcT = getTime(src); - float srcV = getFloat(src); + int srcV = getValueIndex(src); set(dest, srcT, srcV); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/QuickIntTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/QuickIntTVList.java index bc44cf7905357..c898fe99dfa15 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/QuickIntTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/QuickIntTVList.java @@ -29,9 +29,9 @@ public void sort() { @Override public void swap(int p, int q) { - int valueP = getInt(p); + int valueP = getValueIndex(p); long timeP = getTime(p); - int valueQ = getInt(q); + int valueQ = getValueIndex(q); long timeQ = getTime(q); set(p, timeQ, valueQ); set(q, timeP, valueP); @@ -40,7 +40,7 @@ public void swap(int p, int q) { @Override protected void set(int src, int dest) { long srcT = getTime(src); - int srcV = getInt(src); + int srcV = getValueIndex(src); set(dest, srcT, srcV); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/QuickLongTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/QuickLongTVList.java index 16d8fab360670..423ce2bf19dae 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/QuickLongTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/QuickLongTVList.java @@ -28,9 +28,9 @@ public int compare(int idx1, int idx2) { @Override public void swap(int p, int q) { - long valueP = getLong(p); + int valueP = getValueIndex(p); long timeP = getTime(p); - long valueQ = getLong(q); + int valueQ = getValueIndex(q); long timeQ = getTime(q); set(p, timeQ, valueQ); set(q, timeP, valueP); @@ -47,7 +47,7 @@ public void sort() { @Override protected void set(int src, int dest) { long srcT = getTime(src); - long srcV = getLong(src); + int srcV = getValueIndex(src); set(dest, srcT, srcV); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java index d75fc857c26cb..4a2d7dde2f3e5 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java @@ -58,6 +58,15 @@ public abstract class TVList implements WALEntryValue { protected List timestamps; protected int rowCount; + // List of index array, add 1 when expanded -> data point index array + // Index relation: arrayIndex -> elementIndex + // Used in sort method, sort only changes indices + protected List indices; + + // used by non-aligned TVList + // Index relation: arrayIndex -> elementIndex + protected List bitMap; + protected boolean sorted = true; protected long maxTime; // record reference count of this tv list @@ -67,6 +76,7 @@ public abstract class TVList implements WALEntryValue { protected TVList() { timestamps = new ArrayList<>(); + indices = new ArrayList<>(); rowCount = 0; maxTime = Long.MIN_VALUE; referenceCount = new AtomicInteger(); @@ -136,6 +146,105 @@ public long getTime(int index) { return timestamps.get(arrayIndex)[elementIndex]; } + protected void set(int index, long timestamp, int value) { + if (index >= rowCount) { + throw new ArrayIndexOutOfBoundsException(index); + } + int arrayIndex = index / ARRAY_SIZE; + int elementIndex = index % ARRAY_SIZE; + timestamps.get(arrayIndex)[elementIndex] = timestamp; + indices.get(arrayIndex)[elementIndex] = value; + } + + @SuppressWarnings("squid:S3776") // Suppress high Cognitive Complexity warning + protected int[] cloneIndex(int[] array) { + int[] cloneArray = new int[array.length]; + System.arraycopy(array, 0, cloneArray, 0, array.length); + return cloneArray; + } + + /** + * Get the row index value in index column. + * + * @param index row index + */ + public int getValueIndex(int index) { + if (index >= rowCount) { + throw new ArrayIndexOutOfBoundsException(index); + } + int arrayIndex = index / ARRAY_SIZE; + int elementIndex = index % ARRAY_SIZE; + return indices.get(arrayIndex)[elementIndex]; + } + + protected void markNullValue(int arrayIndex, int elementIndex) { + // init bitMap if doesn't have + if (bitMap == null) { + bitMap = new ArrayList<>(); + for (int i = 0; i < timestamps.size(); i++) { + bitMap.add(new BitMap(ARRAY_SIZE)); + } + } + // if the bitmap in arrayIndex is null, init the bitmap + if (bitMap.get(arrayIndex) == null) { + bitMap.set(arrayIndex, new BitMap(ARRAY_SIZE)); + } + + // mark the null value in the current bitmap + bitMap.get(arrayIndex).mark(elementIndex); + } + + /** + * Get whether value is null at the given position in TvList. + * + * @param rowIndex value index + * @return boolean + */ + protected boolean isNullValue(int rowIndex) { + if (rowIndex >= rowCount) { + return false; + } + if (bitMap == null || bitMap.get(rowIndex / ARRAY_SIZE) == null) { + return false; + } + int arrayIndex = rowIndex / ARRAY_SIZE; + int elementIndex = rowIndex % ARRAY_SIZE; + return bitMap.get(arrayIndex).isMarked(elementIndex); + } + + protected void cloneSlicesAndBitMap(TVList cloneList) { + if (indices != null) { + for (int[] indicesArray : indices) { + cloneList.indices.add(cloneIndex(indicesArray)); + } + } + if (bitMap != null) { + cloneList.bitMap = new ArrayList<>(); + for (BitMap bm : bitMap) { + cloneList.bitMap.add(bm == null ? null : bm.clone()); + } + } + } + + protected void clearSlicesAndBitMap() { + if (indices != null) { + for (int[] dataArray : indices) { + PrimitiveArrayManager.release(dataArray); + } + indices.clear(); + } + if (bitMap != null) { + bitMap.clear(); + } + } + + protected void expandSlicesAndBitMap() { + indices.add((int[]) getPrimitiveArraysByType(TSDataType.INT32)); + if (bitMap != null) { + bitMap.add(null); + } + } + public void putLong(long time, long value) { throw new UnsupportedOperationException(ERR_DATATYPE_NOT_CONSISTENT); } @@ -230,10 +339,6 @@ public TVList getTvListByColumnIndex( throw new UnsupportedOperationException(ERR_DATATYPE_NOT_CONSISTENT); } - public int getValueIndex(int index) { - throw new UnsupportedOperationException(ERR_DATATYPE_NOT_CONSISTENT); - } - public long getMaxTime() { return maxTime; } @@ -261,27 +366,20 @@ protected void releaseLastTimeArray() { } public int delete(long lowerBound, long upperBound) { - int newSize = 0; - maxTime = Long.MIN_VALUE; + int deletedNumber = 0; + long maxTime = Long.MIN_VALUE; for (int i = 0; i < rowCount; i++) { long time = getTime(i); - if (time < lowerBound || time > upperBound) { - set(i, newSize++); + if (time >= lowerBound && time <= upperBound) { + int originRowIndex = getValueIndex(i); + int arrayIndex = originRowIndex / ARRAY_SIZE; + int elementIndex = originRowIndex % ARRAY_SIZE; + markNullValue(arrayIndex, elementIndex); + deletedNumber++; + } else { maxTime = Math.max(time, maxTime); } } - int deletedNumber = rowCount - newSize; - rowCount = newSize; - // release primitive arrays that are empty - int newArrayNum = newSize / ARRAY_SIZE; - if (newSize % ARRAY_SIZE != 0) { - newArrayNum++; - } - int oldArrayNum = timestamps.size(); - for (int releaseIdx = newArrayNum; releaseIdx < oldArrayNum; releaseIdx++) { - releaseLastTimeArray(); - releaseLastValueArray(); - } return deletedNumber; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TimBinaryTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TimBinaryTVList.java index 82575c7b838b7..f0a2b5c2c45d8 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TimBinaryTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TimBinaryTVList.java @@ -21,17 +21,15 @@ import org.apache.iotdb.db.storageengine.rescon.memory.PrimitiveArrayManager; import org.apache.tsfile.enums.TSDataType; -import org.apache.tsfile.utils.Binary; import static org.apache.iotdb.db.storageengine.rescon.memory.PrimitiveArrayManager.ARRAY_SIZE; public class TimBinaryTVList extends BinaryTVList implements TimSort { - private long[][] sortedTimestamps; - private long pivotTime; + private int[][] sortedIndices; - private Binary[][] sortedValues; - private Binary pivotValue; + private long pivotTime; + private int pivotIndex; @Override public void sort() { @@ -40,10 +38,10 @@ public void sort() { sortedTimestamps = (long[][]) PrimitiveArrayManager.createDataListsByType(TSDataType.INT64, rowCount); } - if (sortedValues == null - || sortedValues.length < PrimitiveArrayManager.getArrayRowCount(rowCount)) { - sortedValues = - (Binary[][]) PrimitiveArrayManager.createDataListsByType(TSDataType.TEXT, rowCount); + if (sortedIndices == null + || sortedIndices.length < PrimitiveArrayManager.getArrayRowCount(rowCount)) { + sortedIndices = + (int[][]) PrimitiveArrayManager.createDataListsByType(TSDataType.INT32, rowCount); } sort(0, rowCount); clearSortedValue(); @@ -59,20 +57,20 @@ public void tim_set(int src, int dest) { @Override public void set(int src, int dest) { long srcT = getTime(src); - Binary srcV = getBinary(src); + int srcV = getValueIndex(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] = getBinary(src); + sortedIndices[dest / ARRAY_SIZE][dest % ARRAY_SIZE] = getValueIndex(src); } @Override public void saveAsPivot(int pos) { pivotTime = getTime(pos); - pivotValue = getBinary(pos); + pivotIndex = getValueIndex(pos); } @Override @@ -80,12 +78,12 @@ public void setFromSorted(int src, int dest) { set( dest, sortedTimestamps[src / ARRAY_SIZE][src % ARRAY_SIZE], - sortedValues[src / ARRAY_SIZE][src % ARRAY_SIZE]); + sortedIndices[src / ARRAY_SIZE][src % ARRAY_SIZE]); } @Override public void setPivotTo(int pos) { - set(pos, pivotTime, pivotValue); + set(pos, pivotTime, pivotIndex); } @Override @@ -97,8 +95,8 @@ public void clearSortedTime() { @Override public void clearSortedValue() { - if (sortedValues != null) { - sortedValues = null; + if (sortedIndices != null) { + sortedIndices = null; } } @@ -114,9 +112,9 @@ public void reverseRange(int lo, int hi) { hi--; while (lo < hi) { long loT = getTime(lo); - Binary loV = getBinary(lo); + int loV = getValueIndex(lo); long hiT = getTime(hi); - Binary hiV = getBinary(hi); + int hiV = getValueIndex(hi); set(lo++, hiT, hiV); set(hi--, loT, loV); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TimBooleanTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TimBooleanTVList.java index e571b818ea379..39de1435f04f7 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TimBooleanTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TimBooleanTVList.java @@ -26,10 +26,10 @@ public class TimBooleanTVList extends BooleanTVList implements TimSort { private long[][] sortedTimestamps; - private long pivotTime; + private int[][] sortedIndices; - private boolean[][] sortedValues; - private boolean pivotValue; + private long pivotTime; + private int pivotIndex; @Override public void sort() { @@ -38,10 +38,10 @@ public void sort() { sortedTimestamps = (long[][]) PrimitiveArrayManager.createDataListsByType(TSDataType.INT64, rowCount); } - if (sortedValues == null - || sortedValues.length < PrimitiveArrayManager.getArrayRowCount(rowCount)) { - sortedValues = - (boolean[][]) PrimitiveArrayManager.createDataListsByType(TSDataType.BOOLEAN, rowCount); + if (sortedIndices == null + || sortedIndices.length < PrimitiveArrayManager.getArrayRowCount(rowCount)) { + sortedIndices = + (int[][]) PrimitiveArrayManager.createDataListsByType(TSDataType.INT32, rowCount); } if (!sorted) { sort(0, rowCount); @@ -59,20 +59,20 @@ public void tim_set(int src, int dest) { @Override public void set(int src, int dest) { long srcT = getTime(src); - boolean srcV = getBoolean(src); + int srcV = getValueIndex(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] = getBoolean(src); + sortedIndices[dest / ARRAY_SIZE][dest % ARRAY_SIZE] = getValueIndex(src); } @Override public void saveAsPivot(int pos) { pivotTime = getTime(pos); - pivotValue = getBoolean(pos); + pivotIndex = getValueIndex(pos); } @Override @@ -80,12 +80,12 @@ public void setFromSorted(int src, int dest) { set( dest, sortedTimestamps[src / ARRAY_SIZE][src % ARRAY_SIZE], - sortedValues[src / ARRAY_SIZE][src % ARRAY_SIZE]); + sortedIndices[src / ARRAY_SIZE][src % ARRAY_SIZE]); } @Override public void setPivotTo(int pos) { - set(pos, pivotTime, pivotValue); + set(pos, pivotTime, pivotIndex); } @Override @@ -97,8 +97,8 @@ public void clearSortedTime() { @Override public void clearSortedValue() { - if (sortedValues != null) { - sortedValues = null; + if (sortedIndices != null) { + sortedIndices = null; } } @@ -114,9 +114,9 @@ public void reverseRange(int lo, int hi) { hi--; while (lo < hi) { long loT = getTime(lo); - boolean loV = getBoolean(lo); + int loV = getValueIndex(lo); long hiT = getTime(hi); - boolean hiV = getBoolean(hi); + int hiV = getValueIndex(hi); set(lo++, hiT, hiV); set(hi--, loT, loV); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TimDoubleTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TimDoubleTVList.java index 99739d3507511..f62bfa94cefd8 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TimDoubleTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TimDoubleTVList.java @@ -26,10 +26,10 @@ public class TimDoubleTVList extends DoubleTVList implements TimSort { private long[][] sortedTimestamps; - private long pivotTime; + private int[][] sortedIndices; - private double[][] sortedValues; - private double pivotValue; + private long pivotTime; + private int pivotIndex; @Override public void sort() { @@ -38,10 +38,10 @@ public void sort() { sortedTimestamps = (long[][]) PrimitiveArrayManager.createDataListsByType(TSDataType.INT64, rowCount); } - if (sortedValues == null - || sortedValues.length < PrimitiveArrayManager.getArrayRowCount(rowCount)) { - sortedValues = - (double[][]) PrimitiveArrayManager.createDataListsByType(TSDataType.DOUBLE, rowCount); + if (sortedIndices == null + || sortedIndices.length < PrimitiveArrayManager.getArrayRowCount(rowCount)) { + sortedIndices = + (int[][]) PrimitiveArrayManager.createDataListsByType(TSDataType.INT32, rowCount); } if (!sorted) { sort(0, rowCount); @@ -59,20 +59,20 @@ public void tim_set(int src, int dest) { @Override public void set(int src, int dest) { long srcT = getTime(src); - double srcV = getDouble(src); + int srcV = getValueIndex(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] = getDouble(src); + sortedIndices[dest / ARRAY_SIZE][dest % ARRAY_SIZE] = getValueIndex(src); } @Override public void saveAsPivot(int pos) { pivotTime = getTime(pos); - pivotValue = getDouble(pos); + pivotIndex = getValueIndex(pos); } @Override @@ -80,12 +80,12 @@ public void setFromSorted(int src, int dest) { set( dest, sortedTimestamps[src / ARRAY_SIZE][src % ARRAY_SIZE], - sortedValues[src / ARRAY_SIZE][src % ARRAY_SIZE]); + sortedIndices[src / ARRAY_SIZE][src % ARRAY_SIZE]); } @Override public void setPivotTo(int pos) { - set(pos, pivotTime, pivotValue); + set(pos, pivotTime, pivotIndex); } @Override @@ -97,8 +97,8 @@ public void clearSortedTime() { @Override public void clearSortedValue() { - if (sortedValues != null) { - sortedValues = null; + if (sortedIndices != null) { + sortedIndices = null; } } @@ -114,9 +114,9 @@ public void reverseRange(int lo, int hi) { hi--; while (lo < hi) { long loT = getTime(lo); - double loV = getDouble(lo); + int loV = getValueIndex(lo); long hiT = getTime(hi); - double hiV = getDouble(hi); + int hiV = getValueIndex(hi); set(lo++, hiT, hiV); set(hi--, loT, loV); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TimFloatTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TimFloatTVList.java index 814b0f7d9701a..f5b0d03846358 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TimFloatTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TimFloatTVList.java @@ -25,12 +25,11 @@ import static org.apache.iotdb.db.storageengine.rescon.memory.PrimitiveArrayManager.ARRAY_SIZE; public class TimFloatTVList extends FloatTVList implements TimSort { - private long[][] sortedTimestamps; - private long pivotTime; + private int[][] sortedIndices; - private float[][] sortedValues; - private float pivotValue; + private long pivotTime; + private int pivotIndex; @Override public void sort() { @@ -39,10 +38,10 @@ public void sort() { sortedTimestamps = (long[][]) PrimitiveArrayManager.createDataListsByType(TSDataType.INT64, rowCount); } - if (sortedValues == null - || sortedValues.length < PrimitiveArrayManager.getArrayRowCount(rowCount)) { - sortedValues = - (float[][]) PrimitiveArrayManager.createDataListsByType(TSDataType.FLOAT, rowCount); + if (sortedIndices == null + || sortedIndices.length < PrimitiveArrayManager.getArrayRowCount(rowCount)) { + sortedIndices = + (int[][]) PrimitiveArrayManager.createDataListsByType(TSDataType.INT32, rowCount); } if (!sorted) { sort(0, rowCount); @@ -60,20 +59,20 @@ public void tim_set(int src, int dest) { @Override public void set(int src, int dest) { long srcT = getTime(src); - float srcV = getFloat(src); + int srcV = getValueIndex(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] = getFloat(src); + sortedIndices[dest / ARRAY_SIZE][dest % ARRAY_SIZE] = getValueIndex(src); } @Override public void saveAsPivot(int pos) { pivotTime = getTime(pos); - pivotValue = getFloat(pos); + pivotIndex = getValueIndex(pos); } @Override @@ -81,12 +80,12 @@ public void setFromSorted(int src, int dest) { set( dest, sortedTimestamps[src / ARRAY_SIZE][src % ARRAY_SIZE], - sortedValues[src / ARRAY_SIZE][src % ARRAY_SIZE]); + sortedIndices[src / ARRAY_SIZE][src % ARRAY_SIZE]); } @Override public void setPivotTo(int pos) { - set(pos, pivotTime, pivotValue); + set(pos, pivotTime, pivotIndex); } @Override @@ -98,8 +97,8 @@ public void clearSortedTime() { @Override public void clearSortedValue() { - if (sortedValues != null) { - sortedValues = null; + if (sortedIndices != null) { + sortedIndices = null; } } @@ -115,9 +114,9 @@ public void reverseRange(int lo, int hi) { hi--; while (lo < hi) { long loT = getTime(lo); - float loV = getFloat(lo); + int loV = getValueIndex(lo); long hiT = getTime(hi); - float hiV = getFloat(hi); + int hiV = getValueIndex(hi); set(lo++, hiT, hiV); set(hi--, loT, loV); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TimIntTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TimIntTVList.java index a7fcba12675eb..421fbe2916d20 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TimIntTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TimIntTVList.java @@ -25,11 +25,10 @@ import static org.apache.iotdb.db.storageengine.rescon.memory.PrimitiveArrayManager.ARRAY_SIZE; public class TimIntTVList extends IntTVList implements TimSort { - private long[][] sortedTimestamps; - private int[][] sortedValues; + private int[][] sortedIndices; - private int pivotValue; + private int pivotIndex; private long pivotTime; @Override @@ -39,9 +38,9 @@ public void sort() { sortedTimestamps = (long[][]) PrimitiveArrayManager.createDataListsByType(TSDataType.INT64, rowCount); } - if (sortedValues == null - || sortedValues.length < PrimitiveArrayManager.getArrayRowCount(rowCount)) { - sortedValues = + if (sortedIndices == null + || sortedIndices.length < PrimitiveArrayManager.getArrayRowCount(rowCount)) { + sortedIndices = (int[][]) PrimitiveArrayManager.createDataListsByType(TSDataType.INT32, rowCount); } if (!sorted) { @@ -60,20 +59,20 @@ public void tim_set(int src, int dest) { @Override public void set(int src, int dest) { long srcT = getTime(src); - int srcV = getInt(src); + int srcV = getValueIndex(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); + sortedIndices[dest / ARRAY_SIZE][dest % ARRAY_SIZE] = getValueIndex(src); } @Override public void saveAsPivot(int pos) { pivotTime = getTime(pos); - pivotValue = getInt(pos); + pivotIndex = getValueIndex(pos); } @Override @@ -81,12 +80,12 @@ public void setFromSorted(int src, int dest) { set( dest, sortedTimestamps[src / ARRAY_SIZE][src % ARRAY_SIZE], - sortedValues[src / ARRAY_SIZE][src % ARRAY_SIZE]); + sortedIndices[src / ARRAY_SIZE][src % ARRAY_SIZE]); } @Override public void setPivotTo(int pos) { - set(pos, pivotTime, pivotValue); + set(pos, pivotTime, pivotIndex); } @Override @@ -98,8 +97,8 @@ public void clearSortedTime() { @Override public void clearSortedValue() { - if (sortedValues != null) { - sortedValues = null; + if (sortedIndices != null) { + sortedIndices = null; } } @@ -115,9 +114,9 @@ public void reverseRange(int lo, int hi) { hi--; while (lo < hi) { long loT = getTime(lo); - int loV = getInt(lo); + int loV = getValueIndex(lo); long hiT = getTime(hi); - int hiV = getInt(hi); + int hiV = getValueIndex(hi); set(lo++, hiT, hiV); set(hi--, loT, loV); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TimLongTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TimLongTVList.java index 43e6ab4a58823..7c0f5640eeeb8 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TimLongTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TimLongTVList.java @@ -26,10 +26,10 @@ public class TimLongTVList extends LongTVList implements TimSort { private long[][] sortedTimestamps; - private long pivotTime; + private int[][] sortedIndices; - private long[][] sortedValues; - private long pivotValue; + private long pivotTime; + private int pivotIndex; @Override public void sort() { @@ -38,10 +38,10 @@ public void sort() { sortedTimestamps = (long[][]) PrimitiveArrayManager.createDataListsByType(TSDataType.INT64, rowCount); } - if (sortedValues == null - || sortedValues.length < PrimitiveArrayManager.getArrayRowCount(rowCount)) { - sortedValues = - (long[][]) PrimitiveArrayManager.createDataListsByType(TSDataType.INT64, rowCount); + if (sortedIndices == null + || sortedIndices.length < PrimitiveArrayManager.getArrayRowCount(rowCount)) { + sortedIndices = + (int[][]) PrimitiveArrayManager.createDataListsByType(TSDataType.INT32, rowCount); } if (!sorted) { sort(0, rowCount); @@ -59,20 +59,20 @@ public void tim_set(int src, int dest) { @Override public void set(int src, int dest) { long srcT = getTime(src); - long srcV = getLong(src); + int srcV = getValueIndex(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] = getLong(src); + sortedIndices[dest / ARRAY_SIZE][dest % ARRAY_SIZE] = getValueIndex(src); } @Override public void saveAsPivot(int pos) { pivotTime = getTime(pos); - pivotValue = getLong(pos); + pivotIndex = getValueIndex(pos); } @Override @@ -80,12 +80,12 @@ public void setFromSorted(int src, int dest) { set( dest, sortedTimestamps[src / ARRAY_SIZE][src % ARRAY_SIZE], - sortedValues[src / ARRAY_SIZE][src % ARRAY_SIZE]); + sortedIndices[src / ARRAY_SIZE][src % ARRAY_SIZE]); } @Override public void setPivotTo(int pos) { - set(pos, pivotTime, pivotValue); + set(pos, pivotTime, pivotIndex); } @Override @@ -97,8 +97,8 @@ public void clearSortedTime() { @Override public void clearSortedValue() { - if (sortedValues != null) { - sortedValues = null; + if (sortedIndices != null) { + sortedIndices = null; } } @@ -114,9 +114,9 @@ public void reverseRange(int lo, int hi) { hi--; while (lo < hi) { long loT = getTime(lo); - long loV = getLong(lo); + int loV = getValueIndex(lo); long hiT = getTime(hi); - long hiV = getLong(hi); + int hiV = getValueIndex(hi); set(lo++, hiT, hiV); set(hi--, loT, loV); } From 7a150738bb3bbf351532e606574f30f97062b8b0 Mon Sep 17 00:00:00 2001 From: shizy Date: Sat, 30 Nov 2024 17:22:33 +0800 Subject: [PATCH 02/54] update min/max timestamp and sequential part of tvlist during insert --- .../db/utils/datastructure/AlignedTVList.java | 10 ++-- .../db/utils/datastructure/BinaryTVList.java | 30 +++++++++--- .../db/utils/datastructure/BooleanTVList.java | 30 +++++++++--- .../db/utils/datastructure/DoubleTVList.java | 30 +++++++++--- .../db/utils/datastructure/FloatTVList.java | 30 +++++++++--- .../db/utils/datastructure/IntTVList.java | 30 +++++++++--- .../db/utils/datastructure/LongTVList.java | 30 +++++++++--- .../iotdb/db/utils/datastructure/TVList.java | 49 +++++++++++++++++-- 8 files changed, 191 insertions(+), 48 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/AlignedTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/AlignedTVList.java index 442fe84b37838..2c00e3c78d086 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/AlignedTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/AlignedTVList.java @@ -231,8 +231,12 @@ public void putAlignedValue(long timestamp, Object[] value) { } indices.get(arrayIndex)[elementIndex] = rowCount; rowCount++; - if (sorted && rowCount > 1 && timestamp < getTime(rowCount - 2)) { - sorted = false; + if (sorted) { + if (rowCount > 1 && timestamp < getTime(rowCount - 2)) { + sorted = false; + } else { + seqRowCount++; + } } } @@ -748,7 +752,7 @@ public void putAlignedValues( checkExpansion(); int idx = start; - updateMaxTimeAndSorted(time, start, end); + updateMinMaxTimeAndSorted(time, start, end); while (idx < end) { int inputRemaining = end - idx; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BinaryTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BinaryTVList.java index 0e112d7815976..a8cef34e7b68a 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BinaryTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BinaryTVList.java @@ -92,12 +92,17 @@ public void putBinary(long timestamp, Binary value) { int arrayIndex = rowCount / ARRAY_SIZE; int elementIndex = rowCount % ARRAY_SIZE; maxTime = Math.max(maxTime, timestamp); + minTime = Math.min(minTime, timestamp); timestamps.get(arrayIndex)[elementIndex] = timestamp; values.get(arrayIndex)[elementIndex] = value; indices.get(arrayIndex)[elementIndex] = rowCount; rowCount++; - if (sorted && rowCount > 1 && timestamp < getTime(rowCount - 2)) { - sorted = false; + if (sorted) { + if (rowCount > 1 && timestamp < getTime(rowCount - 2)) { + sorted = false; + } else { + seqRowCount++; + } } memoryBinaryChunkSize += getBinarySize(value); } @@ -186,10 +191,10 @@ public void putBinaries(long[] time, Binary[] value, BitMap bitMap, int start, i timeIdxOffset = start; // drop null at the end of value array int nullCnt = - dropNullValThenUpdateMaxTimeAndSorted(time, value, bitMap, start, end, timeIdxOffset); + dropNullValThenUpdateMinMaxTimeAndSorted(time, value, bitMap, start, end, timeIdxOffset); end -= nullCnt; } else { - updateMaxTimeAndSorted(time, start, end); + updateMinMaxTimeAndSorted(time, start, end); } // update raw size @@ -227,12 +232,13 @@ public void putBinaries(long[] time, Binary[] value, BitMap bitMap, int start, i } // move null values to the end of time array and value array, then return number of null values - int dropNullValThenUpdateMaxTimeAndSorted( + int dropNullValThenUpdateMinMaxTimeAndSorted( long[] time, Binary[] values, BitMap bitMap, int start, int end, int tIdxOffset) { long inPutMinTime = Long.MAX_VALUE; boolean inputSorted = true; int nullCnt = 0; + int inputSeqRowCount = 0; for (int vIdx = start; vIdx < end; vIdx++) { if (bitMap.isMarked(vIdx)) { nullCnt++; @@ -248,11 +254,21 @@ int dropNullValThenUpdateMaxTimeAndSorted( tIdx = tIdx - nullCnt; inPutMinTime = Math.min(inPutMinTime, time[tIdx]); maxTime = Math.max(maxTime, time[tIdx]); - if (inputSorted && tIdx > 0 && time[tIdx - 1] > time[tIdx]) { - inputSorted = false; + minTime = Math.min(minTime, time[tIdx]); + if (inputSorted) { + if (tIdx > 0 && time[tIdx - 1] > time[tIdx]) { + inputSorted = false; + } else { + inputSeqRowCount++; + } } } + if (sorted + && (rowCount == 0 + || (end - start > nullCnt) && time[start - tIdxOffset] > getTime(rowCount - 1))) { + seqRowCount += inputSeqRowCount; + } sorted = sorted && inputSorted && (rowCount == 0 || inPutMinTime >= getTime(rowCount - 1)); return nullCnt; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BooleanTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BooleanTVList.java index 4673083c78ab8..839544d5713c8 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BooleanTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BooleanTVList.java @@ -85,12 +85,17 @@ public void putBoolean(long timestamp, boolean value) { int arrayIndex = rowCount / ARRAY_SIZE; int elementIndex = rowCount % ARRAY_SIZE; maxTime = Math.max(maxTime, timestamp); + minTime = Math.min(minTime, timestamp); timestamps.get(arrayIndex)[elementIndex] = timestamp; values.get(arrayIndex)[elementIndex] = value; indices.get(arrayIndex)[elementIndex] = rowCount; rowCount++; - if (sorted && rowCount > 1 && timestamp < getTime(rowCount - 2)) { - sorted = false; + if (sorted) { + if (rowCount > 1 && timestamp < getTime(rowCount - 2)) { + sorted = false; + } else { + seqRowCount++; + } } } @@ -173,10 +178,10 @@ public void putBooleans(long[] time, boolean[] value, BitMap bitMap, int start, timeIdxOffset = start; // drop null at the end of value array int nullCnt = - dropNullValThenUpdateMaxTimeAndSorted(time, value, bitMap, start, end, timeIdxOffset); + dropNullValThenUpdateMinMaxTimeAndSorted(time, value, bitMap, start, end, timeIdxOffset); end -= nullCnt; } else { - updateMaxTimeAndSorted(time, start, end); + updateMinMaxTimeAndSorted(time, start, end); } while (idx < end) { @@ -209,12 +214,13 @@ public void putBooleans(long[] time, boolean[] value, BitMap bitMap, int start, } // move null values to the end of time array and value array, then return number of null values - int dropNullValThenUpdateMaxTimeAndSorted( + int dropNullValThenUpdateMinMaxTimeAndSorted( long[] time, boolean[] values, BitMap bitMap, int start, int end, int tIdxOffset) { long inPutMinTime = Long.MAX_VALUE; boolean inputSorted = true; int nullCnt = 0; + int inputSeqRowCount = 0; for (int vIdx = start; vIdx < end; vIdx++) { if (bitMap.isMarked(vIdx)) { nullCnt++; @@ -230,11 +236,21 @@ int dropNullValThenUpdateMaxTimeAndSorted( tIdx = tIdx - nullCnt; inPutMinTime = Math.min(inPutMinTime, time[tIdx]); maxTime = Math.max(maxTime, time[tIdx]); - if (inputSorted && tIdx > 0 && time[tIdx - 1] > time[tIdx]) { - inputSorted = false; + minTime = Math.min(minTime, time[tIdx]); + if (inputSorted) { + if (tIdx > 0 && time[tIdx - 1] > time[tIdx]) { + inputSorted = false; + } else { + inputSeqRowCount++; + } } } + if (sorted + && (rowCount == 0 + || (end - start > nullCnt) && time[start - tIdxOffset] > getTime(rowCount - 1))) { + seqRowCount += inputSeqRowCount; + } sorted = sorted && inputSorted && (rowCount == 0 || inPutMinTime >= getTime(rowCount - 1)); return nullCnt; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/DoubleTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/DoubleTVList.java index e9dc4298a5550..d15851e54deea 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/DoubleTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/DoubleTVList.java @@ -86,12 +86,17 @@ public void putDouble(long timestamp, double value) { int arrayIndex = rowCount / ARRAY_SIZE; int elementIndex = rowCount % ARRAY_SIZE; maxTime = Math.max(maxTime, timestamp); + minTime = Math.min(minTime, timestamp); timestamps.get(arrayIndex)[elementIndex] = timestamp; values.get(arrayIndex)[elementIndex] = value; indices.get(arrayIndex)[elementIndex] = rowCount; rowCount++; - if (sorted && rowCount > 1 && timestamp < getTime(rowCount - 2)) { - sorted = false; + if (sorted) { + if (rowCount > 1 && timestamp < getTime(rowCount - 2)) { + sorted = false; + } else { + seqRowCount++; + } } } @@ -179,10 +184,10 @@ public void putDoubles(long[] time, double[] value, BitMap bitMap, int start, in timeIdxOffset = start; // drop null at the end of value array int nullCnt = - dropNullValThenUpdateMaxTimeAndSorted(time, value, bitMap, start, end, timeIdxOffset); + dropNullValThenUpdateMinMaxTimeAndSorted(time, value, bitMap, start, end, timeIdxOffset); end -= nullCnt; } else { - updateMaxTimeAndSorted(time, start, end); + updateMinMaxTimeAndSorted(time, start, end); } while (idx < end) { @@ -215,12 +220,13 @@ public void putDoubles(long[] time, double[] value, BitMap bitMap, int start, in } // move null values to the end of time array and value array, then return number of null values - int dropNullValThenUpdateMaxTimeAndSorted( + int dropNullValThenUpdateMinMaxTimeAndSorted( long[] time, double[] values, BitMap bitMap, int start, int end, int tIdxOffset) { long inPutMinTime = Long.MAX_VALUE; boolean inputSorted = true; int nullCnt = 0; + int inputSeqRowCount = 0; for (int vIdx = start; vIdx < end; vIdx++) { if (bitMap.isMarked(vIdx)) { nullCnt++; @@ -236,11 +242,21 @@ int dropNullValThenUpdateMaxTimeAndSorted( tIdx = tIdx - nullCnt; inPutMinTime = Math.min(inPutMinTime, time[tIdx]); maxTime = Math.max(maxTime, time[tIdx]); - if (inputSorted && tIdx > 0 && time[tIdx - 1] > time[tIdx]) { - inputSorted = false; + minTime = Math.min(minTime, time[tIdx]); + if (inputSorted) { + if (tIdx > 0 && time[tIdx - 1] > time[tIdx]) { + inputSorted = false; + } else { + inputSeqRowCount++; + } } } + if (sorted + && (rowCount == 0 + || (end - start > nullCnt) && time[start - tIdxOffset] > getTime(rowCount - 1))) { + seqRowCount += inputSeqRowCount; + } sorted = sorted && inputSorted && (rowCount == 0 || inPutMinTime >= getTime(rowCount - 1)); return nullCnt; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/FloatTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/FloatTVList.java index 1165aec71ed84..3aea905fa3f15 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/FloatTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/FloatTVList.java @@ -86,12 +86,17 @@ public void putFloat(long timestamp, float value) { int arrayIndex = rowCount / ARRAY_SIZE; int elementIndex = rowCount % ARRAY_SIZE; maxTime = Math.max(maxTime, timestamp); + minTime = Math.min(minTime, timestamp); timestamps.get(arrayIndex)[elementIndex] = timestamp; values.get(arrayIndex)[elementIndex] = value; indices.get(arrayIndex)[elementIndex] = rowCount; rowCount++; - if (sorted && rowCount > 1 && timestamp < getTime(rowCount - 2)) { - sorted = false; + if (sorted) { + if (rowCount > 1 && timestamp < getTime(rowCount - 2)) { + sorted = false; + } else { + seqRowCount++; + } } } @@ -179,10 +184,10 @@ public void putFloats(long[] time, float[] value, BitMap bitMap, int start, int timeIdxOffset = start; // drop null at the end of value array int nullCnt = - dropNullValThenUpdateMaxTimeAndSorted(time, value, bitMap, start, end, timeIdxOffset); + dropNullValThenUpdateMinMaxTimeAndSorted(time, value, bitMap, start, end, timeIdxOffset); end -= nullCnt; } else { - updateMaxTimeAndSorted(time, start, end); + updateMinMaxTimeAndSorted(time, start, end); } while (idx < end) { @@ -215,12 +220,13 @@ public void putFloats(long[] time, float[] value, BitMap bitMap, int start, int } // move null values to the end of time array and value array, then return number of null values - int dropNullValThenUpdateMaxTimeAndSorted( + int dropNullValThenUpdateMinMaxTimeAndSorted( long[] time, float[] values, BitMap bitMap, int start, int end, int tIdxOffset) { long inPutMinTime = Long.MAX_VALUE; boolean inputSorted = true; int nullCnt = 0; + int inputSeqRowCount = 0; for (int vIdx = start; vIdx < end; vIdx++) { if (bitMap.isMarked(vIdx)) { nullCnt++; @@ -236,11 +242,21 @@ int dropNullValThenUpdateMaxTimeAndSorted( tIdx = tIdx - nullCnt; inPutMinTime = Math.min(inPutMinTime, time[tIdx]); maxTime = Math.max(maxTime, time[tIdx]); - if (inputSorted && tIdx > 0 && time[tIdx - 1] > time[tIdx]) { - inputSorted = false; + minTime = Math.min(minTime, time[tIdx]); + if (inputSorted) { + if (tIdx > 0 && time[tIdx - 1] > time[tIdx]) { + inputSorted = false; + } else { + inputSeqRowCount++; + } } } + if (sorted + && (rowCount == 0 + || (end - start > nullCnt) && time[start - tIdxOffset] > getTime(rowCount - 1))) { + seqRowCount += inputSeqRowCount; + } sorted = sorted && inputSorted && (rowCount == 0 || inPutMinTime >= getTime(rowCount - 1)); return nullCnt; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/IntTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/IntTVList.java index 3af718817d449..eedf412a40a6e 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/IntTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/IntTVList.java @@ -85,12 +85,17 @@ public void putInt(long timestamp, int value) { int arrayIndex = rowCount / ARRAY_SIZE; int elementIndex = rowCount % ARRAY_SIZE; maxTime = Math.max(maxTime, timestamp); + minTime = Math.min(minTime, timestamp); timestamps.get(arrayIndex)[elementIndex] = timestamp; values.get(arrayIndex)[elementIndex] = value; indices.get(arrayIndex)[elementIndex] = rowCount; rowCount++; - if (sorted && rowCount > 1 && timestamp < getTime(rowCount - 2)) { - sorted = false; + if (sorted) { + if (rowCount > 1 && timestamp < getTime(rowCount - 2)) { + sorted = false; + } else { + seqRowCount++; + } } } @@ -172,10 +177,10 @@ public void putInts(long[] time, int[] value, BitMap bitMap, int start, int end) timeIdxOffset = start; // drop null at the end of value array int nullCnt = - dropNullValThenUpdateMaxTimeAndSorted(time, value, bitMap, start, end, timeIdxOffset); + dropNullValThenUpdateMinMaxTimeAndSorted(time, value, bitMap, start, end, timeIdxOffset); end -= nullCnt; } else { - updateMaxTimeAndSorted(time, start, end); + updateMinMaxTimeAndSorted(time, start, end); } while (idx < end) { @@ -208,12 +213,13 @@ public void putInts(long[] time, int[] value, BitMap bitMap, int start, int end) } // move null values to the end of time array and value array, then return number of null values - int dropNullValThenUpdateMaxTimeAndSorted( + int dropNullValThenUpdateMinMaxTimeAndSorted( long[] time, int[] values, BitMap bitMap, int start, int end, int tIdxOffset) { long inPutMinTime = Long.MAX_VALUE; boolean inputSorted = true; int nullCnt = 0; + int inputSeqRowCount = 0; for (int vIdx = start; vIdx < end; vIdx++) { if (bitMap.isMarked(vIdx)) { nullCnt++; @@ -229,11 +235,21 @@ int dropNullValThenUpdateMaxTimeAndSorted( tIdx = tIdx - nullCnt; inPutMinTime = Math.min(inPutMinTime, time[tIdx]); maxTime = Math.max(maxTime, time[tIdx]); - if (inputSorted && tIdx > 0 && time[tIdx - 1] > time[tIdx]) { - inputSorted = false; + minTime = Math.min(minTime, time[tIdx]); + if (inputSorted) { + if (tIdx > 0 && time[tIdx - 1] > time[tIdx]) { + inputSorted = false; + } else { + inputSeqRowCount++; + } } } + if (sorted + && (rowCount == 0 + || (end - start > nullCnt) && time[start - tIdxOffset] > getTime(rowCount - 1))) { + seqRowCount += inputSeqRowCount; + } sorted = sorted && inputSorted && (rowCount == 0 || inPutMinTime >= getTime(rowCount - 1)); return nullCnt; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/LongTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/LongTVList.java index 557acdcb95480..73e2d792c5192 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/LongTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/LongTVList.java @@ -85,12 +85,17 @@ public void putLong(long timestamp, long value) { int arrayIndex = rowCount / ARRAY_SIZE; int elementIndex = rowCount % ARRAY_SIZE; maxTime = Math.max(maxTime, timestamp); + minTime = Math.min(minTime, timestamp); timestamps.get(arrayIndex)[elementIndex] = timestamp; values.get(arrayIndex)[elementIndex] = value; indices.get(arrayIndex)[elementIndex] = rowCount; rowCount++; - if (sorted && rowCount > 1 && timestamp < getTime(rowCount - 2)) { - sorted = false; + if (sorted) { + if (rowCount > 1 && timestamp < getTime(rowCount - 2)) { + sorted = false; + } else { + seqRowCount++; + } } } @@ -172,10 +177,10 @@ public void putLongs(long[] time, long[] value, BitMap bitMap, int start, int en timeIdxOffset = start; // drop null at the end of value array int nullCnt = - dropNullValThenUpdateMaxTimeAndSorted(time, value, bitMap, start, end, timeIdxOffset); + dropNullValThenUpdateMinMaxTimeAndSorted(time, value, bitMap, start, end, timeIdxOffset); end -= nullCnt; } else { - updateMaxTimeAndSorted(time, start, end); + updateMinMaxTimeAndSorted(time, start, end); } while (idx < end) { @@ -208,12 +213,13 @@ public void putLongs(long[] time, long[] value, BitMap bitMap, int start, int en } // move null values to the end of time array and value array, then return number of null values - int dropNullValThenUpdateMaxTimeAndSorted( + int dropNullValThenUpdateMinMaxTimeAndSorted( long[] time, long[] values, BitMap bitMap, int start, int end, int tIdxOffset) { long inPutMinTime = Long.MAX_VALUE; boolean inputSorted = true; int nullCnt = 0; + int inputSeqRowCount = 0; for (int vIdx = start; vIdx < end; vIdx++) { if (bitMap.isMarked(vIdx)) { nullCnt++; @@ -229,11 +235,21 @@ int dropNullValThenUpdateMaxTimeAndSorted( tIdx = tIdx - nullCnt; inPutMinTime = Math.min(inPutMinTime, time[tIdx]); maxTime = Math.max(maxTime, time[tIdx]); - if (inputSorted && tIdx > 0 && time[tIdx - 1] > time[tIdx]) { - inputSorted = false; + minTime = Math.min(minTime, time[tIdx]); + if (inputSorted) { + if (tIdx > 0 && time[tIdx - 1] > time[tIdx]) { + inputSorted = false; + } else { + inputSeqRowCount++; + } } } + if (sorted + && (rowCount == 0 + || (end - start > nullCnt) && time[start - tIdxOffset] > getTime(rowCount - 1))) { + seqRowCount += inputSeqRowCount; + } sorted = sorted && inputSorted && (rowCount == 0 || inPutMinTime >= getTime(rowCount - 1)); return nullCnt; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java index 4a2d7dde2f3e5..96d0dda659725 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java @@ -22,6 +22,7 @@ import org.apache.iotdb.common.rpc.thrift.TSStatus; import org.apache.iotdb.commons.utils.TestOnly; import org.apache.iotdb.db.conf.IoTDBDescriptor; +import org.apache.iotdb.db.queryengine.execution.fragment.QueryContext; import org.apache.iotdb.db.storageengine.dataregion.wal.buffer.WALEntryValue; import org.apache.iotdb.db.storageengine.rescon.memory.PrimitiveArrayManager; import org.apache.iotdb.db.utils.MathUtils; @@ -57,6 +58,8 @@ public abstract class TVList implements WALEntryValue { // index relation: arrayIndex -> elementIndex protected List timestamps; protected int rowCount; + // the count of sequential part started from the beginning + protected int seqRowCount; // List of index array, add 1 when expanded -> data point index array // Index relation: arrayIndex -> elementIndex @@ -67,8 +70,16 @@ public abstract class TVList implements WALEntryValue { // Index relation: arrayIndex -> elementIndex protected List bitMap; + // list of query that this TVList is used + protected final List queryContextList; + + // the owner query which is obligated to release the TVList. + // When it is null, the TVList is owned by insert thread and released after flush. + protected QueryContext ownerQuery; + protected boolean sorted = true; protected long maxTime; + protected long minTime; // record reference count of this tv list // currently this reference will only be increase because we can't know when to decrease it protected AtomicInteger referenceCount; @@ -78,7 +89,10 @@ protected TVList() { timestamps = new ArrayList<>(); indices = new ArrayList<>(); rowCount = 0; + seqRowCount = 0; maxTime = Long.MIN_VALUE; + minTime = Long.MAX_VALUE; + queryContextList = new ArrayList<>(); referenceCount = new AtomicInteger(); } @@ -343,6 +357,10 @@ public long getMaxTime() { return maxTime; } + public long getMinTime() { + return minTime; + } + public long getVersion() { return version; } @@ -368,6 +386,7 @@ protected void releaseLastTimeArray() { public int delete(long lowerBound, long upperBound) { int deletedNumber = 0; long maxTime = Long.MIN_VALUE; + long minTime = Long.MAX_VALUE; for (int i = 0; i < rowCount; i++) { long time = getTime(i); if (time >= lowerBound && time <= upperBound) { @@ -378,6 +397,7 @@ public int delete(long lowerBound, long upperBound) { deletedNumber++; } else { maxTime = Math.max(time, maxTime); + minTime = Math.min(time, minTime); } } return deletedNumber; @@ -388,14 +408,20 @@ protected void cloneAs(TVList cloneList) { cloneList.timestamps.add(cloneTime(timestampArray)); } cloneList.rowCount = rowCount; + cloneList.seqRowCount = seqRowCount; cloneList.sorted = sorted; cloneList.maxTime = maxTime; + cloneList.minTime = minTime; } public void clear() { rowCount = 0; + seqRowCount = 0; sorted = true; maxTime = Long.MIN_VALUE; + minTime = Long.MAX_VALUE; + queryContextList.clear(); + ownerQuery = null; clearTime(); clearValue(); } @@ -428,17 +454,26 @@ protected long[] cloneTime(long[] array) { return cloneArray; } - void updateMaxTimeAndSorted(long[] time, int start, int end) { + void updateMinMaxTimeAndSorted(long[] time, int start, int end) { int length = time.length; long inPutMinTime = Long.MAX_VALUE; boolean inputSorted = true; + int inputSeqRowCount = 0; for (int i = start; i < end; i++) { inPutMinTime = Math.min(inPutMinTime, time[i]); maxTime = Math.max(maxTime, time[i]); - if (inputSorted && i < length - 1 && time[i] > time[i + 1]) { - inputSorted = false; + minTime = Math.min(minTime, time[i]); + if (inputSorted) { + if (i < length - 1 && time[i] > time[i + 1]) { + inputSorted = false; + } else { + inputSeqRowCount++; + } } } + if (sorted && (rowCount == 0 || time[start] > getTime(rowCount - 1))) { + seqRowCount += inputSeqRowCount; + } sorted = sorted && inputSorted && (rowCount == 0 || inPutMinTime >= getTime(rowCount - 1)); } @@ -514,4 +549,12 @@ public static TVList deserialize(DataInputStream stream) throws IOException { public List getTimestamps() { return timestamps; } + + public void setOwnerQuery(QueryContext queryCtx) { + this.ownerQuery = queryCtx; + } + + public List getQueryContextList() { + return queryContextList; + } } From c28fe3f3a235813f4eb01e0895387f3d38f15403 Mon Sep 17 00:00:00 2001 From: shizy Date: Sun, 1 Dec 2024 17:09:47 +0800 Subject: [PATCH 03/54] mutable & immutable tvlists in writable memchunk --- .../fragment/FragmentInstanceContext.java | 32 ++ .../execution/fragment/QueryContext.java | 9 + .../utils/ResourceByPathUtils.java | 79 ++++- .../dataregion/memtable/AbstractMemTable.java | 9 +- .../memtable/IWritableMemChunk.java | 10 + .../dataregion/memtable/ReadOnlyMemChunk.java | 263 ++++++++++++--- .../dataregion/memtable/WritableMemChunk.java | 317 +++++++++++++----- .../memtable/WritableMemChunkGroup.java | 2 +- .../read/reader/chunk/MemChunkReader.java | 136 +++++++- .../read/reader/chunk/MemPageReader.java | 126 ++++++- .../reader/chunk/metadata/PageMetadata.java | 90 +++++ .../MergeSortTvListIterator.java | 148 ++++++++ .../iotdb/db/utils/datastructure/TVList.java | 90 ++++- .../read/reader/chunk/MemChunkLoaderTest.java | 130 +++---- .../read/reader/chunk/MemPageReaderTest.java | 12 +- 15 files changed, 1227 insertions(+), 226 deletions(-) create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/metadata/PageMetadata.java create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MergeSortTvListIterator.java diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceContext.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceContext.java index 6d035e1dc3c6f..4dea7f584cd42 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceContext.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceContext.java @@ -41,6 +41,7 @@ import org.apache.iotdb.db.storageengine.dataregion.read.QueryDataSourceType; import org.apache.iotdb.db.storageengine.dataregion.read.control.FileReaderManager; import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; +import org.apache.iotdb.db.utils.datastructure.TVList; import org.apache.iotdb.mpp.rpc.thrift.TFetchFragmentInstanceStatisticsResp; import org.apache.tsfile.file.metadata.IDeviceID; @@ -649,6 +650,34 @@ public void releaseResourceWhenAllDriversAreClosed() { releaseResource(); } + private void releaseTVListOwnedByQuery() { + for (TVList tvList : tvListSet) { + tvList.lockQueryList(); + List queryContextList = tvList.getQueryContextList(); + try { + queryContextList.remove(this); + if (tvList.getOwnerQuery() == this) { + tvList.setOwnerQuery(null); + if (queryContextList.isEmpty()) { + LOGGER.debug( + "TVList {} is released by the query, FragmentInstance Id is {}", + tvList, + this.getId()); + tvList.clear(); + } else { + LOGGER.debug( + "TVList {} is owned by another query, FragmentInstance Id is {}", + tvList, + ((FragmentInstanceContext) queryContextList.get(0)).getId()); + tvList.setOwnerQuery(queryContextList.get(0)); + } + } + } finally { + tvList.unlockQueryList(); + } + } + } + /** * All file paths used by this fragment instance must be cleared and thus the usage reference must * be decreased. @@ -669,6 +698,9 @@ public synchronized void releaseResource() { unClosedFilePaths = null; } + // release TVList owned by current query + releaseTVListOwnedByQuery(); + dataRegion = null; globalTimeFilter = null; sharedQueryDataSource = null; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/QueryContext.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/QueryContext.java index 566e998d2c4b1..242d92a3d5ae2 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/QueryContext.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/QueryContext.java @@ -28,6 +28,7 @@ import org.apache.iotdb.db.utils.ModificationUtils; import org.apache.iotdb.db.utils.datastructure.PatternTreeMapFactory; import org.apache.iotdb.db.utils.datastructure.PatternTreeMapFactory.ModsSerializer; +import org.apache.iotdb.db.utils.datastructure.TVList; import org.apache.tsfile.file.metadata.IDeviceID; import org.slf4j.Logger; @@ -35,6 +36,7 @@ import java.util.ArrayList; import java.util.Collections; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; @@ -71,6 +73,9 @@ public class QueryContext { private final Set nonExistentModFiles = new CopyOnWriteArraySet<>(); + // accessed tvlists for the query + protected final Set tvListSet = new HashSet<>(); + public QueryContext() {} public QueryContext(long queryId) { @@ -206,4 +211,8 @@ public boolean isIgnoreAllNullRows() { public void setIgnoreAllNullRows(boolean ignoreAllNullRows) { this.ignoreAllNullRows = ignoreAllNullRows; } + + public void addTVListToSet(Map tvListMap) { + tvListSet.addAll(tvListMap.keySet()); + } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/utils/ResourceByPathUtils.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/utils/ResourceByPathUtils.java index e7306ef2eb23e..f72f928d4e222 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/utils/ResourceByPathUtils.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/utils/ResourceByPathUtils.java @@ -22,14 +22,16 @@ import org.apache.iotdb.commons.path.IFullPath; import org.apache.iotdb.commons.path.NonAlignedFullPath; import org.apache.iotdb.db.exception.query.QueryProcessException; +import org.apache.iotdb.db.queryengine.execution.fragment.FragmentInstanceContext; import org.apache.iotdb.db.queryengine.execution.fragment.QueryContext; +import org.apache.iotdb.db.queryengine.plan.planner.memory.MemoryReservationManager; import org.apache.iotdb.db.storageengine.dataregion.memtable.AlignedReadOnlyMemChunk; import org.apache.iotdb.db.storageengine.dataregion.memtable.AlignedWritableMemChunk; import org.apache.iotdb.db.storageengine.dataregion.memtable.AlignedWritableMemChunkGroup; import org.apache.iotdb.db.storageengine.dataregion.memtable.IMemTable; -import org.apache.iotdb.db.storageengine.dataregion.memtable.IWritableMemChunk; import org.apache.iotdb.db.storageengine.dataregion.memtable.IWritableMemChunkGroup; import org.apache.iotdb.db.storageengine.dataregion.memtable.ReadOnlyMemChunk; +import org.apache.iotdb.db.storageengine.dataregion.memtable.WritableMemChunk; import org.apache.iotdb.db.storageengine.dataregion.modification.ModEntry; import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; import org.apache.iotdb.db.utils.ModificationUtils; @@ -51,11 +53,14 @@ import org.apache.tsfile.write.schema.IMeasurementSchema; import org.apache.tsfile.write.schema.VectorMeasurementSchema; import org.apache.tsfile.write.writer.RestorableTsFileIOWriter; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.io.IOException; import java.io.Serializable; import java.util.ArrayList; import java.util.Collections; +import java.util.LinkedHashMap; import java.util.List; import java.util.Map; @@ -322,6 +327,9 @@ public List getVisibleMetadataListFromWriter( class MeasurementResourceByPathUtils extends ResourceByPathUtils { + private static final Logger LOGGER = + LoggerFactory.getLogger(MeasurementResourceByPathUtils.class); + NonAlignedFullPath fullPath; protected MeasurementResourceByPathUtils(IFullPath fullPath) { @@ -359,6 +367,62 @@ public ITimeSeriesMetadata generateTimeSeriesMetadata( return timeSeriesMetadata; } + private Map prepareTvListMapForQuery( + WritableMemChunk memChunk, boolean isWorkMemTable, QueryContext context) { + Map tvListQueryMap = new LinkedHashMap<>(); + // immutable sorted lists + for (TVList tvList : memChunk.getSortedList()) { + tvList.lockQueryList(); + try { + LOGGER.debug( + "Flushing/Working MemTable - Add current query context to immutable TVList's query list"); + tvList.getQueryContextList().add(context); + tvListQueryMap.put(tvList, tvList.rowCount()); + } finally { + tvList.unlockQueryList(); + } + } + + // mutable tvlist + TVList list = memChunk.getTVList(); + list.lockQueryList(); + try { + if (!isWorkMemTable) { + LOGGER.debug( + "Flushing MemTable - add current query context to mutable TVList's query list"); + list.getQueryContextList().add(context); + tvListQueryMap.put(list, list.rowCount()); + } else { + if (list.isSorted() || list.getQueryContextList().isEmpty()) { + LOGGER.debug( + "Working MemTable - add current query context to mutable TVList's query list when it's sorted or no other query on it"); + list.getQueryContextList().add(context); + tvListQueryMap.put(list, list.rowCount()); + } else { + LOGGER.debug( + "Working MemTable - clone mutable TVList and replace old TVList in working MemTable"); + QueryContext firstQuery = list.getQueryContextList().get(0); + // reserve query memory + if (firstQuery instanceof FragmentInstanceContext) { + MemoryReservationManager memoryReservationManager = + ((FragmentInstanceContext) firstQuery).getMemoryReservationContext(); + memoryReservationManager.reserveMemoryCumulatively(list.calculateRamSize()); + } + list.setOwnerQuery(firstQuery); + + // clone TVList + TVList cloneList = list.clone(); + cloneList.getQueryContextList().add(context); + tvListQueryMap.put(cloneList, cloneList.rowCount()); + memChunk.setTVList(cloneList); + } + } + } finally { + list.unlockQueryList(); + } + return tvListQueryMap; + } + @Override public ReadOnlyMemChunk getReadOnlyMemChunkFromMemTable( QueryContext context, @@ -373,10 +437,13 @@ public ReadOnlyMemChunk getReadOnlyMemChunkFromMemTable( || !memTableMap.get(deviceID).contains(fullPath.getMeasurement())) { return null; } - IWritableMemChunk memChunk = - memTableMap.get(deviceID).getMemChunkMap().get(fullPath.getMeasurement()); - // get sorted tv list is synchronized so different query can get right sorted list reference - TVList chunkCopy = memChunk.getSortedTvListForQuery(); + WritableMemChunk memChunk = + (WritableMemChunk) + memTableMap.get(deviceID).getMemChunkMap().get(fullPath.getMeasurement()); + // prepare TVList for query. It should clone and sort TVList if necessary. + // Also, the map keeps TVlist length at this moment. + Map tvListQueryMap = + prepareTvListMapForQuery(memChunk, modsToMemtable == null, context); List deletionList = null; if (modsToMemtable != null) { deletionList = @@ -392,7 +459,7 @@ public ReadOnlyMemChunk getReadOnlyMemChunkFromMemTable( fullPath.getMeasurement(), fullPath.getMeasurementSchema().getType(), fullPath.getMeasurementSchema().getEncodingType(), - chunkCopy, + tvListQueryMap, fullPath.getMeasurementSchema().getProps(), deletionList); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AbstractMemTable.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AbstractMemTable.java index fff93680c9c85..5e18b7582d624 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AbstractMemTable.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AbstractMemTable.java @@ -570,7 +570,7 @@ private void getMemChunkHandleFromMemTable( buildChunkMetaDataForMemoryChunk( measurementId, timestamps[0], - timestamps[tvListCopy.rowCount() - 1], + timestamps[tvListCopy.count() - 1], Collections.emptyList())); memChunkHandleMap .computeIfAbsent(measurementId, k -> new ArrayList<>()) @@ -669,7 +669,7 @@ private void getMemChunkHandleFromMemTable( buildChunkMetaDataForMemoryChunk( measurementId, timestamps[0], - timestamps[tvListCopy.rowCount() - 1], + timestamps[tvListCopy.count() - 1], Collections.emptyList())); memChunkHandleMap .computeIfAbsent(measurementId, k -> new ArrayList<>()) @@ -759,7 +759,7 @@ private IChunkMetadata buildChunkMetaDataForMemoryChunk( } private long[] filterDeletedTimestamp(TVList tvList, List deletionList) { - if (deletionList.isEmpty()) { + if (tvList.getBitMap() == null && deletionList.isEmpty()) { long[] timestamps = tvList.getTimestamps().stream().flatMapToLong(LongStream::of).toArray(); return Arrays.copyOfRange(timestamps, 0, tvList.rowCount()); } @@ -771,7 +771,8 @@ private long[] filterDeletedTimestamp(TVList tvList, List deletionLis for (int i = 0; i < rowCount; i++) { long curTime = tvList.getTime(i); - if (!ModificationUtils.isPointDeleted(curTime, deletionList, deletionCursor) + if (!tvList.isNullValue(i) + && !ModificationUtils.isPointDeleted(curTime, deletionList, deletionCursor) && (i == rowCount - 1 || curTime != lastTime)) { result.add(curTime); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/IWritableMemChunk.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/IWritableMemChunk.java index 2d631e8020974..ced8d8643b0fc 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/IWritableMemChunk.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/IWritableMemChunk.java @@ -31,6 +31,8 @@ import java.util.List; public interface IWritableMemChunk extends WALEntryValue { + // TODO: read from configuration file + int SORT_THRESHOLD = 5000; boolean putLongWithFlushCheck(long t, long v); @@ -84,6 +86,10 @@ boolean writeAlignedValuesWithFlushCheck( long count(); + default long rowCount() { + return 0; + } + IMeasurementSchema getSchema(); /** @@ -133,6 +139,10 @@ default long getMaxTime() { return Long.MAX_VALUE; } + default long getMinTime() { + return Long.MIN_VALUE; + } + /** * @return how many points are deleted */ diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/ReadOnlyMemChunk.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/ReadOnlyMemChunk.java index b860094ac1f5e..9e6e43e62e4f8 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/ReadOnlyMemChunk.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/ReadOnlyMemChunk.java @@ -22,6 +22,8 @@ import org.apache.iotdb.db.exception.query.QueryProcessException; import org.apache.iotdb.db.queryengine.execution.fragment.QueryContext; import org.apache.iotdb.db.storageengine.dataregion.read.reader.chunk.MemChunkLoader; +import org.apache.iotdb.db.utils.MathUtils; +import org.apache.iotdb.db.utils.datastructure.MergeSortTvListIterator; import org.apache.iotdb.db.utils.datastructure.TVList; import org.apache.tsfile.common.conf.TSFileDescriptor; @@ -31,16 +33,22 @@ import org.apache.tsfile.file.metadata.IChunkMetadata; import org.apache.tsfile.file.metadata.enums.TSEncoding; import org.apache.tsfile.file.metadata.statistics.Statistics; +import org.apache.tsfile.read.TimeValuePair; import org.apache.tsfile.read.common.TimeRange; import org.apache.tsfile.read.common.block.TsBlock; +import org.apache.tsfile.read.common.block.TsBlockBuilder; import org.apache.tsfile.read.reader.IPointReader; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; import java.util.List; import java.util.Map; +import static org.apache.iotdb.db.utils.ModificationUtils.isPointDeleted; + /** * ReadOnlyMemChunk is a snapshot of the working MemTable and flushing memtable in the memory used * for querying. @@ -59,6 +67,21 @@ public class ReadOnlyMemChunk { protected TsBlock tsBlock; + private int floatPrecision; + private TSEncoding encoding; + private List deletionList; + + // Read only chunk is now regarded as multiple pages. Apart from chunk statistics, + // we need to collect page statistic and MergeSortTvListIterator offset for each page. + private List pageStatisticsList; + private List pageOffsetsList; + + // tvlist rowCount during query + protected Map tvListQueryMap; + + public static final int MAX_NUMBER_OF_POINTS_IN_PAGE = + TSFileDescriptor.getInstance().getConfig().getMaxNumberOfPointsInPage(); + protected ReadOnlyMemChunk(QueryContext context) { this.context = context; } @@ -68,7 +91,7 @@ public ReadOnlyMemChunk( String measurementUid, TSDataType dataType, TSEncoding encoding, - TVList tvList, + Map tvListQueryMap, Map props, List deletionList) throws IOException, QueryProcessException { @@ -92,58 +115,97 @@ public ReadOnlyMemChunk( floatPrecision = TSFileDescriptor.getInstance().getConfig().getFloatPrecision(); } } - this.tsBlock = tvList.buildTsBlock(floatPrecision, encoding, deletionList); - initChunkMetaFromTsBlock(); + this.floatPrecision = floatPrecision; + this.encoding = encoding; + this.deletionList = deletionList; + this.tvListQueryMap = tvListQueryMap; + this.pageStatisticsList = new ArrayList<>(); + this.pageOffsetsList = new ArrayList<>(); + this.context.addTVListToSet(tvListQueryMap); + + initChunkAndPageStatistics(); } - private void initChunkMetaFromTsBlock() throws QueryProcessException { - Statistics statsByType = Statistics.getStatsByType(dataType); + private void initChunkAndPageStatistics() { + // create chunk metadata + Statistics chunkStatistics = Statistics.getStatsByType(dataType); IChunkMetadata metaData = - new ChunkMetadata(measurementUid, dataType, null, null, 0, statsByType); - if (!isEmpty()) { - switch (dataType) { - case BOOLEAN: - for (int i = 0; i < tsBlock.getPositionCount(); i++) { - statsByType.update(tsBlock.getTimeByIndex(i), tsBlock.getColumn(0).getBoolean(i)); - } - break; - case TEXT: - case BLOB: - case STRING: - for (int i = 0; i < tsBlock.getPositionCount(); i++) { - statsByType.update(tsBlock.getTimeByIndex(i), tsBlock.getColumn(0).getBinary(i)); - } - break; - case FLOAT: - for (int i = 0; i < tsBlock.getPositionCount(); i++) { - statsByType.update(tsBlock.getTimeByIndex(i), tsBlock.getColumn(0).getFloat(i)); - } - break; - case INT32: - case DATE: - for (int i = 0; i < tsBlock.getPositionCount(); i++) { - statsByType.update(tsBlock.getTimeByIndex(i), tsBlock.getColumn(0).getInt(i)); - } - break; - case INT64: - case TIMESTAMP: - for (int i = 0; i < tsBlock.getPositionCount(); i++) { - statsByType.update(tsBlock.getTimeByIndex(i), tsBlock.getColumn(0).getLong(i)); - } - break; - case DOUBLE: - for (int i = 0; i < tsBlock.getPositionCount(); i++) { - statsByType.update(tsBlock.getTimeByIndex(i), tsBlock.getColumn(0).getDouble(i)); - } - break; - default: - throw new QueryProcessException("Unsupported data type:" + dataType); - } - } - statsByType.setEmpty(isEmpty()); + new ChunkMetadata(measurementUid, dataType, null, null, 0, chunkStatistics); metaData.setChunkLoader(new MemChunkLoader(context, this)); metaData.setVersion(Long.MAX_VALUE); cachedMetaData = metaData; + + sortTvLists(); + updateChunkAndPageStatisticsFromTvLists(); + } + + private void sortTvLists() { + for (Map.Entry entry : getTvListQueryMap().entrySet()) { + TVList tvList = entry.getKey(); + int queryRowCount = entry.getValue(); + if (!tvList.isSorted() && queryRowCount > tvList.seqRowCount()) { + tvList.sort(); + } + } + } + + private void updateChunkAndPageStatisticsFromTvLists() { + Statistics chunkStatistics = cachedMetaData.getStatistics(); + + int cnt = 0; + int[] deleteCursor = {0}; + List tvLists = new ArrayList<>(tvListQueryMap.keySet()); + MergeSortTvListIterator timeValuePairIterator = + new MergeSortTvListIterator(dataType, encoding, floatPrecision, tvLists); + int[] tvListOffsets = timeValuePairIterator.getTVListOffsets(); + while (timeValuePairIterator.hasNextTimeValuePair()) { + TimeValuePair tvPair = timeValuePairIterator.nextTimeValuePair(); + if (!isPointDeleted(tvPair.getTimestamp(), deletionList, deleteCursor)) { + if (cnt % MAX_NUMBER_OF_POINTS_IN_PAGE == 0) { + Statistics stats = Statistics.getStatsByType(dataType); + pageStatisticsList.add(stats); + pageOffsetsList.add(tvListOffsets); + } + + Statistics pageStatistics = pageStatisticsList.get(pageStatisticsList.size() - 1); + switch (dataType) { + case BOOLEAN: + chunkStatistics.update(tvPair.getTimestamp(), tvPair.getValue().getBoolean()); + pageStatistics.update(tvPair.getTimestamp(), tvPair.getValue().getBoolean()); + break; + case INT32: + case DATE: + chunkStatistics.update(tvPair.getTimestamp(), tvPair.getValue().getInt()); + pageStatistics.update(tvPair.getTimestamp(), tvPair.getValue().getInt()); + break; + case INT64: + case TIMESTAMP: + chunkStatistics.update(tvPair.getTimestamp(), tvPair.getValue().getLong()); + pageStatistics.update(tvPair.getTimestamp(), tvPair.getValue().getLong()); + break; + case FLOAT: + chunkStatistics.update(tvPair.getTimestamp(), tvPair.getValue().getFloat()); + pageStatistics.update(tvPair.getTimestamp(), tvPair.getValue().getFloat()); + break; + case DOUBLE: + chunkStatistics.update(tvPair.getTimestamp(), tvPair.getValue().getDouble()); + pageStatistics.update(tvPair.getTimestamp(), tvPair.getValue().getDouble()); + break; + case TEXT: + case BLOB: + case STRING: + chunkStatistics.update(tvPair.getTimestamp(), tvPair.getValue().getBinary()); + pageStatistics.update(tvPair.getTimestamp(), tvPair.getValue().getBinary()); + break; + default: + // do nothing + } + pageStatistics.setEmpty(false); + } + tvListOffsets = timeValuePairIterator.getTVListOffsets(); + cnt++; + } + chunkStatistics.setEmpty(cnt == 0); } public TSDataType getDataType() { @@ -151,6 +213,9 @@ public TSDataType getDataType() { } public boolean isEmpty() { + if (tsBlock == null) { + return count() == 0; + } return tsBlock.isEmpty(); } @@ -158,11 +223,115 @@ public IChunkMetadata getChunkMetaData() { return cachedMetaData; } + // we do not call getPointReader in MemChunkReader anymore. However, unit testcases + // still test this method. public IPointReader getPointReader() { + for (Map.Entry entry : tvListQueryMap.entrySet()) { + TVList tvList = entry.getKey(); + int queryLength = entry.getValue(); + if (!tvList.isSorted() && queryLength > tvList.seqRowCount()) { + tvList.sort(); + } + } + TsBlock tsBlock = buildTsBlock(); return tsBlock.getTsBlockSingleColumnIterator(); } + private TsBlock buildTsBlock() { + try { + TsBlockBuilder builder = new TsBlockBuilder(Collections.singletonList(dataType)); + writeValidValuesIntoTsBlock(builder); + return builder.build(); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + // read all data in memory chunk and write to tsblock + private void writeValidValuesIntoTsBlock(TsBlockBuilder builder) throws IOException { + int[] deleteCursor = {0}; + List tvLists = new ArrayList<>(tvListQueryMap.keySet()); + IPointReader timeValuePairIterator = + new MergeSortTvListIterator(dataType, encoding, floatPrecision, tvLists); + + while (timeValuePairIterator.hasNextTimeValuePair()) { + TimeValuePair tvPair = timeValuePairIterator.nextTimeValuePair(); + if (!isPointDeleted(tvPair.getTimestamp(), deletionList, deleteCursor)) { + builder.getTimeColumnBuilder().writeLong(tvPair.getTimestamp()); + switch (dataType) { + case BOOLEAN: + builder.getColumnBuilder(0).writeBoolean(tvPair.getValue().getBoolean()); + break; + case INT32: + case DATE: + builder.getColumnBuilder(0).writeInt(tvPair.getValue().getInt()); + break; + case INT64: + case TIMESTAMP: + builder.getColumnBuilder(0).writeLong(tvPair.getValue().getLong()); + break; + case FLOAT: + float fv = tvPair.getValue().getFloat(); + if (!Float.isNaN(fv) + && (encoding == TSEncoding.RLE || encoding == TSEncoding.TS_2DIFF)) { + fv = MathUtils.roundWithGivenPrecision(fv, floatPrecision); + } + builder.getColumnBuilder(0).writeFloat(fv); + break; + case DOUBLE: + double dv = tvPair.getValue().getDouble(); + if (!Double.isNaN(dv) + && (encoding == TSEncoding.RLE || encoding == TSEncoding.TS_2DIFF)) { + dv = MathUtils.roundWithGivenPrecision(dv, floatPrecision); + } + builder.getColumnBuilder(0).writeDouble(dv); + break; + case TEXT: + case STRING: + case BLOB: + builder.getColumnBuilder(0).writeBinary(tvPair.getValue().getBinary()); + break; + default: + break; + } + builder.declarePosition(); + } + } + } + public TsBlock getTsBlock() { return tsBlock; } + + public Map getTvListQueryMap() { + return tvListQueryMap; + } + + public int count() { + int count = 0; + for (TVList list : tvListQueryMap.keySet()) { + count += list.count(); + } + return count; + } + + public int getFloatPrecision() { + return floatPrecision; + } + + public TSEncoding getEncoding() { + return encoding; + } + + public List getDeletionList() { + return deletionList; + } + + public List getPageStatisticsList() { + return pageStatisticsList; + } + + public List getPageOffsetsList() { + return pageOffsetsList; + } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/WritableMemChunk.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/WritableMemChunk.java index 4aad11dee5356..7e4477c64ffda 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/WritableMemChunk.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/WritableMemChunk.java @@ -21,10 +21,15 @@ import org.apache.iotdb.common.rpc.thrift.TSStatus; import org.apache.iotdb.db.conf.IoTDBConfig; import org.apache.iotdb.db.conf.IoTDBDescriptor; +import org.apache.iotdb.db.queryengine.execution.fragment.FragmentInstanceContext; +import org.apache.iotdb.db.queryengine.execution.fragment.QueryContext; +import org.apache.iotdb.db.queryengine.plan.planner.memory.MemoryReservationManager; import org.apache.iotdb.db.storageengine.dataregion.wal.buffer.IWALByteBufferView; +import org.apache.iotdb.db.utils.datastructure.MergeSortTvListIterator; import org.apache.iotdb.db.utils.datastructure.TVList; import org.apache.tsfile.enums.TSDataType; +import org.apache.tsfile.read.TimeValuePair; import org.apache.tsfile.utils.Binary; import org.apache.tsfile.utils.BitMap; import org.apache.tsfile.write.UnSupportedDataTypeException; @@ -38,12 +43,14 @@ import java.io.DataInputStream; import java.io.IOException; import java.nio.ByteBuffer; +import java.util.ArrayList; import java.util.List; public class WritableMemChunk implements IWritableMemChunk { private IMeasurementSchema schema; private TVList list; + private List sortedList; private static final String UNSUPPORTED_TYPE = "Unsupported data type:"; private static final Logger LOGGER = LoggerFactory.getLogger(WritableMemChunk.class); @@ -52,32 +59,80 @@ public class WritableMemChunk implements IWritableMemChunk { public WritableMemChunk(IMeasurementSchema schema) { this.schema = schema; this.list = TVList.newList(schema.getType()); + this.sortedList = new ArrayList<>(); } private WritableMemChunk() {} + protected void handoverTvList() { + // ensure query contexts won't be removed from list during handover process. + list.lockQueryList(); + try { + if (list.isSorted()) { + sortedList.add(list); + return; + } + + if (list.getQueryContextList().isEmpty()) { + list.sort(); + sortedList.add(list); + } else { + QueryContext firstQuery = list.getQueryContextList().get(0); + // reserve query memory + if (firstQuery instanceof FragmentInstanceContext) { + MemoryReservationManager memoryReservationManager = + ((FragmentInstanceContext) firstQuery).getMemoryReservationContext(); + memoryReservationManager.reserveMemoryCumulatively(list.calculateRamSize()); + } + // update current TVList owner to first query in the list + list.setOwnerQuery(firstQuery); + // clone tv list + TVList cloneList = list.clone(); + sortedList.add(cloneList); + } + } finally { + list.unlockQueryList(); + } + this.list = TVList.newList(schema.getType()); + } + @Override public boolean writeWithFlushCheck(long insertTime, Object objectValue) { + boolean shouldFlush; switch (schema.getType()) { case BOOLEAN: - return putBooleanWithFlushCheck(insertTime, (boolean) objectValue); + shouldFlush = putBooleanWithFlushCheck(insertTime, (boolean) objectValue); + break; case INT32: case DATE: - return putIntWithFlushCheck(insertTime, (int) objectValue); + shouldFlush = putIntWithFlushCheck(insertTime, (int) objectValue); + break; case INT64: case TIMESTAMP: - return putLongWithFlushCheck(insertTime, (long) objectValue); + shouldFlush = putLongWithFlushCheck(insertTime, (long) objectValue); + break; case FLOAT: - return putFloatWithFlushCheck(insertTime, (float) objectValue); + shouldFlush = putFloatWithFlushCheck(insertTime, (float) objectValue); + break; case DOUBLE: - return putDoubleWithFlushCheck(insertTime, (double) objectValue); + shouldFlush = putDoubleWithFlushCheck(insertTime, (double) objectValue); + break; case TEXT: case BLOB: case STRING: - return putBinaryWithFlushCheck(insertTime, (Binary) objectValue); + shouldFlush = putBinaryWithFlushCheck(insertTime, (Binary) objectValue); + break; default: throw new UnSupportedDataTypeException(UNSUPPORTED_TYPE + schema.getType().name()); } + if (shouldFlush) { + return true; + } + + if (list.rowCount() >= SORT_THRESHOLD) { + handoverTvList(); + } + return false; } @Override @@ -89,32 +144,46 @@ public boolean writeAlignedValueWithFlushCheck( @Override public boolean writeWithFlushCheck( long[] times, Object valueList, BitMap bitMap, TSDataType dataType, int start, int end) { + boolean shouldFlush; switch (dataType) { case BOOLEAN: boolean[] boolValues = (boolean[]) valueList; - return putBooleansWithFlushCheck(times, boolValues, bitMap, start, end); + shouldFlush = putBooleansWithFlushCheck(times, boolValues, bitMap, start, end); + break; case INT32: case DATE: int[] intValues = (int[]) valueList; - return putIntsWithFlushCheck(times, intValues, bitMap, start, end); + shouldFlush = putIntsWithFlushCheck(times, intValues, bitMap, start, end); + break; case INT64: case TIMESTAMP: long[] longValues = (long[]) valueList; return putLongsWithFlushCheck(times, longValues, bitMap, start, end); case FLOAT: float[] floatValues = (float[]) valueList; - return putFloatsWithFlushCheck(times, floatValues, bitMap, start, end); + shouldFlush = putFloatsWithFlushCheck(times, floatValues, bitMap, start, end); + break; case DOUBLE: double[] doubleValues = (double[]) valueList; - return putDoublesWithFlushCheck(times, doubleValues, bitMap, start, end); + shouldFlush = putDoublesWithFlushCheck(times, doubleValues, bitMap, start, end); + break; case TEXT: case BLOB: case STRING: Binary[] binaryValues = (Binary[]) valueList; - return putBinariesWithFlushCheck(times, binaryValues, bitMap, start, end); + shouldFlush = putBinariesWithFlushCheck(times, binaryValues, bitMap, start, end); + break; default: throw new UnSupportedDataTypeException(UNSUPPORTED_TYPE + dataType.name()); } + if (shouldFlush) { + return true; + } + + if (list.rowCount() >= SORT_THRESHOLD) { + handoverTvList(); + } + return false; } @Override @@ -241,7 +310,9 @@ private void sortTVList() { @Override public synchronized void sortTvListForFlush() { - sortTVList(); + if (!list.isSorted()) { + list.sort(); + } } @Override @@ -249,9 +320,26 @@ public TVList getTVList() { return list; } + public void setTVList(TVList list) { + this.list = list; + } + @Override public long count() { - return list.rowCount(); + long count = list.count(); + for (TVList tvList : sortedList) { + count += tvList.count(); + } + return count; + } + + @Override + public long rowCount() { + long rowCount = list.rowCount(); + for (TVList tvList : sortedList) { + rowCount += tvList.rowCount(); + } + return rowCount; } @Override @@ -261,35 +349,50 @@ public IMeasurementSchema getSchema() { @Override public long getMaxTime() { - return list.getMaxTime(); + long maxTime = list.getMaxTime(); + for (TVList tvList : sortedList) { + maxTime = Math.max(maxTime, tvList.getMaxTime()); + } + return maxTime; + } + + @Override + public long getMinTime() { + long minTime = list.getMinTime(); + for (TVList tvList : sortedList) { + minTime = Math.min(minTime, tvList.getMinTime()); + } + return minTime; } @Override public long getFirstPoint() { - if (list.rowCount() == 0) { + if (count() == 0) { return Long.MAX_VALUE; } - return getSortedTvListForQuery().getTimeValuePair(0).getTimestamp(); + return getMinTime(); } @Override public long getLastPoint() { - if (list.rowCount() == 0) { + if (count() == 0) { return Long.MIN_VALUE; } - return getSortedTvListForQuery() - .getTimeValuePair(getSortedTvListForQuery().rowCount() - 1) - .getTimestamp(); + return getMaxTime(); } @Override public boolean isEmpty() { - return list.rowCount() == 0; + return count() == 0; } @Override public int delete(long lowerBound, long upperBound) { - return list.delete(lowerBound, upperBound); + int deletedNumber = list.delete(lowerBound, upperBound); + for (TVList tvList : sortedList) { + deletedNumber += tvList.delete(lowerBound, upperBound); + } + return deletedNumber; } @Override @@ -299,96 +402,140 @@ public IChunkWriter createIChunkWriter() { @Override public String toString() { - int size = list.rowCount(); - int firstIndex = 0; - int lastIndex = size - 1; + TimeValuePair firstTvPair = null; + TimeValuePair lastTvPair = null; + int size = 0; long minTime = Long.MAX_VALUE; long maxTime = Long.MIN_VALUE; - for (int i = 0; i < size; i++) { - long currentTime = list.getTime(i); - if (currentTime < minTime) { - firstIndex = i; - minTime = currentTime; - } - if (currentTime >= maxTime) { - lastIndex = i; - maxTime = currentTime; + + List tvLists = new ArrayList<>(sortedList); + tvLists.add(list); + for (TVList tvList : tvLists) { + for (int i = 0; i < tvList.rowCount(); i++) { + if (tvList.isNullValue(i)) { + continue; + } + size++; + long currentTime = list.getTime(i); + if (currentTime < minTime) { + firstTvPair = tvList.getTimeValuePair(i); + minTime = currentTime; + } + if (currentTime >= maxTime) { + lastTvPair = tvList.getTimeValuePair(i); + maxTime = currentTime; + } } } StringBuilder out = new StringBuilder("MemChunk Size: " + size + System.lineSeparator()); if (size != 0) { out.append("Data type:").append(schema.getType()).append(System.lineSeparator()); - out.append("First point:") - .append(list.getTimeValuePair(firstIndex)) - .append(System.lineSeparator()); - out.append("Last point:") - .append(list.getTimeValuePair(lastIndex)) - .append(System.lineSeparator()); + out.append("First point:").append(firstTvPair).append(System.lineSeparator()); + out.append("Last point:").append(lastTvPair).append(System.lineSeparator()); } return out.toString(); } + private void writeData(ChunkWriterImpl chunkWriterImpl, TimeValuePair tvPair) { + switch (schema.getType()) { + case BOOLEAN: + chunkWriterImpl.write(tvPair.getTimestamp(), tvPair.getValue().getBoolean()); + break; + case INT32: + case DATE: + chunkWriterImpl.write(tvPair.getTimestamp(), tvPair.getValue().getInt()); + break; + case INT64: + case TIMESTAMP: + chunkWriterImpl.write(tvPair.getTimestamp(), tvPair.getValue().getLong()); + break; + case FLOAT: + chunkWriterImpl.write(tvPair.getTimestamp(), tvPair.getValue().getFloat()); + break; + case DOUBLE: + chunkWriterImpl.write(tvPair.getTimestamp(), tvPair.getValue().getDouble()); + break; + case TEXT: + case BLOB: + case STRING: + chunkWriterImpl.write(tvPair.getTimestamp(), tvPair.getValue().getBinary()); + break; + default: + LOGGER.error("WritableMemChunk does not support data type: {}", schema.getType()); + break; + } + } + @Override public void encode(IChunkWriter chunkWriter) { - ChunkWriterImpl chunkWriterImpl = (ChunkWriterImpl) chunkWriter; - for (int sortedRowIndex = 0; sortedRowIndex < list.rowCount(); sortedRowIndex++) { - long time = list.getTime(sortedRowIndex); - - TSDataType tsDataType = schema.getType(); + // create MergeSortTvListIterator. It need not handle float/double precision here. + List tvLists = new ArrayList<>(sortedList); + tvLists.add(list); + MergeSortTvListIterator iterator = new MergeSortTvListIterator(schema.getType(), tvLists); - // skip duplicated data - if ((sortedRowIndex + 1 < list.rowCount() && (time == list.getTime(sortedRowIndex + 1)))) { + TimeValuePair prevTvPair = null; + while (iterator.hasNextTimeValuePair()) { + TimeValuePair currTvPair = iterator.nextTimeValuePair(); + if (prevTvPair == null) { + prevTvPair = currTvPair; continue; } + writeData(chunkWriterImpl, prevTvPair); + prevTvPair = currTvPair; + } + // last point for SDT + if (prevTvPair != null) { + chunkWriterImpl.setLastPoint(true); + writeData(chunkWriterImpl, prevTvPair); + } + } - // store last point for SDT - if (sortedRowIndex + 1 == list.rowCount()) { - chunkWriterImpl.setLastPoint(true); - } - - switch (tsDataType) { - case BOOLEAN: - chunkWriterImpl.write(time, list.getBoolean(sortedRowIndex)); - break; - case INT32: - case DATE: - chunkWriterImpl.write(time, list.getInt(sortedRowIndex)); - break; - case INT64: - case TIMESTAMP: - chunkWriterImpl.write(time, list.getLong(sortedRowIndex)); - break; - case FLOAT: - chunkWriterImpl.write(time, list.getFloat(sortedRowIndex)); - break; - case DOUBLE: - chunkWriterImpl.write(time, list.getDouble(sortedRowIndex)); - break; - case TEXT: - case BLOB: - case STRING: - chunkWriterImpl.write(time, list.getBinary(sortedRowIndex)); - break; - default: - LOGGER.error("WritableMemChunk does not support data type: {}", tsDataType); - break; + /** + * Release process for memtable flush. Release the TVList if there is no query on it, otherwise + * set query owner and release the TVList until query finishes. + * + * @param tvList + */ + private void maybeReleaseTvList(TVList tvList) { + tvList.lockQueryList(); + try { + if (tvList.getQueryContextList().isEmpty()) { + tvList.clear(); + } else { + QueryContext firstQuery = tvList.getQueryContextList().get(0); + // transfer memory from write process to read process. Here it reserves read memory and + // releaseFlushedMemTable will release write memory. + if (firstQuery instanceof FragmentInstanceContext) { + MemoryReservationManager memoryReservationManager = + ((FragmentInstanceContext) firstQuery).getMemoryReservationContext(); + memoryReservationManager.reserveMemoryCumulatively(tvList.calculateRamSize()); + } + // update current TVList owner to first query in the list + tvList.setOwnerQuery(firstQuery); } + } finally { + tvList.unlockQueryList(); } } @Override public void release() { - if (list.getReferenceCount() == 0) { - list.clear(); + maybeReleaseTvList(list); + for (TVList tvList : sortedList) { + maybeReleaseTvList(tvList); } } @Override public int serializedSize() { - return schema.serializedSize() + list.serializedSize(); + int serializedSize = schema.serializedSize() + list.serializedSize(); + for (TVList tvList : sortedList) { + serializedSize += tvList.serializedSize(); + } + return serializedSize; } @Override @@ -396,7 +543,9 @@ public void serializeToWAL(IWALByteBufferView buffer) { byte[] bytes = new byte[schema.serializedSize()]; schema.serializeTo(ByteBuffer.wrap(bytes)); buffer.put(bytes); - + for (TVList tvList : sortedList) { + tvList.serializeToWAL(buffer); + } list.serializeToWAL(buffer); } @@ -406,4 +555,8 @@ public static WritableMemChunk deserialize(DataInputStream stream) throws IOExce memChunk.list = TVList.deserialize(stream); return memChunk; } + + public List getSortedList() { + return sortedList; + } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/WritableMemChunkGroup.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/WritableMemChunkGroup.java index b3fade015a38d..9ecd6a46d0fb7 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/WritableMemChunkGroup.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/WritableMemChunkGroup.java @@ -151,7 +151,7 @@ public long getCurrentTVListSize(String measurement) { if (!memChunkMap.containsKey(measurement)) { return 0; } - return memChunkMap.get(measurement).getTVList().rowCount(); + return memChunkMap.get(measurement).rowCount(); } @Override diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemChunkReader.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemChunkReader.java index 7db089288b84e..b007617f0da99 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemChunkReader.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemChunkReader.java @@ -20,21 +20,33 @@ package org.apache.iotdb.db.storageengine.dataregion.read.reader.chunk; import org.apache.iotdb.db.storageengine.dataregion.memtable.ReadOnlyMemChunk; +import org.apache.iotdb.db.utils.datastructure.MergeSortTvListIterator; +import org.apache.iotdb.db.utils.datastructure.TVList; +import org.apache.tsfile.enums.TSDataType; +import org.apache.tsfile.file.metadata.IChunkMetadata; +import org.apache.tsfile.file.metadata.statistics.Statistics; import org.apache.tsfile.read.TimeValuePair; import org.apache.tsfile.read.common.BatchData; +import org.apache.tsfile.read.common.block.TsBlock; +import org.apache.tsfile.read.common.block.TsBlockBuilder; import org.apache.tsfile.read.filter.basic.Filter; import org.apache.tsfile.read.reader.IChunkReader; import org.apache.tsfile.read.reader.IPageReader; import org.apache.tsfile.read.reader.IPointReader; import java.io.IOException; +import java.util.ArrayList; import java.util.Collections; import java.util.List; +import java.util.function.Supplier; + +import static org.apache.iotdb.db.utils.ModificationUtils.isPointDeleted; /** To read chunk data in memory. */ public class MemChunkReader implements IChunkReader, IPointReader { + private final ReadOnlyMemChunk readableChunk; private final IPointReader timeValuePairIterator; private final Filter globalTimeFilter; private final List pageReaderList; @@ -43,13 +55,38 @@ public class MemChunkReader implements IChunkReader, IPointReader { private TimeValuePair cachedTimeValuePair; public MemChunkReader(ReadOnlyMemChunk readableChunk, Filter globalTimeFilter) { - timeValuePairIterator = readableChunk.getPointReader(); + this.readableChunk = readableChunk; + List tvLists = new ArrayList<>(readableChunk.getTvListQueryMap().keySet()); + timeValuePairIterator = + new MergeSortTvListIterator( + readableChunk.getDataType(), + readableChunk.getEncoding(), + readableChunk.getFloatPrecision(), + tvLists); this.globalTimeFilter = globalTimeFilter; - // we treat one ReadOnlyMemChunk as one Page - this.pageReaderList = - Collections.singletonList( - new MemPageReader( - readableChunk.getTsBlock(), readableChunk.getChunkMetaData(), globalTimeFilter)); + this.pageReaderList = new ArrayList<>(); + initAllPageReaders( + readableChunk.getChunkMetaData(), + readableChunk.getPageStatisticsList(), + readableChunk.getPageOffsetsList()); + } + + private void initAllPageReaders( + IChunkMetadata metadata, List pageStats, List pageOffsetsList) { + Supplier tsBlockSupplier = new TsBlockSupplier(); + for (int i = 0; i < pageStats.size(); i++) { + MemPageReader pageReader = + new MemPageReader( + tsBlockSupplier, + (MergeSortTvListIterator) timeValuePairIterator, + pageOffsetsList.get(i), + i < pageStats.size() - 1 ? pageOffsetsList.get(i + 1) : null, + metadata.getDataType(), + metadata.getMeasurementUid(), + pageStats.get(i), + globalTimeFilter); + this.pageReaderList.add(pageReader); + } } @Override @@ -114,4 +151,91 @@ public void close() { public List loadPageReaderList() { return this.pageReaderList; } + + /** + * TsBlockSupplier enables to read pages in MemTable lazily. All MemPageReaders share one + * TsBlockSupplier object. + */ + class TsBlockSupplier implements Supplier { + private int[] pageEndOffsets; + + public TsBlockSupplier() {} + + public void setPageEndOffsets(int[] pageEndOffsets) { + this.pageEndOffsets = pageEndOffsets; + } + + @Override + public TsBlock get() { + return buildTsBlock(); + } + + private TsBlock buildTsBlock() { + try { + TSDataType tsDataType = readableChunk.getDataType(); + TsBlockBuilder builder = new TsBlockBuilder(Collections.singletonList(tsDataType)); + writeValidValuesIntoTsBlock(builder); + return builder.build(); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + private boolean isOutOfMemPageBounds() { + if (pageEndOffsets == null) { + return false; + } + int[] currTvListOffsets = + ((MergeSortTvListIterator) timeValuePairIterator).getTVListOffsets(); + for (int i = 0; i < pageEndOffsets.length; i++) { + if (currTvListOffsets[i] > pageEndOffsets[i]) { + return true; + } + } + return false; + } + + // read one page and write to tsblock + private synchronized void writeValidValuesIntoTsBlock(TsBlockBuilder builder) + throws IOException { + TSDataType tsDataType = readableChunk.getDataType(); + int[] deleteCursor = {0}; + while (timeValuePairIterator.hasNextTimeValuePair()) { + if (isOutOfMemPageBounds()) { + break; + } + TimeValuePair tvPair = timeValuePairIterator.nextTimeValuePair(); + if (!isPointDeleted(tvPair.getTimestamp(), readableChunk.getDeletionList(), deleteCursor)) { + builder.getTimeColumnBuilder().writeLong(tvPair.getTimestamp()); + switch (tsDataType) { + case BOOLEAN: + builder.getColumnBuilder(0).writeBoolean(tvPair.getValue().getBoolean()); + break; + case INT32: + case DATE: + builder.getColumnBuilder(0).writeInt(tvPair.getValue().getInt()); + break; + case INT64: + case TIMESTAMP: + builder.getColumnBuilder(0).writeLong(tvPair.getValue().getLong()); + break; + case FLOAT: + builder.getColumnBuilder(0).writeFloat(tvPair.getValue().getFloat()); + break; + case DOUBLE: + builder.getColumnBuilder(0).writeDouble(tvPair.getValue().getDouble()); + break; + case TEXT: + case STRING: + case BLOB: + builder.getColumnBuilder(0).writeBinary(tvPair.getValue().getBinary()); + break; + default: + break; + } + builder.declarePosition(); + } + } + } + } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemPageReader.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemPageReader.java index b8cb3f9a880ee..d05ea3d105f77 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemPageReader.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemPageReader.java @@ -19,10 +19,12 @@ package org.apache.iotdb.db.storageengine.dataregion.read.reader.chunk; +import org.apache.iotdb.db.storageengine.dataregion.read.reader.chunk.metadata.PageMetadata; +import org.apache.iotdb.db.utils.datastructure.MergeSortTvListIterator; + import org.apache.tsfile.block.column.Column; import org.apache.tsfile.block.column.ColumnBuilder; import org.apache.tsfile.enums.TSDataType; -import org.apache.tsfile.file.metadata.IChunkMetadata; import org.apache.tsfile.file.metadata.statistics.Statistics; import org.apache.tsfile.read.common.BatchData; import org.apache.tsfile.read.common.BatchDataFactory; @@ -40,34 +42,54 @@ import java.util.Collections; import java.util.List; import java.util.Optional; +import java.util.function.Supplier; import static org.apache.tsfile.read.reader.series.PaginationController.UNLIMITED_PAGINATION_CONTROLLER; public class MemPageReader implements IPageReader { - private final TsBlock tsBlock; - private final IChunkMetadata chunkMetadata; - + private TsBlock tsBlock; private Filter recordFilter; + private final MergeSortTvListIterator mergeSortTvListIterator; + // MemPage range - [pageStartOffsets, pageEndOffsets) + private final int[] pageStartOffsets; + private final int[] pageEndOffsets; + private final Supplier tsBlockSupplier; + private final TSDataType tsDataType; + private final PageMetadata pageMetadata; + private PaginationController paginationController = UNLIMITED_PAGINATION_CONTROLLER; - public MemPageReader(TsBlock tsBlock, IChunkMetadata chunkMetadata, Filter recordFilter) { - this.tsBlock = tsBlock; - this.chunkMetadata = chunkMetadata; + public MemPageReader( + Supplier tsBlockSupplier, + MergeSortTvListIterator mergeSortTvListIterator, + int[] pageStartOffsets, + int[] pageEndOffSets, + TSDataType tsDataType, + String measurementUid, + Statistics statistics, + Filter recordFilter) { + this.tsBlockSupplier = tsBlockSupplier; + this.mergeSortTvListIterator = mergeSortTvListIterator; + this.pageStartOffsets = pageStartOffsets; + this.pageEndOffsets = pageEndOffSets; this.recordFilter = recordFilter; + this.tsDataType = tsDataType; + this.pageMetadata = new PageMetadata(measurementUid, tsDataType, statistics); } @Override public BatchData getAllSatisfiedPageData(boolean ascending) throws IOException { - TSDataType dataType = chunkMetadata.getDataType(); - BatchData batchData = BatchDataFactory.createBatchData(dataType, ascending, false); + getTsBlock(); + + BatchData batchData = BatchDataFactory.createBatchData(tsDataType, ascending, false); boolean[] satisfyInfo = buildSatisfyInfoArray(); for (int i = 0; i < tsBlock.getPositionCount(); i++) { if (satisfyInfo[i]) { - switch (dataType) { + switch (tsDataType) { case BOOLEAN: batchData.putBoolean( tsBlock.getTimeColumn().getLong(i), tsBlock.getColumn(0).getBoolean(i)); @@ -95,7 +117,7 @@ public BatchData getAllSatisfiedPageData(boolean ascending) throws IOException { tsBlock.getTimeColumn().getLong(i), tsBlock.getColumn(0).getBinary(i)); break; default: - throw new UnSupportedDataTypeException(String.valueOf(dataType)); + throw new UnSupportedDataTypeException(String.valueOf(tsDataType)); } } } @@ -104,8 +126,9 @@ public BatchData getAllSatisfiedPageData(boolean ascending) throws IOException { @Override public TsBlock getAllSatisfiedData() { - TsBlockBuilder builder = - new TsBlockBuilder(Collections.singletonList(chunkMetadata.getDataType())); + getTsBlock(); + + TsBlockBuilder builder = new TsBlockBuilder(Collections.singletonList(tsDataType)); boolean[] satisfyInfo = buildSatisfyInfoArray(); @@ -176,23 +199,23 @@ private void buildValueColumn(TsBlockBuilder builder, boolean[] satisfyInfo, int @Override public Statistics getStatistics() { - return chunkMetadata.getStatistics(); + return pageMetadata.getStatistics(); } @Override public Statistics getTimeStatistics() { - return chunkMetadata.getTimeStatistics(); + return pageMetadata.getTimeStatistics(); } @Override public Optional> getMeasurementStatistics( int measurementIndex) { - return chunkMetadata.getMeasurementStatistics(measurementIndex); + return pageMetadata.getMeasurementStatistics(measurementIndex); } @Override public boolean hasNullValue(int measurementIndex) { - return chunkMetadata.hasNullValue(measurementIndex); + return pageMetadata.hasNullValue(measurementIndex); } @Override @@ -214,4 +237,73 @@ public boolean isModified() { public void initTsBlockBuilder(List dataTypes) { // non-aligned page reader don't need to init TsBlockBuilder at the very beginning } + + private void getTsBlock() { + if (tsBlock == null) { + InitializeOffsets(); + tsBlock = tsBlockSupplier.get(); + if (pageMetadata.getStatistics() == null) { + initPageStatistics(); + } + } + } + + private void InitializeOffsets() { + if (pageStartOffsets != null) { + mergeSortTvListIterator.setTVListOffsets(pageStartOffsets); + } + ((MemChunkReader.TsBlockSupplier) tsBlockSupplier).setPageEndOffsets(pageEndOffsets); + } + + // memory page statistics should be initialized when constructing ReadOnlyMemChunk object. + // We do the initialization if it is not set, especially in test cases. + private void initPageStatistics() { + Statistics statistics = Statistics.getStatsByType(tsDataType); + updatePageStatisticsFromTsBlock(statistics, tsDataType); + statistics.setEmpty(tsBlock.isEmpty()); + pageMetadata.setStatistics(statistics); + } + + private void updatePageStatisticsFromTsBlock(Statistics statistics, TSDataType dataType) { + if (!tsBlock.isEmpty()) { + switch (dataType) { + case BOOLEAN: + for (int i = 0; i < tsBlock.getPositionCount(); i++) { + statistics.update(tsBlock.getTimeByIndex(i), tsBlock.getColumn(0).getBoolean(i)); + } + break; + case TEXT: + case BLOB: + case STRING: + for (int i = 0; i < tsBlock.getPositionCount(); i++) { + statistics.update(tsBlock.getTimeByIndex(i), tsBlock.getColumn(0).getBinary(i)); + } + break; + case FLOAT: + for (int i = 0; i < tsBlock.getPositionCount(); i++) { + statistics.update(tsBlock.getTimeByIndex(i), tsBlock.getColumn(0).getFloat(i)); + } + break; + case INT32: + case DATE: + for (int i = 0; i < tsBlock.getPositionCount(); i++) { + statistics.update(tsBlock.getTimeByIndex(i), tsBlock.getColumn(0).getInt(i)); + } + break; + case INT64: + case TIMESTAMP: + for (int i = 0; i < tsBlock.getPositionCount(); i++) { + statistics.update(tsBlock.getTimeByIndex(i), tsBlock.getColumn(0).getLong(i)); + } + break; + case DOUBLE: + for (int i = 0; i < tsBlock.getPositionCount(); i++) { + statistics.update(tsBlock.getTimeByIndex(i), tsBlock.getColumn(0).getDouble(i)); + } + break; + default: + // do nothing + } + } + } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/metadata/PageMetadata.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/metadata/PageMetadata.java new file mode 100644 index 0000000000000..d188195baf769 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/metadata/PageMetadata.java @@ -0,0 +1,90 @@ +/* + * 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.storageengine.dataregion.read.reader.chunk.metadata; + +import org.apache.tsfile.enums.TSDataType; +import org.apache.tsfile.file.metadata.IMetadata; +import org.apache.tsfile.file.metadata.statistics.Statistics; + +import java.io.Serializable; +import java.util.Optional; + +import static org.apache.tsfile.utils.Preconditions.checkArgument; + +public class PageMetadata implements IMetadata { + private final String measurementUid; + private final TSDataType tsDataType; + private Statistics statistics; + private boolean modified; + + public PageMetadata( + String measurementUid, TSDataType tsDataType, Statistics statistics) { + this.measurementUid = measurementUid; + this.tsDataType = tsDataType; + this.statistics = statistics; + } + + @Override + public String toString() { + return String.format( + "measurementId: %s, datatype: %s, " + "Statistics: %s", + measurementUid, tsDataType, statistics); + } + + public boolean isModified() { + return modified; + } + + public void setModified(boolean modified) { + this.modified = modified; + } + + public String getMeasurementUid() { + return measurementUid; + } + + public void setStatistics(Statistics statistics) { + this.statistics = statistics; + } + + @Override + public Statistics getStatistics() { + return statistics; + } + + @Override + public Statistics getTimeStatistics() { + return getStatistics(); + } + + @Override + public Optional> getMeasurementStatistics( + int measurementIndex) { + checkArgument( + measurementIndex == 0, + "Non-aligned chunk only has one measurement, but measurementIndex is " + measurementIndex); + return Optional.ofNullable(statistics); + } + + @Override + public boolean hasNullValue(int measurementIndex) { + return false; + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MergeSortTvListIterator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MergeSortTvListIterator.java new file mode 100644 index 0000000000000..223d05ac81a66 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MergeSortTvListIterator.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 org.apache.iotdb.db.utils.MathUtils; + +import org.apache.tsfile.enums.TSDataType; +import org.apache.tsfile.file.metadata.enums.TSEncoding; +import org.apache.tsfile.read.TimeValuePair; +import org.apache.tsfile.read.reader.IPointReader; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +public class MergeSortTvListIterator implements IPointReader { + private final List tvListIterators; + private final TSDataType tsDataType; + private TSEncoding encoding; + private int floatPrecision = -1; + + private TimeValuePair currentTvPair; + + public MergeSortTvListIterator(TSDataType tsDataType, List tvLists) { + this.tsDataType = tsDataType; + tvListIterators = new ArrayList<>(); + for (TVList tvList : tvLists) { + tvListIterators.add(tvList.iterator()); + } + } + + public MergeSortTvListIterator( + TSDataType tsDataType, TSEncoding encoding, int floatPrecision, List tvLists) { + this(tsDataType, tvLists); + this.encoding = encoding; + this.floatPrecision = floatPrecision; + } + + private int getSelectedTVListIndex() { + long time = Long.MAX_VALUE; + int selectedTVListIndex = -1; + for (int i = 0; i < tvListIterators.size(); i++) { + TVList.TVListIterator iterator = tvListIterators.get(i); + TimeValuePair currTvPair = null; + if (iterator.hasNext()) { + currTvPair = iterator.current(); + } + + // update minimum time and remember selected TVList + if (currTvPair != null && currTvPair.getTimestamp() <= time) { + time = currTvPair.getTimestamp(); + selectedTVListIndex = i; + } + } + return selectedTVListIndex; + } + + @Override + public boolean hasNextTimeValuePair() { + boolean hasNext = false; + int selectedTVListIndex = getSelectedTVListIndex(); + if (selectedTVListIndex >= 0) { + currentTvPair = tvListIterators.get(selectedTVListIndex).next(); + hasNext = true; + + // call next to skip identical timestamp in other iterators + for (int i = 0; i < tvListIterators.size(); i++) { + TimeValuePair tvPair = tvListIterators.get(i).current(); + if (tvPair != null && tvPair.getTimestamp() == currentTvPair.getTimestamp()) { + tvListIterators.get(i).next(); + } + } + } + return hasNext; + } + + @Override + public TimeValuePair nextTimeValuePair() { + return currentTimeValuePair(); + } + + @Override + public TimeValuePair currentTimeValuePair() { + if (encoding != null && floatPrecision != -1) { + if (tsDataType == TSDataType.FLOAT) { + float value = currentTvPair.getValue().getFloat(); + if (!Float.isNaN(value) + && (encoding == TSEncoding.RLE || encoding == TSEncoding.TS_2DIFF)) { + currentTvPair + .getValue() + .setFloat(MathUtils.roundWithGivenPrecision(value, floatPrecision)); + } + } else if (tsDataType == TSDataType.DOUBLE) { + double value = currentTvPair.getValue().getDouble(); + if (!Double.isNaN(value) + && (encoding == TSEncoding.RLE || encoding == TSEncoding.TS_2DIFF)) { + currentTvPair + .getValue() + .setDouble(MathUtils.roundWithGivenPrecision(value, floatPrecision)); + } + } + } + return currentTvPair; + } + + @Override + public long getUsedMemorySize() { + return 0; + } + + @Override + public void close() throws IOException { + tvListIterators.clear(); + } + + public int[] getTVListOffsets() { + int size = tvListIterators.size(); + int[] tvListOffsets = new int[size]; + for (int i = 0; i < size; i++) { + tvListOffsets[i] = tvListIterators.get(i).getIndex(); + } + return tvListOffsets; + } + + public void setTVListOffsets(int[] tvListOffsets) { + for (int i = 0; i < tvListIterators.size(); i++) { + tvListIterators.get(i).setIndex(tvListOffsets[i]); + } + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java index 96d0dda659725..78d23c5e98bee 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java @@ -43,6 +43,7 @@ import java.util.Collections; import java.util.List; import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.locks.ReentrantLock; import static org.apache.iotdb.db.storageengine.rescon.memory.PrimitiveArrayManager.ARRAY_SIZE; import static org.apache.tsfile.utils.RamUsageEstimator.NUM_BYTES_ARRAY_HEADER; @@ -70,6 +71,8 @@ public abstract class TVList implements WALEntryValue { // Index relation: arrayIndex -> elementIndex protected List bitMap; + // lock to provide synchronization for query list + private final ReentrantLock queryListLock = new ReentrantLock(); // list of query that this TVList is used protected final List queryContextList; @@ -133,6 +136,10 @@ public static long tvListArrayMemCost(TSDataType type) { return size; } + public long calculateRamSize() { + return timestamps.size() * tvListArrayMemCost(getDataType()); + } + public boolean isSorted() { return sorted; } @@ -151,6 +158,23 @@ public int rowCount() { return rowCount; } + public int seqRowCount() { + return seqRowCount; + } + + public int count() { + if (bitMap == null) { + return rowCount; + } + int count = 0; + for (int row = 0; row < rowCount; row++) { + if (!isNullValue(row)) { + count++; + } + } + return count; + } + public long getTime(int index) { if (index >= rowCount) { throw new ArrayIndexOutOfBoundsException(index); @@ -214,7 +238,7 @@ protected void markNullValue(int arrayIndex, int elementIndex) { * @param rowIndex value index * @return boolean */ - protected boolean isNullValue(int rowIndex) { + public boolean isNullValue(int rowIndex) { if (rowIndex >= rowCount) { return false; } @@ -554,7 +578,71 @@ public void setOwnerQuery(QueryContext queryCtx) { this.ownerQuery = queryCtx; } + public QueryContext getOwnerQuery() { + return ownerQuery; + } + public List getQueryContextList() { return queryContextList; } + + public List getBitMap() { + return bitMap; + } + + public void lockQueryList() { + queryListLock.lock(); + } + + public void unlockQueryList() { + queryListLock.unlock(); + } + + public TVListIterator iterator() { + return new TVListIterator(); + } + + /* TVList Iterator */ + public class TVListIterator { + private int index; + + public TVListIterator() { + index = 0; + } + + public boolean hasNext() { + if (bitMap != null) { + // skip deleted & duplicated timestamp + while ((index < rowCount && isNullValue(getValueIndex(index))) + || (index + 1 < rowCount && getTime(index + 1) == getTime(index))) { + index++; + } + } else { + // skip duplicated timestamp + while (index + 1 < rowCount && getTime(index + 1) == getTime(index)) { + index++; + } + } + return index < rowCount; + } + + public TimeValuePair next() { + return getTimeValuePair(index++); + } + + public TimeValuePair current() { + if (index >= rowCount || isNullValue(getValueIndex(index))) { + return null; + } + return getTimeValuePair(index); + } + + public int getIndex() { + return index; + } + + public void setIndex(int index) { + this.index = index; + } + } } diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemChunkLoaderTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemChunkLoaderTest.java index a9bdc453d7a6a..944ab4af2a365 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemChunkLoaderTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemChunkLoaderTest.java @@ -21,6 +21,7 @@ import org.apache.iotdb.db.queryengine.execution.fragment.QueryContext; import org.apache.iotdb.db.storageengine.dataregion.memtable.ReadOnlyMemChunk; +import org.apache.iotdb.db.utils.datastructure.TVList; import org.apache.tsfile.common.conf.TSFileConfig; import org.apache.tsfile.enums.TSDataType; @@ -28,7 +29,6 @@ import org.apache.tsfile.file.metadata.statistics.Statistics; import org.apache.tsfile.read.common.BatchData; import org.apache.tsfile.read.common.block.TsBlock; -import org.apache.tsfile.read.common.block.TsBlockBuilder; import org.apache.tsfile.read.reader.IPageReader; import org.apache.tsfile.utils.Binary; import org.junit.Test; @@ -36,7 +36,9 @@ import java.io.IOException; import java.util.Collections; +import java.util.LinkedHashMap; import java.util.List; +import java.util.Map; import static org.apache.tsfile.read.reader.series.PaginationController.UNLIMITED_PAGINATION_CONTROLLER; import static org.junit.Assert.assertEquals; @@ -51,6 +53,8 @@ public class MemChunkLoaderTest { @Test public void testBooleanMemChunkLoader() throws IOException { ReadOnlyMemChunk chunk = Mockito.mock(ReadOnlyMemChunk.class); + Mockito.when(chunk.getDataType()).thenReturn(TSDataType.BOOLEAN); + Mockito.when(chunk.getTvListQueryMap()).thenReturn(buildBooleanTvListMap()); ChunkMetadata chunkMetadata = Mockito.mock(ChunkMetadata.class); MemChunkLoader memChunkLoader = new MemChunkLoader(new QueryContext(), chunk); @@ -63,7 +67,6 @@ public void testBooleanMemChunkLoader() throws IOException { ChunkMetadata chunkMetadata1 = Mockito.mock(ChunkMetadata.class); - Mockito.when(chunk.getTsBlock()).thenReturn(buildBooleanTsBlock()); Mockito.when(chunk.getChunkMetaData()).thenReturn(chunkMetadata1); Mockito.when(chunk.getPointReader()).thenReturn(null); Statistics statistics = Mockito.mock(Statistics.class); @@ -105,20 +108,22 @@ public void testBooleanMemChunkLoader() throws IOException { memChunkLoader.close(); } - private TsBlock buildBooleanTsBlock() { - TsBlockBuilder builder = new TsBlockBuilder(Collections.singletonList(TSDataType.BOOLEAN)); - builder.getTimeColumnBuilder().writeLong(1L); - builder.getColumnBuilder(0).writeBoolean(true); - builder.declarePosition(); - builder.getTimeColumnBuilder().writeLong(2L); - builder.getColumnBuilder(0).writeBoolean(false); - builder.declarePosition(); - return builder.build(); + private Map buildBooleanTvListMap() { + TVList tvList = TVList.newList(TSDataType.BOOLEAN); + if (tvList != null) { + tvList.putBoolean(1L, true); + tvList.putBoolean(2L, true); + } + Map tvListMap = new LinkedHashMap<>(); + tvListMap.put(tvList, 2); + return tvListMap; } @Test public void testInt32MemChunkLoader() throws IOException { ReadOnlyMemChunk chunk = Mockito.mock(ReadOnlyMemChunk.class); + Mockito.when(chunk.getDataType()).thenReturn(TSDataType.INT32); + Mockito.when(chunk.getTvListQueryMap()).thenReturn(buildInt32TvListMap()); ChunkMetadata chunkMetadata = Mockito.mock(ChunkMetadata.class); MemChunkLoader memChunkLoader = new MemChunkLoader(new QueryContext(), chunk); @@ -131,7 +136,6 @@ public void testInt32MemChunkLoader() throws IOException { ChunkMetadata chunkMetadata1 = Mockito.mock(ChunkMetadata.class); - Mockito.when(chunk.getTsBlock()).thenReturn(buildInt32TsBlock()); Mockito.when(chunk.getChunkMetaData()).thenReturn(chunkMetadata1); Mockito.when(chunk.getPointReader()).thenReturn(null); Statistics statistics = Mockito.mock(Statistics.class); @@ -173,20 +177,22 @@ public void testInt32MemChunkLoader() throws IOException { memChunkLoader.close(); } - private TsBlock buildInt32TsBlock() { - TsBlockBuilder builder = new TsBlockBuilder(Collections.singletonList(TSDataType.INT32)); - builder.getTimeColumnBuilder().writeLong(1L); - builder.getColumnBuilder(0).writeInt(1); - builder.declarePosition(); - builder.getTimeColumnBuilder().writeLong(2L); - builder.getColumnBuilder(0).writeInt(2); - builder.declarePosition(); - return builder.build(); + private Map buildInt32TvListMap() { + TVList tvList = TVList.newList(TSDataType.INT32); + if (tvList != null) { + tvList.putInt(1L, 1); + tvList.putInt(2L, 2); + } + Map tvListMap = new LinkedHashMap<>(); + tvListMap.put(tvList, 2); + return tvListMap; } @Test public void testInt64MemChunkLoader() throws IOException { ReadOnlyMemChunk chunk = Mockito.mock(ReadOnlyMemChunk.class); + Mockito.when(chunk.getDataType()).thenReturn(TSDataType.INT64); + Mockito.when(chunk.getTvListQueryMap()).thenReturn(buildInt64TvListMap()); ChunkMetadata chunkMetadata = Mockito.mock(ChunkMetadata.class); MemChunkLoader memChunkLoader = new MemChunkLoader(new QueryContext(), chunk); @@ -199,7 +205,6 @@ public void testInt64MemChunkLoader() throws IOException { ChunkMetadata chunkMetadata1 = Mockito.mock(ChunkMetadata.class); - Mockito.when(chunk.getTsBlock()).thenReturn(buildInt64TsBlock()); Mockito.when(chunk.getChunkMetaData()).thenReturn(chunkMetadata1); Mockito.when(chunk.getPointReader()).thenReturn(null); Statistics statistics = Mockito.mock(Statistics.class); @@ -241,20 +246,22 @@ public void testInt64MemChunkLoader() throws IOException { memChunkLoader.close(); } - private TsBlock buildInt64TsBlock() { - TsBlockBuilder builder = new TsBlockBuilder(Collections.singletonList(TSDataType.INT64)); - builder.getTimeColumnBuilder().writeLong(1L); - builder.getColumnBuilder(0).writeLong(1L); - builder.declarePosition(); - builder.getTimeColumnBuilder().writeLong(2L); - builder.getColumnBuilder(0).writeLong(2L); - builder.declarePosition(); - return builder.build(); + private Map buildInt64TvListMap() { + TVList tvList = TVList.newList(TSDataType.INT64); + if (tvList != null) { + tvList.putLong(1L, 1L); + tvList.putLong(2L, 2L); + } + Map tvListMap = new LinkedHashMap<>(); + tvListMap.put(tvList, 2); + return tvListMap; } @Test public void testFloatMemChunkLoader() throws IOException { ReadOnlyMemChunk chunk = Mockito.mock(ReadOnlyMemChunk.class); + Mockito.when(chunk.getDataType()).thenReturn(TSDataType.FLOAT); + Mockito.when(chunk.getTvListQueryMap()).thenReturn(buildFloatTvListMap()); ChunkMetadata chunkMetadata = Mockito.mock(ChunkMetadata.class); MemChunkLoader memChunkLoader = new MemChunkLoader(new QueryContext(), chunk); @@ -267,7 +274,6 @@ public void testFloatMemChunkLoader() throws IOException { ChunkMetadata chunkMetadata1 = Mockito.mock(ChunkMetadata.class); - Mockito.when(chunk.getTsBlock()).thenReturn(buildFloatTsBlock()); Mockito.when(chunk.getChunkMetaData()).thenReturn(chunkMetadata1); Mockito.when(chunk.getPointReader()).thenReturn(null); Statistics statistics = Mockito.mock(Statistics.class); @@ -309,20 +315,22 @@ public void testFloatMemChunkLoader() throws IOException { memChunkLoader.close(); } - private TsBlock buildFloatTsBlock() { - TsBlockBuilder builder = new TsBlockBuilder(Collections.singletonList(TSDataType.FLOAT)); - builder.getTimeColumnBuilder().writeLong(1L); - builder.getColumnBuilder(0).writeFloat(1.1f); - builder.declarePosition(); - builder.getTimeColumnBuilder().writeLong(2L); - builder.getColumnBuilder(0).writeFloat(2.1f); - builder.declarePosition(); - return builder.build(); + private Map buildFloatTvListMap() { + TVList tvList = TVList.newList(TSDataType.FLOAT); + if (tvList != null) { + tvList.putFloat(1L, 1.1f); + tvList.putFloat(2L, 2.1f); + } + Map tvListMap = new LinkedHashMap<>(); + tvListMap.put(tvList, 2); + return tvListMap; } @Test public void testDoubleMemChunkLoader() throws IOException { ReadOnlyMemChunk chunk = Mockito.mock(ReadOnlyMemChunk.class); + Mockito.when(chunk.getDataType()).thenReturn(TSDataType.DOUBLE); + Mockito.when(chunk.getTvListQueryMap()).thenReturn(buildDoubleTvListMap()); ChunkMetadata chunkMetadata = Mockito.mock(ChunkMetadata.class); MemChunkLoader memChunkLoader = new MemChunkLoader(new QueryContext(), chunk); @@ -335,7 +343,6 @@ public void testDoubleMemChunkLoader() throws IOException { ChunkMetadata chunkMetadata1 = Mockito.mock(ChunkMetadata.class); - Mockito.when(chunk.getTsBlock()).thenReturn(buildDoubleTsBlock()); Mockito.when(chunk.getChunkMetaData()).thenReturn(chunkMetadata1); Mockito.when(chunk.getPointReader()).thenReturn(null); Statistics statistics = Mockito.mock(Statistics.class); @@ -377,20 +384,22 @@ public void testDoubleMemChunkLoader() throws IOException { memChunkLoader.close(); } - private TsBlock buildDoubleTsBlock() { - TsBlockBuilder builder = new TsBlockBuilder(Collections.singletonList(TSDataType.DOUBLE)); - builder.getTimeColumnBuilder().writeLong(1L); - builder.getColumnBuilder(0).writeDouble(1.1d); - builder.declarePosition(); - builder.getTimeColumnBuilder().writeLong(2L); - builder.getColumnBuilder(0).writeDouble(2.1d); - builder.declarePosition(); - return builder.build(); + private Map buildDoubleTvListMap() { + TVList tvList = TVList.newList(TSDataType.DOUBLE); + if (tvList != null) { + tvList.putDouble(1L, 1.1d); + tvList.putDouble(2L, 2.1d); + } + Map tvListMap = new LinkedHashMap<>(); + tvListMap.put(tvList, 2); + return tvListMap; } @Test public void testTextMemChunkLoader() throws IOException { ReadOnlyMemChunk chunk = Mockito.mock(ReadOnlyMemChunk.class); + Mockito.when(chunk.getDataType()).thenReturn(TSDataType.TEXT); + Mockito.when(chunk.getTvListQueryMap()).thenReturn(buildTextTvListMap()); ChunkMetadata chunkMetadata = Mockito.mock(ChunkMetadata.class); MemChunkLoader memChunkLoader = new MemChunkLoader(new QueryContext(), chunk); @@ -403,7 +412,6 @@ public void testTextMemChunkLoader() throws IOException { ChunkMetadata chunkMetadata1 = Mockito.mock(ChunkMetadata.class); - Mockito.when(chunk.getTsBlock()).thenReturn(buildTextTsBlock()); Mockito.when(chunk.getChunkMetaData()).thenReturn(chunkMetadata1); Mockito.when(chunk.getPointReader()).thenReturn(null); Statistics statistics = Mockito.mock(Statistics.class); @@ -445,14 +453,14 @@ public void testTextMemChunkLoader() throws IOException { memChunkLoader.close(); } - private TsBlock buildTextTsBlock() { - TsBlockBuilder builder = new TsBlockBuilder(Collections.singletonList(TSDataType.TEXT)); - builder.getTimeColumnBuilder().writeLong(1L); - builder.getColumnBuilder(0).writeBinary(new Binary(BINARY_STR, TSFileConfig.STRING_CHARSET)); - builder.declarePosition(); - builder.getTimeColumnBuilder().writeLong(2L); - builder.getColumnBuilder(0).writeBinary(new Binary(BINARY_STR, TSFileConfig.STRING_CHARSET)); - builder.declarePosition(); - return builder.build(); + private Map buildTextTvListMap() { + TVList tvList = TVList.newList(TSDataType.TEXT); + if (tvList != null) { + tvList.putBinary(1L, new Binary(BINARY_STR, TSFileConfig.STRING_CHARSET)); + tvList.putBinary(2L, new Binary(BINARY_STR, TSFileConfig.STRING_CHARSET)); + } + Map tvListMap = new LinkedHashMap<>(); + tvListMap.put(tvList, 2); + return tvListMap; } } diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemPageReaderTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemPageReaderTest.java index f9a13a71ad88e..695c0b2f195ef 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemPageReaderTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemPageReaderTest.java @@ -35,6 +35,7 @@ import java.io.IOException; import java.util.Collections; import java.util.Optional; +import java.util.function.Supplier; import static org.apache.tsfile.read.filter.factory.ValueFilterApi.DEFAULT_MEASUREMENT_INDEX; @@ -61,7 +62,16 @@ public class MemPageReaderTest { } private MemPageReader generatePageReader() { - return new MemPageReader(tsBlock, chunkMetadata, null); + Supplier tsBlockSupplier = () -> tsBlock; + return new MemPageReader( + tsBlockSupplier, + null, + null, + null, + chunkMetadata.getDataType(), + chunkMetadata.getMeasurementUid(), + null, + null); } @Test From 6df2e10983f0d3ffdefea2934c919e8a9a782e24 Mon Sep 17 00:00:00 2001 From: shizy Date: Sun, 1 Dec 2024 17:31:14 +0800 Subject: [PATCH 04/54] copy-on-write array list --- .../schemaregion/utils/ResourceByPathUtils.java | 1 + .../iotdb/db/utils/datastructure/BinaryTVList.java | 4 ++-- .../iotdb/db/utils/datastructure/BooleanTVList.java | 4 ++-- .../iotdb/db/utils/datastructure/DoubleTVList.java | 4 ++-- .../iotdb/db/utils/datastructure/FloatTVList.java | 4 ++-- .../apache/iotdb/db/utils/datastructure/IntTVList.java | 4 ++-- .../iotdb/db/utils/datastructure/LongTVList.java | 4 ++-- .../utils/datastructure/MergeSortTvListIterator.java | 1 - .../apache/iotdb/db/utils/datastructure/TVList.java | 10 ++++++---- 9 files changed, 19 insertions(+), 17 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/utils/ResourceByPathUtils.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/utils/ResourceByPathUtils.java index f72f928d4e222..e3e9abc044685 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/utils/ResourceByPathUtils.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/utils/ResourceByPathUtils.java @@ -367,6 +367,7 @@ public ITimeSeriesMetadata generateTimeSeriesMetadata( return timeSeriesMetadata; } + // TODO: global time filter pushdown private Map prepareTvListMapForQuery( WritableMemChunk memChunk, boolean isWorkMemTable, QueryContext context) { Map tvListQueryMap = new LinkedHashMap<>(); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BinaryTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BinaryTVList.java index a8cef34e7b68a..edf6b3ce3d34b 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BinaryTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BinaryTVList.java @@ -34,8 +34,8 @@ import java.io.DataInputStream; import java.io.IOException; -import java.util.ArrayList; import java.util.List; +import java.util.concurrent.CopyOnWriteArrayList; import java.util.stream.IntStream; import static org.apache.iotdb.db.storageengine.rescon.memory.PrimitiveArrayManager.ARRAY_SIZE; @@ -53,7 +53,7 @@ public abstract class BinaryTVList extends TVList { BinaryTVList() { super(); - values = new ArrayList<>(); + values = new CopyOnWriteArrayList<>(); memoryBinaryChunkSize = 0; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BooleanTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BooleanTVList.java index 839544d5713c8..aa7e67da5ebf7 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BooleanTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BooleanTVList.java @@ -33,8 +33,8 @@ import java.io.DataInputStream; import java.io.IOException; -import java.util.ArrayList; import java.util.List; +import java.util.concurrent.CopyOnWriteArrayList; import java.util.stream.IntStream; import static org.apache.iotdb.db.storageengine.rescon.memory.PrimitiveArrayManager.ARRAY_SIZE; @@ -48,7 +48,7 @@ public abstract class BooleanTVList extends TVList { BooleanTVList() { super(); - values = new ArrayList<>(); + values = new CopyOnWriteArrayList<>(); } public static BooleanTVList newList() { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/DoubleTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/DoubleTVList.java index d15851e54deea..0e2080933f74e 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/DoubleTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/DoubleTVList.java @@ -34,8 +34,8 @@ import java.io.DataInputStream; import java.io.IOException; -import java.util.ArrayList; import java.util.List; +import java.util.concurrent.CopyOnWriteArrayList; import java.util.stream.IntStream; import static org.apache.iotdb.db.storageengine.rescon.memory.PrimitiveArrayManager.ARRAY_SIZE; @@ -49,7 +49,7 @@ public abstract class DoubleTVList extends TVList { DoubleTVList() { super(); - values = new ArrayList<>(); + values = new CopyOnWriteArrayList<>(); } public static DoubleTVList newList() { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/FloatTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/FloatTVList.java index 3aea905fa3f15..e934f8fe4c378 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/FloatTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/FloatTVList.java @@ -34,8 +34,8 @@ import java.io.DataInputStream; import java.io.IOException; -import java.util.ArrayList; import java.util.List; +import java.util.concurrent.CopyOnWriteArrayList; import java.util.stream.IntStream; import static org.apache.iotdb.db.storageengine.rescon.memory.PrimitiveArrayManager.ARRAY_SIZE; @@ -49,7 +49,7 @@ public abstract class FloatTVList extends TVList { FloatTVList() { super(); - values = new ArrayList<>(); + values = new CopyOnWriteArrayList<>(); } public static FloatTVList newList() { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/IntTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/IntTVList.java index eedf412a40a6e..30fdf7b3b4cb9 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/IntTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/IntTVList.java @@ -33,8 +33,8 @@ import java.io.DataInputStream; import java.io.IOException; -import java.util.ArrayList; import java.util.List; +import java.util.concurrent.CopyOnWriteArrayList; import java.util.stream.IntStream; import static org.apache.iotdb.db.storageengine.rescon.memory.PrimitiveArrayManager.ARRAY_SIZE; @@ -48,7 +48,7 @@ public abstract class IntTVList extends TVList { IntTVList() { super(); - values = new ArrayList<>(); + values = new CopyOnWriteArrayList<>(); } public static IntTVList newList() { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/LongTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/LongTVList.java index 73e2d792c5192..8d49e739bef3e 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/LongTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/LongTVList.java @@ -33,8 +33,8 @@ import java.io.DataInputStream; import java.io.IOException; -import java.util.ArrayList; import java.util.List; +import java.util.concurrent.CopyOnWriteArrayList; import java.util.stream.IntStream; import static org.apache.iotdb.db.storageengine.rescon.memory.PrimitiveArrayManager.ARRAY_SIZE; @@ -48,7 +48,7 @@ public abstract class LongTVList extends TVList { LongTVList() { super(); - values = new ArrayList<>(); + values = new CopyOnWriteArrayList<>(); } public static LongTVList newList() { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MergeSortTvListIterator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MergeSortTvListIterator.java index 223d05ac81a66..156e044fa0d29 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MergeSortTvListIterator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MergeSortTvListIterator.java @@ -28,7 +28,6 @@ import java.io.IOException; import java.util.ArrayList; -import java.util.Arrays; import java.util.List; public class MergeSortTvListIterator implements IPointReader { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java index 78d23c5e98bee..75e68bf094138 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java @@ -42,6 +42,7 @@ import java.util.ArrayList; import java.util.Collections; import java.util.List; +import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.locks.ReentrantLock; @@ -89,8 +90,8 @@ public abstract class TVList implements WALEntryValue { private long version; protected TVList() { - timestamps = new ArrayList<>(); - indices = new ArrayList<>(); + timestamps = new CopyOnWriteArrayList<>(); + indices = new CopyOnWriteArrayList<>(); rowCount = 0; seqRowCount = 0; maxTime = Long.MIN_VALUE; @@ -123,6 +124,7 @@ public static TVList newList(TSDataType dataType) { return null; } + // TODO: memory cost for indices and bitmap public static long tvListArrayMemCost(TSDataType type) { long size = 0; // time array mem size @@ -218,7 +220,7 @@ public int getValueIndex(int index) { protected void markNullValue(int arrayIndex, int elementIndex) { // init bitMap if doesn't have if (bitMap == null) { - bitMap = new ArrayList<>(); + bitMap = new CopyOnWriteArrayList<>(); for (int i = 0; i < timestamps.size(); i++) { bitMap.add(new BitMap(ARRAY_SIZE)); } @@ -257,7 +259,7 @@ protected void cloneSlicesAndBitMap(TVList cloneList) { } } if (bitMap != null) { - cloneList.bitMap = new ArrayList<>(); + cloneList.bitMap = new CopyOnWriteArrayList<>(); for (BitMap bm : bitMap) { cloneList.bitMap.add(bm == null ? null : bm.clone()); } From 435d5da2b109099a50ebc6a4c431815ab50b37e3 Mon Sep 17 00:00:00 2001 From: shizy Date: Tue, 3 Dec 2024 20:55:20 +0800 Subject: [PATCH 05/54] review comments part 1 --- .../execution/fragment/FragmentInstanceContext.java | 1 + .../schemaregion/utils/ResourceByPathUtils.java | 4 ++-- .../dataregion/memtable/AlignedWritableMemChunk.java | 2 +- .../memtable/AlignedWritableMemChunkGroup.java | 2 +- .../dataregion/memtable/IWritableMemChunk.java | 2 +- .../dataregion/memtable/ReadOnlyMemChunk.java | 8 ++++---- .../dataregion/memtable/TsFileProcessor.java | 8 +++++--- .../dataregion/memtable/WritableMemChunk.java | 4 ++-- .../dataregion/read/reader/chunk/MemPageReader.java | 7 ++++--- .../apache/iotdb/db/utils/datastructure/TVList.java | 11 ++++++----- .../dataregion/memtable/PrimitiveMemTableTest.java | 2 +- 11 files changed, 28 insertions(+), 23 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceContext.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceContext.java index 4dea7f584cd42..187c0a9065562 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceContext.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceContext.java @@ -664,6 +664,7 @@ private void releaseTVListOwnedByQuery() { tvList, this.getId()); tvList.clear(); + memoryReservationManager.releaseMemoryCumulatively(tvList.calculateRamSize()); } else { LOGGER.debug( "TVList {} is owned by another query, FragmentInstance Id is {}", diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/utils/ResourceByPathUtils.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/utils/ResourceByPathUtils.java index e3e9abc044685..45b98956ffa36 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/utils/ResourceByPathUtils.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/utils/ResourceByPathUtils.java @@ -385,7 +385,7 @@ private Map prepareTvListMapForQuery( } // mutable tvlist - TVList list = memChunk.getTVList(); + TVList list = memChunk.getWorkingTVList(); list.lockQueryList(); try { if (!isWorkMemTable) { @@ -415,7 +415,7 @@ private Map prepareTvListMapForQuery( TVList cloneList = list.clone(); cloneList.getQueryContextList().add(context); tvListQueryMap.put(cloneList, cloneList.rowCount()); - memChunk.setTVList(cloneList); + memChunk.setWorkingTVList(cloneList); } } } finally { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedWritableMemChunk.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedWritableMemChunk.java index 74c4682de6a0e..8cabea95fe2df 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedWritableMemChunk.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedWritableMemChunk.java @@ -241,7 +241,7 @@ private Pair checkAndReorderColumnValuesInInsertPlan( } @Override - public TVList getTVList() { + public TVList getWorkingTVList() { return list; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedWritableMemChunkGroup.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedWritableMemChunkGroup.java index 4e079b6178704..efe64bf5b6e1a 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedWritableMemChunkGroup.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedWritableMemChunkGroup.java @@ -129,7 +129,7 @@ public long deleteTime(ModEntry modEntry) { @Override public long getCurrentTVListSize(String measurement) { - return memChunk.getTVList().rowCount(); + return memChunk.getWorkingTVList().rowCount(); } @Override diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/IWritableMemChunk.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/IWritableMemChunk.java index ced8d8643b0fc..8cc10502c7f95 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/IWritableMemChunk.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/IWritableMemChunk.java @@ -131,7 +131,7 @@ default long rowCount() { */ void sortTvListForFlush(); - default TVList getTVList() { + default TVList getWorkingTVList() { return null; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/ReadOnlyMemChunk.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/ReadOnlyMemChunk.java index 9e6e43e62e4f8..2b3d88f1eac91 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/ReadOnlyMemChunk.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/ReadOnlyMemChunk.java @@ -19,6 +19,7 @@ package org.apache.iotdb.db.storageengine.dataregion.memtable; +import org.apache.iotdb.commons.utils.TestOnly; import org.apache.iotdb.db.exception.query.QueryProcessException; import org.apache.iotdb.db.queryengine.execution.fragment.QueryContext; import org.apache.iotdb.db.storageengine.dataregion.read.reader.chunk.MemChunkLoader; @@ -201,9 +202,9 @@ private void updateChunkAndPageStatisticsFromTvLists() { // do nothing } pageStatistics.setEmpty(false); + tvListOffsets = timeValuePairIterator.getTVListOffsets(); + cnt++; } - tvListOffsets = timeValuePairIterator.getTVListOffsets(); - cnt++; } chunkStatistics.setEmpty(cnt == 0); } @@ -223,8 +224,7 @@ public IChunkMetadata getChunkMetaData() { return cachedMetaData; } - // we do not call getPointReader in MemChunkReader anymore. However, unit testcases - // still test this method. + @TestOnly public IPointReader getPointReader() { for (Map.Entry entry : tvListQueryMap.entrySet()) { TVList tvList = entry.getKey(); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/TsFileProcessor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/TsFileProcessor.java index f5c176521aad5..8f02c0cabee17 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/TsFileProcessor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/TsFileProcessor.java @@ -768,7 +768,8 @@ private long[] checkAlignedMemCostAndAddToTspInfoForRow( } // this insertion will result in a new array if ((alignedMemChunk.alignedListSize() % PrimitiveArrayManager.ARRAY_SIZE) == 0) { - dataTypesInTVList.addAll(((AlignedTVList) alignedMemChunk.getTVList()).getTsDataTypes()); + dataTypesInTVList.addAll( + ((AlignedTVList) alignedMemChunk.getWorkingTVList()).getTsDataTypes()); memTableIncrement += AlignedTVList.alignedTvListArrayMemCost(dataTypesInTVList); } } @@ -860,7 +861,7 @@ private long[] checkAlignedMemCostAndAddToTspInfoForRows(List ins if (((currentChunkPointNum + addingPointNum) % PrimitiveArrayManager.ARRAY_SIZE) == 0) { if (alignedMemChunk != null) { dataTypesInTVList.addAll( - ((AlignedTVList) alignedMemChunk.getTVList()).getTsDataTypes()); + ((AlignedTVList) alignedMemChunk.getWorkingTVList()).getTsDataTypes()); } dataTypesInTVList.addAll(addingPointNumInfo.left.values()); memTableIncrement += AlignedTVList.alignedTvListArrayMemCost(dataTypesInTVList); @@ -1072,7 +1073,8 @@ private void updateAlignedMemCost( if (acquireArray != 0) { // memory of extending the TVList - dataTypesInTVList.addAll(((AlignedTVList) alignedMemChunk.getTVList()).getTsDataTypes()); + dataTypesInTVList.addAll( + ((AlignedTVList) alignedMemChunk.getWorkingTVList()).getTsDataTypes()); memIncrements[0] += acquireArray * AlignedTVList.alignedTvListArrayMemCost(dataTypesInTVList); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/WritableMemChunk.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/WritableMemChunk.java index 7e4477c64ffda..a27c5afd28bef 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/WritableMemChunk.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/WritableMemChunk.java @@ -316,11 +316,11 @@ public synchronized void sortTvListForFlush() { } @Override - public TVList getTVList() { + public TVList getWorkingTVList() { return list; } - public void setTVList(TVList list) { + public void setWorkingTVList(TVList list) { this.list = list; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemPageReader.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemPageReader.java index d05ea3d105f77..8cb82a735b073 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemPageReader.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemPageReader.java @@ -240,7 +240,7 @@ public void initTsBlockBuilder(List dataTypes) { private void getTsBlock() { if (tsBlock == null) { - InitializeOffsets(); + initializeOffsets(); tsBlock = tsBlockSupplier.get(); if (pageMetadata.getStatistics() == null) { initPageStatistics(); @@ -248,7 +248,7 @@ private void getTsBlock() { } } - private void InitializeOffsets() { + private void initializeOffsets() { if (pageStartOffsets != null) { mergeSortTvListIterator.setTVListOffsets(pageStartOffsets); } @@ -302,7 +302,8 @@ private void updatePageStatisticsFromTsBlock(Statistics statistics, TSDataType d } break; default: - // do nothing + throw new UnSupportedDataTypeException( + String.format("Data type %s is not supported.", dataType)); } } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java index 75e68bf094138..78fa143b4d7a9 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java @@ -40,6 +40,7 @@ import java.io.DataInputStream; import java.io.IOException; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collections; import java.util.List; import java.util.concurrent.CopyOnWriteArrayList; @@ -196,11 +197,8 @@ protected void set(int index, long timestamp, int value) { indices.get(arrayIndex)[elementIndex] = value; } - @SuppressWarnings("squid:S3776") // Suppress high Cognitive Complexity warning protected int[] cloneIndex(int[] array) { - int[] cloneArray = new int[array.length]; - System.arraycopy(array, 0, cloneArray, 0, array.length); - return cloneArray; + return Arrays.copyOf(array, array.length); } /** @@ -242,7 +240,7 @@ protected void markNullValue(int arrayIndex, int elementIndex) { */ public boolean isNullValue(int rowIndex) { if (rowIndex >= rowCount) { - return false; + throw new IndexOutOfBoundsException("Index out of bound error!"); } if (bitMap == null || bitMap.get(rowIndex / ARRAY_SIZE) == null) { return false; @@ -629,6 +627,9 @@ public boolean hasNext() { } public TimeValuePair next() { + if (!hasNext()) { + return null; + } return getTimeValuePair(index++); } diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/memtable/PrimitiveMemTableTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/memtable/PrimitiveMemTableTest.java index 57c2225ab643b..582666a6549e8 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/memtable/PrimitiveMemTableTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/memtable/PrimitiveMemTableTest.java @@ -127,7 +127,7 @@ public void memSeriesToStringTest() throws IOException { series.writeWithFlushCheck(99, 20); series.writeWithFlushCheck(20, 21); String str = series.toString(); - Assert.assertFalse(series.getTVList().isSorted()); + Assert.assertFalse(series.getWorkingTVList().isSorted()); Assert.assertEquals( "MemChunk Size: 103" + System.lineSeparator() From d5fad222ae14304b3e8356ccc460c4698de720bd Mon Sep 17 00:00:00 2001 From: shizy Date: Tue, 3 Dec 2024 22:26:43 +0800 Subject: [PATCH 06/54] fix unit test errors --- .../read/reader/chunk/MemChunkLoaderTest.java | 123 ++++++++++++------ .../utils/datastructure/BinaryTVListTest.java | 2 +- 2 files changed, 81 insertions(+), 44 deletions(-) diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemChunkLoaderTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemChunkLoaderTest.java index 944ab4af2a365..1f18cc58652ea 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemChunkLoaderTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemChunkLoaderTest.java @@ -35,6 +35,7 @@ import org.mockito.Mockito; import java.io.IOException; +import java.util.Arrays; import java.util.Collections; import java.util.LinkedHashMap; import java.util.List; @@ -65,12 +66,17 @@ public void testBooleanMemChunkLoader() throws IOException { assertNull(e.getMessage()); } - ChunkMetadata chunkMetadata1 = Mockito.mock(ChunkMetadata.class); + Statistics pageStatitstics = Mockito.mock(Statistics.class); + List pageStats = Arrays.asList(pageStatitstics); + List pageOffsets = Arrays.asList(new int[] {0, 0}); + Mockito.when(chunk.getPageStatisticsList()).thenReturn(pageStats); + Mockito.when(chunk.getPageOffsetsList()).thenReturn(pageOffsets); + ChunkMetadata chunkMetadata1 = Mockito.mock(ChunkMetadata.class); Mockito.when(chunk.getChunkMetaData()).thenReturn(chunkMetadata1); Mockito.when(chunk.getPointReader()).thenReturn(null); Statistics statistics = Mockito.mock(Statistics.class); - Mockito.when(statistics.getCount()).thenReturn(2L); + Mockito.when(statistics.getCount()).thenReturn(3L); Mockito.when(chunkMetadata1.getStatistics()).thenReturn(statistics); Mockito.when(chunkMetadata1.getDataType()).thenReturn(TSDataType.BOOLEAN); @@ -109,13 +115,14 @@ public void testBooleanMemChunkLoader() throws IOException { } private Map buildBooleanTvListMap() { - TVList tvList = TVList.newList(TSDataType.BOOLEAN); - if (tvList != null) { - tvList.putBoolean(1L, true); - tvList.putBoolean(2L, true); - } + TVList tvList1 = TVList.newList(TSDataType.BOOLEAN); + tvList1.putBoolean(1L, true); + tvList1.putBoolean(2L, true); + TVList tvList2 = TVList.newList(TSDataType.BOOLEAN); + tvList2.putBoolean(1L, true); Map tvListMap = new LinkedHashMap<>(); - tvListMap.put(tvList, 2); + tvListMap.put(tvList1, 2); + tvListMap.put(tvList2, 1); return tvListMap; } @@ -134,8 +141,13 @@ public void testInt32MemChunkLoader() throws IOException { assertNull(e.getMessage()); } - ChunkMetadata chunkMetadata1 = Mockito.mock(ChunkMetadata.class); + Statistics pageStatitstics = Mockito.mock(Statistics.class); + List pageStats = Arrays.asList(pageStatitstics); + List pageOffsets = Arrays.asList(new int[] {0, 0}); + Mockito.when(chunk.getPageStatisticsList()).thenReturn(pageStats); + Mockito.when(chunk.getPageOffsetsList()).thenReturn(pageOffsets); + ChunkMetadata chunkMetadata1 = Mockito.mock(ChunkMetadata.class); Mockito.when(chunk.getChunkMetaData()).thenReturn(chunkMetadata1); Mockito.when(chunk.getPointReader()).thenReturn(null); Statistics statistics = Mockito.mock(Statistics.class); @@ -178,13 +190,14 @@ public void testInt32MemChunkLoader() throws IOException { } private Map buildInt32TvListMap() { - TVList tvList = TVList.newList(TSDataType.INT32); - if (tvList != null) { - tvList.putInt(1L, 1); - tvList.putInt(2L, 2); - } + TVList tvList1 = TVList.newList(TSDataType.INT32); + tvList1.putInt(1L, 1); + tvList1.putInt(2L, 2); + TVList tvList2 = TVList.newList(TSDataType.INT32); + tvList2.putInt(1L, 1); Map tvListMap = new LinkedHashMap<>(); - tvListMap.put(tvList, 2); + tvListMap.put(tvList1, 2); + tvListMap.put(tvList2, 1); return tvListMap; } @@ -203,8 +216,13 @@ public void testInt64MemChunkLoader() throws IOException { assertNull(e.getMessage()); } - ChunkMetadata chunkMetadata1 = Mockito.mock(ChunkMetadata.class); + Statistics pageStatitstics = Mockito.mock(Statistics.class); + List pageStats = Arrays.asList(pageStatitstics); + List pageOffsets = Arrays.asList(new int[] {0, 0}); + Mockito.when(chunk.getPageStatisticsList()).thenReturn(pageStats); + Mockito.when(chunk.getPageOffsetsList()).thenReturn(pageOffsets); + ChunkMetadata chunkMetadata1 = Mockito.mock(ChunkMetadata.class); Mockito.when(chunk.getChunkMetaData()).thenReturn(chunkMetadata1); Mockito.when(chunk.getPointReader()).thenReturn(null); Statistics statistics = Mockito.mock(Statistics.class); @@ -247,13 +265,14 @@ public void testInt64MemChunkLoader() throws IOException { } private Map buildInt64TvListMap() { - TVList tvList = TVList.newList(TSDataType.INT64); - if (tvList != null) { - tvList.putLong(1L, 1L); - tvList.putLong(2L, 2L); - } + TVList tvList1 = TVList.newList(TSDataType.INT64); + tvList1.putLong(1L, 1L); + tvList1.putLong(2L, 2L); + TVList tvList2 = TVList.newList(TSDataType.INT64); + tvList2.putLong(1L, 1L); Map tvListMap = new LinkedHashMap<>(); - tvListMap.put(tvList, 2); + tvListMap.put(tvList1, 2); + tvListMap.put(tvList2, 1); return tvListMap; } @@ -272,8 +291,13 @@ public void testFloatMemChunkLoader() throws IOException { assertNull(e.getMessage()); } - ChunkMetadata chunkMetadata1 = Mockito.mock(ChunkMetadata.class); + Statistics pageStatitstics = Mockito.mock(Statistics.class); + List pageStats = Arrays.asList(pageStatitstics); + List pageOffsets = Arrays.asList(new int[] {0, 0}); + Mockito.when(chunk.getPageStatisticsList()).thenReturn(pageStats); + Mockito.when(chunk.getPageOffsetsList()).thenReturn(pageOffsets); + ChunkMetadata chunkMetadata1 = Mockito.mock(ChunkMetadata.class); Mockito.when(chunk.getChunkMetaData()).thenReturn(chunkMetadata1); Mockito.when(chunk.getPointReader()).thenReturn(null); Statistics statistics = Mockito.mock(Statistics.class); @@ -316,13 +340,14 @@ public void testFloatMemChunkLoader() throws IOException { } private Map buildFloatTvListMap() { - TVList tvList = TVList.newList(TSDataType.FLOAT); - if (tvList != null) { - tvList.putFloat(1L, 1.1f); - tvList.putFloat(2L, 2.1f); - } + TVList tvList1 = TVList.newList(TSDataType.FLOAT); + tvList1.putFloat(1L, 1.1f); + tvList1.putFloat(2L, 2.1f); + TVList tvList2 = TVList.newList(TSDataType.FLOAT); + tvList2.putFloat(1L, 1.1f); Map tvListMap = new LinkedHashMap<>(); - tvListMap.put(tvList, 2); + tvListMap.put(tvList1, 2); + tvListMap.put(tvList2, 1); return tvListMap; } @@ -341,8 +366,13 @@ public void testDoubleMemChunkLoader() throws IOException { assertNull(e.getMessage()); } - ChunkMetadata chunkMetadata1 = Mockito.mock(ChunkMetadata.class); + Statistics pageStatitstics = Mockito.mock(Statistics.class); + List pageStats = Arrays.asList(pageStatitstics); + List pageOffsets = Arrays.asList(new int[] {0, 0}); + Mockito.when(chunk.getPageStatisticsList()).thenReturn(pageStats); + Mockito.when(chunk.getPageOffsetsList()).thenReturn(pageOffsets); + ChunkMetadata chunkMetadata1 = Mockito.mock(ChunkMetadata.class); Mockito.when(chunk.getChunkMetaData()).thenReturn(chunkMetadata1); Mockito.when(chunk.getPointReader()).thenReturn(null); Statistics statistics = Mockito.mock(Statistics.class); @@ -385,13 +415,14 @@ public void testDoubleMemChunkLoader() throws IOException { } private Map buildDoubleTvListMap() { - TVList tvList = TVList.newList(TSDataType.DOUBLE); - if (tvList != null) { - tvList.putDouble(1L, 1.1d); - tvList.putDouble(2L, 2.1d); - } + TVList tvList1 = TVList.newList(TSDataType.DOUBLE); + tvList1.putDouble(1L, 1.1d); + tvList1.putDouble(2L, 2.1d); + TVList tvList2 = TVList.newList(TSDataType.DOUBLE); + tvList2.putDouble(1L, 1.1d); Map tvListMap = new LinkedHashMap<>(); - tvListMap.put(tvList, 2); + tvListMap.put(tvList1, 2); + tvListMap.put(tvList2, 1); return tvListMap; } @@ -410,8 +441,13 @@ public void testTextMemChunkLoader() throws IOException { assertNull(e.getMessage()); } - ChunkMetadata chunkMetadata1 = Mockito.mock(ChunkMetadata.class); + Statistics pageStatitstics = Mockito.mock(Statistics.class); + List pageStats = Arrays.asList(pageStatitstics); + List pageOffsets = Arrays.asList(new int[] {0, 0}); + Mockito.when(chunk.getPageStatisticsList()).thenReturn(pageStats); + Mockito.when(chunk.getPageOffsetsList()).thenReturn(pageOffsets); + ChunkMetadata chunkMetadata1 = Mockito.mock(ChunkMetadata.class); Mockito.when(chunk.getChunkMetaData()).thenReturn(chunkMetadata1); Mockito.when(chunk.getPointReader()).thenReturn(null); Statistics statistics = Mockito.mock(Statistics.class); @@ -454,13 +490,14 @@ public void testTextMemChunkLoader() throws IOException { } private Map buildTextTvListMap() { - TVList tvList = TVList.newList(TSDataType.TEXT); - if (tvList != null) { - tvList.putBinary(1L, new Binary(BINARY_STR, TSFileConfig.STRING_CHARSET)); - tvList.putBinary(2L, new Binary(BINARY_STR, TSFileConfig.STRING_CHARSET)); - } + TVList tvList1 = TVList.newList(TSDataType.TEXT); + tvList1.putBinary(1L, new Binary(BINARY_STR, TSFileConfig.STRING_CHARSET)); + tvList1.putBinary(2L, new Binary(BINARY_STR, TSFileConfig.STRING_CHARSET)); + TVList tvList2 = TVList.newList(TSDataType.TEXT); + tvList2.putBinary(1L, new Binary(BINARY_STR, TSFileConfig.STRING_CHARSET)); Map tvListMap = new LinkedHashMap<>(); - tvListMap.put(tvList, 2); + tvListMap.put(tvList1, 2); + tvListMap.put(tvList2, 1); return tvListMap; } } diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/utils/datastructure/BinaryTVListTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/utils/datastructure/BinaryTVListTest.java index 1208ce4878d6d..8424b4af5f1dc 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/utils/datastructure/BinaryTVListTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/utils/datastructure/BinaryTVListTest.java @@ -133,7 +133,7 @@ public void testCalculateChunkSize() { Assert.assertEquals(tvList.memoryBinaryChunkSize, 540); tvList.delete(5, 15); - Assert.assertEquals(tvList.memoryBinaryChunkSize, 252); + Assert.assertEquals(tvList.memoryBinaryChunkSize, 540); tvList.clear(); Assert.assertEquals(tvList.memoryBinaryChunkSize, 0); From 402c8698edef4b54ba73caa035f16761b52bba80 Mon Sep 17 00:00:00 2001 From: shizy Date: Wed, 4 Dec 2024 16:33:03 +0800 Subject: [PATCH 07/54] review comments part 2 --- .../dataregion/memtable/ReadOnlyMemChunk.java | 7 ++- .../read/reader/chunk/MemChunkReader.java | 2 +- .../MergeSortTvListIterator.java | 57 ++++++++++++------- 3 files changed, 40 insertions(+), 26 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/ReadOnlyMemChunk.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/ReadOnlyMemChunk.java index 2b3d88f1eac91..e24e29bb59a52 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/ReadOnlyMemChunk.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/ReadOnlyMemChunk.java @@ -44,6 +44,7 @@ import java.io.IOException; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collections; import java.util.List; import java.util.Map; @@ -158,14 +159,15 @@ private void updateChunkAndPageStatisticsFromTvLists() { List tvLists = new ArrayList<>(tvListQueryMap.keySet()); MergeSortTvListIterator timeValuePairIterator = new MergeSortTvListIterator(dataType, encoding, floatPrecision, tvLists); - int[] tvListOffsets = timeValuePairIterator.getTVListOffsets(); while (timeValuePairIterator.hasNextTimeValuePair()) { + int[] tvListOffsetsBeforeNext = timeValuePairIterator.getLastTVListOffsets(); TimeValuePair tvPair = timeValuePairIterator.nextTimeValuePair(); if (!isPointDeleted(tvPair.getTimestamp(), deletionList, deleteCursor)) { if (cnt % MAX_NUMBER_OF_POINTS_IN_PAGE == 0) { Statistics stats = Statistics.getStatsByType(dataType); pageStatisticsList.add(stats); - pageOffsetsList.add(tvListOffsets); + pageOffsetsList.add( + Arrays.copyOf(tvListOffsetsBeforeNext, tvListOffsetsBeforeNext.length)); } Statistics pageStatistics = pageStatisticsList.get(pageStatisticsList.size() - 1); @@ -202,7 +204,6 @@ private void updateChunkAndPageStatisticsFromTvLists() { // do nothing } pageStatistics.setEmpty(false); - tvListOffsets = timeValuePairIterator.getTVListOffsets(); cnt++; } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemChunkReader.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemChunkReader.java index b007617f0da99..5228cf3615088 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemChunkReader.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemChunkReader.java @@ -188,7 +188,7 @@ private boolean isOutOfMemPageBounds() { int[] currTvListOffsets = ((MergeSortTvListIterator) timeValuePairIterator).getTVListOffsets(); for (int i = 0; i < pageEndOffsets.length; i++) { - if (currTvListOffsets[i] > pageEndOffsets[i]) { + if (currTvListOffsets[i] >= pageEndOffsets[i]) { return true; } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MergeSortTvListIterator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MergeSortTvListIterator.java index 156e044fa0d29..d48cf489fedae 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MergeSortTvListIterator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MergeSortTvListIterator.java @@ -36,14 +36,20 @@ public class MergeSortTvListIterator implements IPointReader { private TSEncoding encoding; private int floatPrecision = -1; + private int selectedTVListIndex = -1; private TimeValuePair currentTvPair; + private final int[] tvListOffsets; + private final int[] lastTvListOffsets; + public MergeSortTvListIterator(TSDataType tsDataType, List tvLists) { this.tsDataType = tsDataType; tvListIterators = new ArrayList<>(); for (TVList tvList : tvLists) { tvListIterators.add(tvList.iterator()); } + this.tvListOffsets = new int[tvLists.size()]; + this.lastTvListOffsets = new int[tvLists.size()]; } public MergeSortTvListIterator( @@ -53,9 +59,9 @@ public MergeSortTvListIterator( this.floatPrecision = floatPrecision; } - private int getSelectedTVListIndex() { + private void prepareNextRow() { long time = Long.MAX_VALUE; - int selectedTVListIndex = -1; + selectedTVListIndex = -1; for (int i = 0; i < tvListIterators.size(); i++) { TVList.TVListIterator iterator = tvListIterators.get(i); TimeValuePair currTvPair = null; @@ -69,30 +75,36 @@ private int getSelectedTVListIndex() { selectedTVListIndex = i; } } - return selectedTVListIndex; } @Override public boolean hasNextTimeValuePair() { - boolean hasNext = false; - int selectedTVListIndex = getSelectedTVListIndex(); - if (selectedTVListIndex >= 0) { - currentTvPair = tvListIterators.get(selectedTVListIndex).next(); - hasNext = true; - - // call next to skip identical timestamp in other iterators - for (int i = 0; i < tvListIterators.size(); i++) { - TimeValuePair tvPair = tvListIterators.get(i).current(); - if (tvPair != null && tvPair.getTimestamp() == currentTvPair.getTimestamp()) { - tvListIterators.get(i).next(); - } - } + if (selectedTVListIndex == -1) { + prepareNextRow(); } - return hasNext; + return selectedTVListIndex >= 0 && selectedTVListIndex < tvListIterators.size(); } @Override public TimeValuePair nextTimeValuePair() { + if (!hasNextTimeValuePair()) { + return null; + } + lastTvListOffsets[selectedTVListIndex] = tvListIterators.get(selectedTVListIndex).getIndex(); + currentTvPair = tvListIterators.get(selectedTVListIndex).next(); + tvListOffsets[selectedTVListIndex] = tvListIterators.get(selectedTVListIndex).getIndex(); + + // call next to skip identical timestamp in other iterators + for (int i = 0; i < tvListIterators.size(); i++) { + TimeValuePair tvPair = tvListIterators.get(i).current(); + if (tvPair != null && tvPair.getTimestamp() == currentTvPair.getTimestamp()) { + lastTvListOffsets[i] = tvListIterators.get(i).getIndex(); + tvListIterators.get(i).next(); + tvListOffsets[i] = tvListIterators.get(i).getIndex(); + } + } + + selectedTVListIndex = -1; return currentTimeValuePair(); } @@ -122,6 +134,7 @@ public TimeValuePair currentTimeValuePair() { @Override public long getUsedMemorySize() { + // not used return 0; } @@ -130,18 +143,18 @@ public void close() throws IOException { tvListIterators.clear(); } + public int[] getLastTVListOffsets() { + return lastTvListOffsets; + } + public int[] getTVListOffsets() { - int size = tvListIterators.size(); - int[] tvListOffsets = new int[size]; - for (int i = 0; i < size; i++) { - tvListOffsets[i] = tvListIterators.get(i).getIndex(); - } return tvListOffsets; } public void setTVListOffsets(int[] tvListOffsets) { for (int i = 0; i < tvListIterators.size(); i++) { tvListIterators.get(i).setIndex(tvListOffsets[i]); + this.tvListOffsets[i] = tvListOffsets[i]; } } } From 5aaa59b6ab655df3193995ab75ec444a75841b17 Mon Sep 17 00:00:00 2001 From: shizy Date: Thu, 5 Dec 2024 09:43:06 +0800 Subject: [PATCH 08/54] push down global time filter --- .../utils/ResourceByPathUtils.java | 35 +++++++++++++------ .../storageengine/dataregion/DataRegion.java | 4 ++- .../dataregion/memtable/AbstractMemTable.java | 7 ++-- .../dataregion/memtable/IMemTable.java | 4 ++- .../dataregion/memtable/TsFileProcessor.java | 9 +++-- .../dataregion/DataRegionTest.java | 3 +- .../memtable/PrimitiveMemTableTest.java | 13 +++---- .../memtable/TsFileProcessorTest.java | 20 +++++------ .../recover/file/TsFilePlanRedoerTest.java | 18 +++++----- 9 files changed, 69 insertions(+), 44 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/utils/ResourceByPathUtils.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/utils/ResourceByPathUtils.java index 45b98956ffa36..3efcc42aaa9ae 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/utils/ResourceByPathUtils.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/utils/ResourceByPathUtils.java @@ -49,6 +49,7 @@ import org.apache.tsfile.file.metadata.enums.TSEncoding; import org.apache.tsfile.file.metadata.statistics.Statistics; import org.apache.tsfile.read.common.TimeRange; +import org.apache.tsfile.read.filter.basic.Filter; import org.apache.tsfile.utils.Pair; import org.apache.tsfile.write.schema.IMeasurementSchema; import org.apache.tsfile.write.schema.VectorMeasurementSchema; @@ -89,7 +90,8 @@ public abstract ReadOnlyMemChunk getReadOnlyMemChunkFromMemTable( QueryContext context, IMemTable memTable, List> modsToMemtable, - long timeLowerBound) + long timeLowerBound, + Filter globalTimeFilter) throws QueryProcessException, IOException; public abstract List getVisibleMetadataListFromWriter( @@ -186,7 +188,8 @@ public ReadOnlyMemChunk getReadOnlyMemChunkFromMemTable( QueryContext context, IMemTable memTable, List> modsToMemtable, - long timeLowerBound) + long timeLowerBound, + Filter globalTimeFilter) throws QueryProcessException { Map memTableMap = memTable.getMemTableMap(); IDeviceID deviceID = alignedFullPath.getDeviceId(); @@ -367,16 +370,22 @@ public ITimeSeriesMetadata generateTimeSeriesMetadata( return timeSeriesMetadata; } - // TODO: global time filter pushdown private Map prepareTvListMapForQuery( - WritableMemChunk memChunk, boolean isWorkMemTable, QueryContext context) { + WritableMemChunk memChunk, + boolean isWorkMemTable, + QueryContext context, + Filter globalTimeFilter) { Map tvListQueryMap = new LinkedHashMap<>(); // immutable sorted lists for (TVList tvList : memChunk.getSortedList()) { + if (globalTimeFilter != null + && !globalTimeFilter.satisfyStartEndTime(tvList.getMinTime(), tvList.getMaxTime())) { + continue; + } tvList.lockQueryList(); try { LOGGER.debug( - "Flushing/Working MemTable - Add current query context to immutable TVList's query list"); + "Flushing/Working MemTable - add current query context to immutable TVList's query list"); tvList.getQueryContextList().add(context); tvListQueryMap.put(tvList, tvList.rowCount()); } finally { @@ -389,10 +398,13 @@ private Map prepareTvListMapForQuery( list.lockQueryList(); try { if (!isWorkMemTable) { - LOGGER.debug( - "Flushing MemTable - add current query context to mutable TVList's query list"); - list.getQueryContextList().add(context); - tvListQueryMap.put(list, list.rowCount()); + if (globalTimeFilter == null + || globalTimeFilter.satisfyStartEndTime(list.getMinTime(), list.getMaxTime())) { + LOGGER.debug( + "Flushing MemTable - add current query context to mutable TVList's query list"); + list.getQueryContextList().add(context); + tvListQueryMap.put(list, list.rowCount()); + } } else { if (list.isSorted() || list.getQueryContextList().isEmpty()) { LOGGER.debug( @@ -429,7 +441,8 @@ public ReadOnlyMemChunk getReadOnlyMemChunkFromMemTable( QueryContext context, IMemTable memTable, List> modsToMemtable, - long timeLowerBound) + long timeLowerBound, + Filter globalTimeFilter) throws QueryProcessException, IOException { Map memTableMap = memTable.getMemTableMap(); IDeviceID deviceID = fullPath.getDeviceId(); @@ -444,7 +457,7 @@ public ReadOnlyMemChunk getReadOnlyMemChunkFromMemTable( // prepare TVList for query. It should clone and sort TVList if necessary. // Also, the map keeps TVlist length at this moment. Map tvListQueryMap = - prepareTvListMapForQuery(memChunk, modsToMemtable == null, context); + prepareTvListMapForQuery(memChunk, modsToMemtable == null, context, globalTimeFilter); List deletionList = null; if (modsToMemtable != null) { deletionList = diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/DataRegion.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/DataRegion.java index d583eafd9d9b2..483bc9cb61b24 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/DataRegion.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/DataRegion.java @@ -2163,7 +2163,9 @@ private List getFileResourceListForQuery( if (tsFileResource.isClosed()) { tsfileResourcesForQuery.add(tsFileResource); } else { - tsFileResource.getProcessor().query(pathList, context, tsfileResourcesForQuery); + tsFileResource + .getProcessor() + .query(pathList, context, tsfileResourcesForQuery, globalTimeFilter); } } catch (IOException e) { throw new MetadataException(e); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AbstractMemTable.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AbstractMemTable.java index 5e18b7582d624..f256db485dc53 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AbstractMemTable.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AbstractMemTable.java @@ -58,6 +58,7 @@ import org.apache.tsfile.file.metadata.enums.TSEncoding; import org.apache.tsfile.file.metadata.statistics.TimeStatistics; import org.apache.tsfile.read.common.TimeRange; +import org.apache.tsfile.read.filter.basic.Filter; import org.apache.tsfile.utils.BitMap; import org.apache.tsfile.utils.Pair; import org.apache.tsfile.utils.ReadWriteIOUtils; @@ -459,10 +460,12 @@ public ReadOnlyMemChunk query( QueryContext context, IFullPath fullPath, long ttlLowerBound, - List> modsToMemtable) + List> modsToMemtable, + Filter globalTimeFilter) throws IOException, QueryProcessException { return ResourceByPathUtils.getResourceInstance(fullPath) - .getReadOnlyMemChunkFromMemTable(context, this, modsToMemtable, ttlLowerBound); + .getReadOnlyMemChunkFromMemTable( + context, this, modsToMemtable, ttlLowerBound, globalTimeFilter); } @Override diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/IMemTable.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/IMemTable.java index 35d9ad0e95c3c..fbd24aea16f1d 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/IMemTable.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/IMemTable.java @@ -33,6 +33,7 @@ import org.apache.tsfile.file.metadata.IChunkMetadata; import org.apache.tsfile.file.metadata.IDeviceID; +import org.apache.tsfile.read.filter.basic.Filter; import org.apache.tsfile.utils.Pair; import org.apache.tsfile.write.schema.IMeasurementSchema; @@ -117,7 +118,8 @@ ReadOnlyMemChunk query( QueryContext context, IFullPath fullPath, long ttlLowerBound, - List> modsToMemtabled) + List> modsToMemtabled, + Filter globalTimeFilter) throws IOException, QueryProcessException, MetadataException; void queryForSeriesRegionScan( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/TsFileProcessor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/TsFileProcessor.java index 8f02c0cabee17..7992c1debf3ad 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/TsFileProcessor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/TsFileProcessor.java @@ -85,6 +85,7 @@ import org.apache.tsfile.file.metadata.IChunkMetadata; import org.apache.tsfile.file.metadata.IDeviceID; import org.apache.tsfile.file.metadata.TableSchema; +import org.apache.tsfile.read.filter.basic.Filter; import org.apache.tsfile.utils.Binary; import org.apache.tsfile.utils.Pair; import org.apache.tsfile.write.writer.RestorableTsFileIOWriter; @@ -2112,7 +2113,8 @@ public void queryForDeviceRegionScan( public void query( List seriesPaths, QueryContext context, - List tsfileResourcesForQuery) + List tsfileResourcesForQuery, + Filter globalTimeFilter) throws IOException { long startTime = System.nanoTime(); try { @@ -2129,14 +2131,15 @@ public void query( continue; } ReadOnlyMemChunk memChunk = - flushingMemTable.query(context, seriesPath, timeLowerBound, modsToMemtable); + flushingMemTable.query( + context, seriesPath, timeLowerBound, modsToMemtable, globalTimeFilter); if (memChunk != null) { readOnlyMemChunks.add(memChunk); } } if (workMemTable != null) { ReadOnlyMemChunk memChunk = - workMemTable.query(context, seriesPath, timeLowerBound, null); + workMemTable.query(context, seriesPath, timeLowerBound, null, globalTimeFilter); if (memChunk != null) { readOnlyMemChunks.add(memChunk); } diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/DataRegionTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/DataRegionTest.java index ddce03e683387..fd551398563e7 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/DataRegionTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/DataRegionTest.java @@ -226,7 +226,8 @@ record = new TSRecord(deviceId, j); tsfileProcessor.query( Collections.singletonList(IFullPath.convertToIFullPath(fullPath)), EnvironmentUtils.TEST_QUERY_CONTEXT, - tsfileResourcesForQuery); + tsfileResourcesForQuery, + null); } Assert.assertEquals(1, tsfileResourcesForQuery.size()); diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/memtable/PrimitiveMemTableTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/memtable/PrimitiveMemTableTest.java index 582666a6549e8..82cc4b1d72c0a 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/memtable/PrimitiveMemTableTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/memtable/PrimitiveMemTableTest.java @@ -168,7 +168,7 @@ public void simpleTest() throws IOException, QueryProcessException, MetadataExce } ReadOnlyMemChunk memChunk = - memTable.query(new QueryContext(), nonAlignedFullPath, Long.MIN_VALUE, null); + memTable.query(new QueryContext(), nonAlignedFullPath, Long.MIN_VALUE, null, null); IPointReader iterator = memChunk.getPointReader(); for (int i = 0; i < dataSize; i++) { iterator.hasNextTimeValuePair(); @@ -259,7 +259,8 @@ public void queryWithDeletionTest() throws IOException, QueryProcessException, M new TreeDeletionEntry(new MeasurementPath(deviceID, measurementId[0]), 10, dataSize); modsToMemtable.add(new Pair<>(deletion, memTable)); ReadOnlyMemChunk memChunk = - memTable.query(new QueryContext(), nonAlignedFullPath, Long.MIN_VALUE, modsToMemtable); + memTable.query( + new QueryContext(), nonAlignedFullPath, Long.MIN_VALUE, modsToMemtable, null); IPointReader iterator = memChunk.getPointReader(); int cnt = 0; while (iterator.hasNextTimeValuePair()) { @@ -304,7 +305,7 @@ public void queryAlignChuckWithDeletionTest() new TreeDeletionEntry(new MeasurementPath(deviceID, measurementId[0]), 10, dataSize); modsToMemtable.add(new Pair<>(deletion, memTable)); ReadOnlyMemChunk memChunk = - memTable.query(new QueryContext(), alignedFullPath, Long.MIN_VALUE, modsToMemtable); + memTable.query(new QueryContext(), alignedFullPath, Long.MIN_VALUE, modsToMemtable, null); IPointReader iterator = memChunk.getPointReader(); int cnt = 0; while (iterator.hasNextTimeValuePair()) { @@ -343,7 +344,7 @@ private void write( CompressionType.UNCOMPRESSED, Collections.emptyMap())); IPointReader tvPair = - memTable.query(new QueryContext(), fullPath, Long.MIN_VALUE, null).getPointReader(); + memTable.query(new QueryContext(), fullPath, Long.MIN_VALUE, null, null).getPointReader(); Arrays.sort(ret); TimeValuePair last = null; for (int i = 0; i < ret.length; i++) { @@ -392,7 +393,7 @@ private void writeVector(IMemTable memTable) Collections.emptyMap()))); IPointReader tvPair = memTable - .query(new QueryContext(), tmpAlignedFullPath, Long.MIN_VALUE, null) + .query(new QueryContext(), tmpAlignedFullPath, Long.MIN_VALUE, null, null) .getPointReader(); for (int i = 0; i < 100; i++) { tvPair.hasNextTimeValuePair(); @@ -421,7 +422,7 @@ private void writeVector(IMemTable memTable) tvPair = memTable - .query(new QueryContext(), tmpAlignedFullPath, Long.MIN_VALUE, null) + .query(new QueryContext(), tmpAlignedFullPath, Long.MIN_VALUE, null, null) .getPointReader(); for (int i = 0; i < 100; i++) { tvPair.hasNextTimeValuePair(); diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/memtable/TsFileProcessorTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/memtable/TsFileProcessorTest.java index 0393067f109f4..d13321eae134a 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/memtable/TsFileProcessorTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/memtable/TsFileProcessorTest.java @@ -126,7 +126,7 @@ public void testWriteAndFlush() throws IOException, WriteProcessException, Metad IDeviceID.Factory.DEFAULT_FACTORY.create(deviceId), new MeasurementSchema( measurementId, dataType, encoding, CompressionType.UNCOMPRESSED, props)); - processor.query(Collections.singletonList(fullPath), context, tsfileResourcesForQuery); + processor.query(Collections.singletonList(fullPath), context, tsfileResourcesForQuery, null); assertTrue(tsfileResourcesForQuery.isEmpty()); for (int i = 1; i <= 100; i++) { @@ -137,7 +137,7 @@ public void testWriteAndFlush() throws IOException, WriteProcessException, Metad // query data in memory tsfileResourcesForQuery.clear(); - processor.query(Collections.singletonList(fullPath), context, tsfileResourcesForQuery); + processor.query(Collections.singletonList(fullPath), context, tsfileResourcesForQuery, null); TsFileResource tsFileResource = tsfileResourcesForQuery.get(0); assertFalse(tsFileResource.getReadOnlyMemChunk(fullPath).isEmpty()); @@ -156,7 +156,7 @@ public void testWriteAndFlush() throws IOException, WriteProcessException, Metad processor.syncFlush(); tsfileResourcesForQuery.clear(); - processor.query(Collections.singletonList(fullPath), context, tsfileResourcesForQuery); + processor.query(Collections.singletonList(fullPath), context, tsfileResourcesForQuery, null); assertTrue(tsfileResourcesForQuery.get(0).getReadOnlyMemChunk(fullPath).isEmpty()); processor.syncClose(); } @@ -184,7 +184,7 @@ public void testWriteAndRestoreMetadata() IDeviceID.Factory.DEFAULT_FACTORY.create(deviceId), new MeasurementSchema( measurementId, dataType, encoding, CompressionType.UNCOMPRESSED, props)); - processor.query(Collections.singletonList(fullPath), context, tsfileResourcesForQuery); + processor.query(Collections.singletonList(fullPath), context, tsfileResourcesForQuery, null); assertTrue(tsfileResourcesForQuery.isEmpty()); for (int i = 1; i <= 100; i++) { @@ -195,7 +195,7 @@ public void testWriteAndRestoreMetadata() // query data in memory tsfileResourcesForQuery.clear(); - processor.query(Collections.singletonList(fullPath), context, tsfileResourcesForQuery); + processor.query(Collections.singletonList(fullPath), context, tsfileResourcesForQuery, null); assertFalse(tsfileResourcesForQuery.get(0).getReadOnlyMemChunk(fullPath).isEmpty()); int num = 1; List memChunks = tsfileResourcesForQuery.get(0).getReadOnlyMemChunk(fullPath); @@ -213,7 +213,7 @@ public void testWriteAndRestoreMetadata() processor.syncFlush(); tsfileResourcesForQuery.clear(); - processor.query(Collections.singletonList(fullPath), context, tsfileResourcesForQuery); + processor.query(Collections.singletonList(fullPath), context, tsfileResourcesForQuery, null); assertTrue(tsfileResourcesForQuery.get(0).getReadOnlyMemChunk(fullPath).isEmpty()); RestorableTsFileIOWriter tsFileIOWriter = processor.getWriter(); @@ -267,7 +267,7 @@ public void testMultiFlush() throws IOException, WriteProcessException, Metadata IDeviceID.Factory.DEFAULT_FACTORY.create(deviceId), new MeasurementSchema( measurementId, dataType, encoding, CompressionType.UNCOMPRESSED, props)); - processor.query(Collections.singletonList(fullPath), context, tsfileResourcesForQuery); + processor.query(Collections.singletonList(fullPath), context, tsfileResourcesForQuery, null); assertTrue(tsfileResourcesForQuery.isEmpty()); for (int flushId = 0; flushId < 10; flushId++) { @@ -281,7 +281,7 @@ public void testMultiFlush() throws IOException, WriteProcessException, Metadata processor.syncFlush(); tsfileResourcesForQuery.clear(); - processor.query(Collections.singletonList(fullPath), context, tsfileResourcesForQuery); + processor.query(Collections.singletonList(fullPath), context, tsfileResourcesForQuery, null); assertFalse(tsfileResourcesForQuery.isEmpty()); assertTrue(tsfileResourcesForQuery.get(0).getReadOnlyMemChunk(fullPath).isEmpty()); processor.syncClose(); @@ -825,7 +825,7 @@ public void testWriteAndClose() throws IOException, WriteProcessException, Metad IDeviceID.Factory.DEFAULT_FACTORY.create(deviceId), new MeasurementSchema( measurementId, dataType, encoding, CompressionType.UNCOMPRESSED, props)); - processor.query(Collections.singletonList(fullPath), context, tsfileResourcesForQuery); + processor.query(Collections.singletonList(fullPath), context, tsfileResourcesForQuery, null); assertTrue(tsfileResourcesForQuery.isEmpty()); for (int i = 1; i <= 100; i++) { @@ -836,7 +836,7 @@ public void testWriteAndClose() throws IOException, WriteProcessException, Metad // query data in memory tsfileResourcesForQuery.clear(); - processor.query(Collections.singletonList(fullPath), context, tsfileResourcesForQuery); + processor.query(Collections.singletonList(fullPath), context, tsfileResourcesForQuery, null); assertFalse(tsfileResourcesForQuery.isEmpty()); assertFalse(tsfileResourcesForQuery.get(0).getReadOnlyMemChunk(fullPath).isEmpty()); List memChunks = tsfileResourcesForQuery.get(0).getReadOnlyMemChunk(fullPath); diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/wal/recover/file/TsFilePlanRedoerTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/wal/recover/file/TsFilePlanRedoerTest.java index f311328f6321d..36830025bc267 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/wal/recover/file/TsFilePlanRedoerTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/wal/recover/file/TsFilePlanRedoerTest.java @@ -159,7 +159,7 @@ public void testRedoInsertRowPlan() throws Exception { new NonAlignedFullPath( DEVICE2_NAME, new MeasurementSchema("s1", TSDataType.FLOAT, TSEncoding.RLE)); ReadOnlyMemChunk memChunk = - recoveryMemTable.query(new QueryContext(), fullPath, Long.MIN_VALUE, null); + recoveryMemTable.query(new QueryContext(), fullPath, Long.MIN_VALUE, null, null); IPointReader iterator = memChunk.getPointReader(); time = 5; while (iterator.hasNextTimeValuePair()) { @@ -173,7 +173,7 @@ public void testRedoInsertRowPlan() throws Exception { fullPath = new NonAlignedFullPath( DEVICE2_NAME, new MeasurementSchema("s2", TSDataType.DOUBLE, TSEncoding.RLE)); - memChunk = recoveryMemTable.query(new QueryContext(), fullPath, Long.MIN_VALUE, null); + memChunk = recoveryMemTable.query(new QueryContext(), fullPath, Long.MIN_VALUE, null, null); iterator = memChunk.getPointReader(); time = 5; while (iterator.hasNextTimeValuePair()) { @@ -259,7 +259,7 @@ public void testRedoInsertAlignedRowPlan() throws Exception { new MeasurementSchema("s4", TSDataType.FLOAT, TSEncoding.RLE), new MeasurementSchema("s5", TSDataType.TEXT, TSEncoding.PLAIN))); ReadOnlyMemChunk memChunk = - recoveryMemTable.query(new QueryContext(), fullPath, Long.MIN_VALUE, null); + recoveryMemTable.query(new QueryContext(), fullPath, Long.MIN_VALUE, null, null); IPointReader iterator = memChunk.getPointReader(); int time = 6; while (iterator.hasNextTimeValuePair()) { @@ -339,7 +339,7 @@ public void testRedoInsertTabletPlan() throws Exception { new NonAlignedFullPath( DEVICE1_NAME, new MeasurementSchema("s1", TSDataType.INT32, TSEncoding.RLE)); ReadOnlyMemChunk memChunk = - recoveryMemTable.query(new QueryContext(), fullPath, Long.MIN_VALUE, null); + recoveryMemTable.query(new QueryContext(), fullPath, Long.MIN_VALUE, null, null); IPointReader iterator = memChunk.getPointReader(); int time = 5; while (iterator.hasNextTimeValuePair()) { @@ -353,7 +353,7 @@ public void testRedoInsertTabletPlan() throws Exception { fullPath = new NonAlignedFullPath( DEVICE1_NAME, new MeasurementSchema("s2", TSDataType.INT64, TSEncoding.RLE)); - memChunk = recoveryMemTable.query(new QueryContext(), fullPath, Long.MIN_VALUE, null); + memChunk = recoveryMemTable.query(new QueryContext(), fullPath, Long.MIN_VALUE, null, null); iterator = memChunk.getPointReader(); time = 5; while (iterator.hasNextTimeValuePair()) { @@ -452,7 +452,7 @@ public void testRedoInsertAlignedTabletPlan() throws Exception { new MeasurementSchema("s4", TSDataType.FLOAT, TSEncoding.RLE), new MeasurementSchema("s5", TSDataType.TEXT, TSEncoding.PLAIN))); ReadOnlyMemChunk memChunk = - recoveryMemTable.query(new QueryContext(), fullPath, Long.MIN_VALUE, null); + recoveryMemTable.query(new QueryContext(), fullPath, Long.MIN_VALUE, null, null); IPointReader iterator = memChunk.getPointReader(); int time = 6; while (iterator.hasNextTimeValuePair()) { @@ -574,13 +574,13 @@ public void testRedoOverLapPlanIntoUnseqFile() throws Exception { new NonAlignedFullPath( DEVICE1_NAME, new MeasurementSchema("s1", TSDataType.INT32, TSEncoding.RLE)); ReadOnlyMemChunk memChunk = - recoveryMemTable.query(new QueryContext(), fullPath, Long.MIN_VALUE, null); + recoveryMemTable.query(new QueryContext(), fullPath, Long.MIN_VALUE, null, null); assertTrue(memChunk == null || memChunk.isEmpty()); // check d1.s2 fullPath = new NonAlignedFullPath( DEVICE1_NAME, new MeasurementSchema("s2", TSDataType.INT64, TSEncoding.RLE)); - memChunk = recoveryMemTable.query(new QueryContext(), fullPath, Long.MIN_VALUE, null); + memChunk = recoveryMemTable.query(new QueryContext(), fullPath, Long.MIN_VALUE, null, null); assertTrue(memChunk == null || memChunk.isEmpty()); } @@ -719,7 +719,7 @@ public void testRedoAlignedInsertAfterDeleteTimeseries() throws Exception { new MeasurementSchema("s4", TSDataType.FLOAT, TSEncoding.RLE), new MeasurementSchema("s5", TSDataType.TEXT, TSEncoding.PLAIN))); ReadOnlyMemChunk memChunk = - recoveryMemTable.query(new QueryContext(), fullPath, Long.MIN_VALUE, null); + recoveryMemTable.query(new QueryContext(), fullPath, Long.MIN_VALUE, null, null); IPointReader iterator = memChunk.getPointReader(); time = 6; while (iterator.hasNextTimeValuePair()) { From cfc9e74203d385b36b33df8f029a6460a7b0b127 Mon Sep 17 00:00:00 2001 From: shizy Date: Thu, 5 Dec 2024 10:02:45 +0800 Subject: [PATCH 09/54] fix MemPageReaderTest case --- .../dataregion/read/reader/chunk/MemPageReader.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemPageReader.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemPageReader.java index 8cb82a735b073..21668df320e9a 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemPageReader.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemPageReader.java @@ -252,7 +252,9 @@ private void initializeOffsets() { if (pageStartOffsets != null) { mergeSortTvListIterator.setTVListOffsets(pageStartOffsets); } - ((MemChunkReader.TsBlockSupplier) tsBlockSupplier).setPageEndOffsets(pageEndOffsets); + if (tsBlockSupplier instanceof MemChunkReader.TsBlockSupplier) { + ((MemChunkReader.TsBlockSupplier) tsBlockSupplier).setPageEndOffsets(pageEndOffsets); + } } // memory page statistics should be initialized when constructing ReadOnlyMemChunk object. From cd3f7d958250490fb267e2664ecb3411f4b048bf Mon Sep 17 00:00:00 2001 From: shizy Date: Thu, 5 Dec 2024 13:15:23 +0800 Subject: [PATCH 10/54] fix memory page offsets error --- .../dataregion/memtable/ReadOnlyMemChunk.java | 16 +++++++--------- .../datastructure/MergeSortTvListIterator.java | 8 -------- 2 files changed, 7 insertions(+), 17 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/ReadOnlyMemChunk.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/ReadOnlyMemChunk.java index e24e29bb59a52..bdb7d04f82cac 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/ReadOnlyMemChunk.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/ReadOnlyMemChunk.java @@ -159,17 +159,15 @@ private void updateChunkAndPageStatisticsFromTvLists() { List tvLists = new ArrayList<>(tvListQueryMap.keySet()); MergeSortTvListIterator timeValuePairIterator = new MergeSortTvListIterator(dataType, encoding, floatPrecision, tvLists); + int[] tvListOffsets = timeValuePairIterator.getTVListOffsets(); while (timeValuePairIterator.hasNextTimeValuePair()) { - int[] tvListOffsetsBeforeNext = timeValuePairIterator.getLastTVListOffsets(); + if (cnt % MAX_NUMBER_OF_POINTS_IN_PAGE == 0) { + Statistics stats = Statistics.getStatsByType(dataType); + pageStatisticsList.add(stats); + pageOffsetsList.add(Arrays.copyOf(tvListOffsets, tvListOffsets.length)); + } TimeValuePair tvPair = timeValuePairIterator.nextTimeValuePair(); if (!isPointDeleted(tvPair.getTimestamp(), deletionList, deleteCursor)) { - if (cnt % MAX_NUMBER_OF_POINTS_IN_PAGE == 0) { - Statistics stats = Statistics.getStatsByType(dataType); - pageStatisticsList.add(stats); - pageOffsetsList.add( - Arrays.copyOf(tvListOffsetsBeforeNext, tvListOffsetsBeforeNext.length)); - } - Statistics pageStatistics = pageStatisticsList.get(pageStatisticsList.size() - 1); switch (dataType) { case BOOLEAN: @@ -204,8 +202,8 @@ private void updateChunkAndPageStatisticsFromTvLists() { // do nothing } pageStatistics.setEmpty(false); - cnt++; } + cnt++; } chunkStatistics.setEmpty(cnt == 0); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MergeSortTvListIterator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MergeSortTvListIterator.java index d48cf489fedae..11f1441c98adc 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MergeSortTvListIterator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MergeSortTvListIterator.java @@ -40,7 +40,6 @@ public class MergeSortTvListIterator implements IPointReader { private TimeValuePair currentTvPair; private final int[] tvListOffsets; - private final int[] lastTvListOffsets; public MergeSortTvListIterator(TSDataType tsDataType, List tvLists) { this.tsDataType = tsDataType; @@ -49,7 +48,6 @@ public MergeSortTvListIterator(TSDataType tsDataType, List tvLists) { tvListIterators.add(tvList.iterator()); } this.tvListOffsets = new int[tvLists.size()]; - this.lastTvListOffsets = new int[tvLists.size()]; } public MergeSortTvListIterator( @@ -90,7 +88,6 @@ public TimeValuePair nextTimeValuePair() { if (!hasNextTimeValuePair()) { return null; } - lastTvListOffsets[selectedTVListIndex] = tvListIterators.get(selectedTVListIndex).getIndex(); currentTvPair = tvListIterators.get(selectedTVListIndex).next(); tvListOffsets[selectedTVListIndex] = tvListIterators.get(selectedTVListIndex).getIndex(); @@ -98,7 +95,6 @@ public TimeValuePair nextTimeValuePair() { for (int i = 0; i < tvListIterators.size(); i++) { TimeValuePair tvPair = tvListIterators.get(i).current(); if (tvPair != null && tvPair.getTimestamp() == currentTvPair.getTimestamp()) { - lastTvListOffsets[i] = tvListIterators.get(i).getIndex(); tvListIterators.get(i).next(); tvListOffsets[i] = tvListIterators.get(i).getIndex(); } @@ -143,10 +139,6 @@ public void close() throws IOException { tvListIterators.clear(); } - public int[] getLastTVListOffsets() { - return lastTvListOffsets; - } - public int[] getTVListOffsets() { return tvListOffsets; } From e3d2d5ec947ea878bbb567515138ffc42186f21c Mon Sep 17 00:00:00 2001 From: shizy Date: Fri, 6 Dec 2024 09:19:55 +0800 Subject: [PATCH 11/54] synchronized sort & MergeSortTvListIterator bug --- .../dataregion/memtable/ReadOnlyMemChunk.java | 4 ++-- .../dataregion/memtable/WritableMemChunk.java | 6 +++--- .../db/utils/datastructure/MergeSortTvListIterator.java | 1 + .../org/apache/iotdb/db/utils/datastructure/TVList.java | 7 ++++++- 4 files changed, 12 insertions(+), 6 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/ReadOnlyMemChunk.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/ReadOnlyMemChunk.java index bdb7d04f82cac..dae2dfc4ebde6 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/ReadOnlyMemChunk.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/ReadOnlyMemChunk.java @@ -146,7 +146,7 @@ private void sortTvLists() { TVList tvList = entry.getKey(); int queryRowCount = entry.getValue(); if (!tvList.isSorted() && queryRowCount > tvList.seqRowCount()) { - tvList.sort(); + tvList.safelySort(); } } } @@ -229,7 +229,7 @@ public IPointReader getPointReader() { TVList tvList = entry.getKey(); int queryLength = entry.getValue(); if (!tvList.isSorted() && queryLength > tvList.seqRowCount()) { - tvList.sort(); + tvList.safelySort(); } } TsBlock tsBlock = buildTsBlock(); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/WritableMemChunk.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/WritableMemChunk.java index a27c5afd28bef..eb044ba6a2099 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/WritableMemChunk.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/WritableMemChunk.java @@ -74,7 +74,7 @@ protected void handoverTvList() { } if (list.getQueryContextList().isEmpty()) { - list.sort(); + list.safelySort(); sortedList.add(list); } else { QueryContext firstQuery = list.getQueryContextList().get(0); @@ -304,14 +304,14 @@ private void sortTVList() { } if (!list.isSorted()) { - list.sort(); + list.safelySort(); } } @Override public synchronized void sortTvListForFlush() { if (!list.isSorted()) { - list.sort(); + list.safelySort(); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MergeSortTvListIterator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MergeSortTvListIterator.java index 11f1441c98adc..4768c6439700e 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MergeSortTvListIterator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MergeSortTvListIterator.java @@ -147,6 +147,7 @@ public void setTVListOffsets(int[] tvListOffsets) { for (int i = 0; i < tvListIterators.size(); i++) { tvListIterators.get(i).setIndex(tvListOffsets[i]); this.tvListOffsets[i] = tvListOffsets[i]; + selectedTVListIndex = -1; } } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java index 78fa143b4d7a9..af8a1cc29f94b 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java @@ -143,12 +143,17 @@ public long calculateRamSize() { return timestamps.size() * tvListArrayMemCost(getDataType()); } - public boolean isSorted() { + public synchronized boolean isSorted() { return sorted; } public abstract void sort(); + public synchronized void safelySort() { + sort(); + seqRowCount = rowCount; + } + public void increaseReferenceCount() { referenceCount.incrementAndGet(); } From 73e9f8300997386192d593db4f300854928c3f12 Mon Sep 17 00:00:00 2001 From: shizy Date: Sat, 7 Dec 2024 15:31:33 +0800 Subject: [PATCH 12/54] tvlist_sort_threshold config property --- .../java/org/apache/iotdb/db/conf/IoTDBConfig.java | 11 +++++++++++ .../org/apache/iotdb/db/conf/IoTDBDescriptor.java | 8 ++++++++ .../dataregion/memtable/IWritableMemChunk.java | 4 ++-- .../dataregion/memtable/WritableMemChunk.java | 4 ++-- .../resources/conf/iotdb-system.properties.template | 4 ++++ 5 files changed, 27 insertions(+), 4 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java index 942fe993c560b..507c6aaf5a201 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java @@ -430,6 +430,9 @@ public class IoTDBConfig { /** The sort algorithm used in TVList */ private TVListSortAlgorithm tvListSortAlgorithm = TVListSortAlgorithm.TIM; + /** the threshold when working TVList is sorted and handover in writable memtable */ + private int tvListSortThreshold = 0; + /** When average series point number reaches this, flush the memtable to disk */ private int avgSeriesPointNumberThreshold = 100000; @@ -2309,6 +2312,14 @@ public void setTvListSortAlgorithm(TVListSortAlgorithm tvListSortAlgorithm) { this.tvListSortAlgorithm = tvListSortAlgorithm; } + public int getTvListSortThreshold() { + return tvListSortThreshold; + } + + public void setTVListSortThreshold(int tvListSortThreshold) { + this.tvListSortThreshold = tvListSortThreshold; + } + public int getAvgSeriesPointNumberThreshold() { return avgSeriesPointNumberThreshold; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java index 37490ec97f249..e4ec90109b670 100755 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java @@ -442,6 +442,14 @@ public void loadProperties(TrimProperties properties) throws BadNodeUrlException .map(String::trim) .orElse(conf.getTvListSortAlgorithm().toString()))); + conf.setTVListSortThreshold( + Integer.parseInt( + Optional.ofNullable( + properties.getProperty( + "tvlist_sort_threshold", Integer.toString(conf.getTvListSortThreshold()))) + .map(String::trim) + .orElse(Integer.toString(conf.getTvListSortThreshold())))); + conf.setAvgSeriesPointNumberThreshold( Integer.parseInt( Optional.ofNullable( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/IWritableMemChunk.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/IWritableMemChunk.java index 8cc10502c7f95..b081e4b65dfef 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/IWritableMemChunk.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/IWritableMemChunk.java @@ -19,6 +19,7 @@ package org.apache.iotdb.db.storageengine.dataregion.memtable; import org.apache.iotdb.common.rpc.thrift.TSStatus; +import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.storageengine.dataregion.wal.buffer.WALEntryValue; import org.apache.iotdb.db.utils.datastructure.TVList; @@ -31,8 +32,7 @@ import java.util.List; public interface IWritableMemChunk extends WALEntryValue { - // TODO: read from configuration file - int SORT_THRESHOLD = 5000; + int TVLIST_SORT_THRESHOLD = IoTDBDescriptor.getInstance().getConfig().getTvListSortThreshold(); boolean putLongWithFlushCheck(long t, long v); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/WritableMemChunk.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/WritableMemChunk.java index eb044ba6a2099..209bb652aa03c 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/WritableMemChunk.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/WritableMemChunk.java @@ -129,7 +129,7 @@ public boolean writeWithFlushCheck(long insertTime, Object objectValue) { return true; } - if (list.rowCount() >= SORT_THRESHOLD) { + if (TVLIST_SORT_THRESHOLD > 0 && list.rowCount() >= TVLIST_SORT_THRESHOLD) { handoverTvList(); } return false; @@ -180,7 +180,7 @@ public boolean writeWithFlushCheck( return true; } - if (list.rowCount() >= SORT_THRESHOLD) { + if (TVLIST_SORT_THRESHOLD > 0 && list.rowCount() >= TVLIST_SORT_THRESHOLD) { handoverTvList(); } return false; diff --git a/iotdb-core/node-commons/src/assembly/resources/conf/iotdb-system.properties.template b/iotdb-core/node-commons/src/assembly/resources/conf/iotdb-system.properties.template index cdcd5a8571a45..e31d369601be2 100644 --- a/iotdb-core/node-commons/src/assembly/resources/conf/iotdb-system.properties.template +++ b/iotdb-core/node-commons/src/assembly/resources/conf/iotdb-system.properties.template @@ -1124,6 +1124,10 @@ unseq_memtable_flush_check_interval_in_ms=30000 # effectiveMode: restart tvlist_sort_algorithm=TIM +# When point number in the working TVList exceeds this, it is sorted and handover in writable memtable +# default 0 means it does not handover working tvlist +tvlist_sort_threshold=0 + # When the average point number of timeseries in memtable exceeds this, the memtable is flushed to disk. The default threshold is 100000. # effectiveMode: restart # Datatype: int From 0545ff1bb164fc89bca42893784d8f048dcd1438 Mon Sep 17 00:00:00 2001 From: shizy Date: Thu, 12 Dec 2024 00:17:01 +0800 Subject: [PATCH 13/54] bug fix: * out of mempage bounds check * overlapped data error during query --- .../dataregion/memtable/ReadOnlyMemChunk.java | 5 ++++- .../read/reader/chunk/MemChunkReader.java | 15 +++++++-------- .../read/reader/chunk/MemChunkLoaderTest.java | 12 ++++++------ 3 files changed, 17 insertions(+), 15 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/ReadOnlyMemChunk.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/ReadOnlyMemChunk.java index dae2dfc4ebde6..ac856b45b6066 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/ReadOnlyMemChunk.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/ReadOnlyMemChunk.java @@ -39,6 +39,7 @@ import org.apache.tsfile.read.common.block.TsBlock; import org.apache.tsfile.read.common.block.TsBlockBuilder; import org.apache.tsfile.read.reader.IPointReader; +import org.apache.tsfile.write.UnSupportedDataTypeException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -199,12 +200,14 @@ private void updateChunkAndPageStatisticsFromTvLists() { pageStatistics.update(tvPair.getTimestamp(), tvPair.getValue().getBinary()); break; default: - // do nothing + throw new UnSupportedDataTypeException( + String.format("Data type %s is not supported.", dataType)); } pageStatistics.setEmpty(false); } cnt++; } + pageOffsetsList.add(Arrays.copyOf(tvListOffsets, tvListOffsets.length)); chunkStatistics.setEmpty(cnt == 0); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemChunkReader.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemChunkReader.java index 5228cf3615088..db107d4b2f586 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemChunkReader.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemChunkReader.java @@ -47,7 +47,7 @@ public class MemChunkReader implements IChunkReader, IPointReader { private final ReadOnlyMemChunk readableChunk; - private final IPointReader timeValuePairIterator; + private final MergeSortTvListIterator timeValuePairIterator; private final Filter globalTimeFilter; private final List pageReaderList; @@ -78,9 +78,9 @@ private void initAllPageReaders( MemPageReader pageReader = new MemPageReader( tsBlockSupplier, - (MergeSortTvListIterator) timeValuePairIterator, + timeValuePairIterator, pageOffsetsList.get(i), - i < pageStats.size() - 1 ? pageOffsetsList.get(i + 1) : null, + pageOffsetsList.get(i + 1), metadata.getDataType(), metadata.getMeasurementUid(), pageStats.get(i), @@ -185,14 +185,13 @@ private boolean isOutOfMemPageBounds() { if (pageEndOffsets == null) { return false; } - int[] currTvListOffsets = - ((MergeSortTvListIterator) timeValuePairIterator).getTVListOffsets(); + int[] currTvListOffsets = timeValuePairIterator.getTVListOffsets(); for (int i = 0; i < pageEndOffsets.length; i++) { - if (currTvListOffsets[i] >= pageEndOffsets[i]) { - return true; + if (currTvListOffsets[i] < pageEndOffsets[i]) { + return false; } } - return false; + return true; } // read one page and write to tsblock diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemChunkLoaderTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemChunkLoaderTest.java index 1f18cc58652ea..ce715e21fa611 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemChunkLoaderTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemChunkLoaderTest.java @@ -68,7 +68,7 @@ public void testBooleanMemChunkLoader() throws IOException { Statistics pageStatitstics = Mockito.mock(Statistics.class); List pageStats = Arrays.asList(pageStatitstics); - List pageOffsets = Arrays.asList(new int[] {0, 0}); + List pageOffsets = Arrays.asList(new int[] {0, 0}, new int[] {2, 1}); Mockito.when(chunk.getPageStatisticsList()).thenReturn(pageStats); Mockito.when(chunk.getPageOffsetsList()).thenReturn(pageOffsets); @@ -143,7 +143,7 @@ public void testInt32MemChunkLoader() throws IOException { Statistics pageStatitstics = Mockito.mock(Statistics.class); List pageStats = Arrays.asList(pageStatitstics); - List pageOffsets = Arrays.asList(new int[] {0, 0}); + List pageOffsets = Arrays.asList(new int[] {0, 0}, new int[] {2, 1}); Mockito.when(chunk.getPageStatisticsList()).thenReturn(pageStats); Mockito.when(chunk.getPageOffsetsList()).thenReturn(pageOffsets); @@ -218,7 +218,7 @@ public void testInt64MemChunkLoader() throws IOException { Statistics pageStatitstics = Mockito.mock(Statistics.class); List pageStats = Arrays.asList(pageStatitstics); - List pageOffsets = Arrays.asList(new int[] {0, 0}); + List pageOffsets = Arrays.asList(new int[] {0, 0}, new int[] {2, 1}); Mockito.when(chunk.getPageStatisticsList()).thenReturn(pageStats); Mockito.when(chunk.getPageOffsetsList()).thenReturn(pageOffsets); @@ -293,7 +293,7 @@ public void testFloatMemChunkLoader() throws IOException { Statistics pageStatitstics = Mockito.mock(Statistics.class); List pageStats = Arrays.asList(pageStatitstics); - List pageOffsets = Arrays.asList(new int[] {0, 0}); + List pageOffsets = Arrays.asList(new int[] {0, 0}, new int[] {2, 1}); Mockito.when(chunk.getPageStatisticsList()).thenReturn(pageStats); Mockito.when(chunk.getPageOffsetsList()).thenReturn(pageOffsets); @@ -368,7 +368,7 @@ public void testDoubleMemChunkLoader() throws IOException { Statistics pageStatitstics = Mockito.mock(Statistics.class); List pageStats = Arrays.asList(pageStatitstics); - List pageOffsets = Arrays.asList(new int[] {0, 0}); + List pageOffsets = Arrays.asList(new int[] {0, 0}, new int[] {2, 1}); Mockito.when(chunk.getPageStatisticsList()).thenReturn(pageStats); Mockito.when(chunk.getPageOffsetsList()).thenReturn(pageOffsets); @@ -443,7 +443,7 @@ public void testTextMemChunkLoader() throws IOException { Statistics pageStatitstics = Mockito.mock(Statistics.class); List pageStats = Arrays.asList(pageStatitstics); - List pageOffsets = Arrays.asList(new int[] {0, 0}); + List pageOffsets = Arrays.asList(new int[] {0, 0}, new int[] {2, 1}); Mockito.when(chunk.getPageStatisticsList()).thenReturn(pageStats); Mockito.when(chunk.getPageOffsetsList()).thenReturn(pageOffsets); From 2fcdcdb8d59e994cdc52037c628685f4210d3ab4 Mon Sep 17 00:00:00 2001 From: shizy Date: Thu, 12 Dec 2024 00:44:42 +0800 Subject: [PATCH 14/54] optimize TVListIterator & MergeSortTvListIterator --- .../MergeSortTvListIterator.java | 19 +++++------ .../iotdb/db/utils/datastructure/TVList.java | 33 ++++++++++++++++--- 2 files changed, 38 insertions(+), 14 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MergeSortTvListIterator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MergeSortTvListIterator.java index 4768c6439700e..24a3a07029064 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MergeSortTvListIterator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MergeSortTvListIterator.java @@ -62,14 +62,10 @@ private void prepareNextRow() { selectedTVListIndex = -1; for (int i = 0; i < tvListIterators.size(); i++) { TVList.TVListIterator iterator = tvListIterators.get(i); - TimeValuePair currTvPair = null; - if (iterator.hasNext()) { - currTvPair = iterator.current(); - } - + boolean hasNext = iterator.hasNext(); // update minimum time and remember selected TVList - if (currTvPair != null && currTvPair.getTimestamp() <= time) { - time = currTvPair.getTimestamp(); + if (hasNext && iterator.currentTime() <= time) { + time = iterator.currentTime(); selectedTVListIndex = i; } } @@ -93,9 +89,12 @@ public TimeValuePair nextTimeValuePair() { // call next to skip identical timestamp in other iterators for (int i = 0; i < tvListIterators.size(); i++) { - TimeValuePair tvPair = tvListIterators.get(i).current(); - if (tvPair != null && tvPair.getTimestamp() == currentTvPair.getTimestamp()) { - tvListIterators.get(i).next(); + if (selectedTVListIndex == i) { + continue; + } + TVList.TVListIterator iterator = tvListIterators.get(i); + if (iterator.hasCurrent() && iterator.currentTime() == currentTvPair.getTimestamp()) { + tvListIterators.get(i).step(); tvListOffsets[i] = tvListIterators.get(i).getIndex(); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java index af8a1cc29f94b..8f13b3a39722d 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java @@ -610,21 +610,24 @@ public TVListIterator iterator() { /* TVList Iterator */ public class TVListIterator { private int index; + private long currentTime; public TVListIterator() { index = 0; + currentTime = index < rowCount ? getTime(index) : Long.MIN_VALUE; } public boolean hasNext() { if (bitMap != null) { // skip deleted & duplicated timestamp while ((index < rowCount && isNullValue(getValueIndex(index))) - || (index + 1 < rowCount && getTime(index + 1) == getTime(index))) { + || (index + 1 < rowCount && getTime(index + 1) == currentTime)) { index++; } + currentTime = index < rowCount ? getTime(index) : Long.MIN_VALUE; } else { // skip duplicated timestamp - while (index + 1 < rowCount && getTime(index + 1) == getTime(index)) { + while (index + 1 < rowCount && getTime(index + 1) == currentTime) { index++; } } @@ -635,22 +638,44 @@ public TimeValuePair next() { if (!hasNext()) { return null; } - return getTimeValuePair(index++); + TimeValuePair ret = getTimeValuePair(index++); + currentTime = index < rowCount ? getTime(index) : Long.MIN_VALUE; + return ret; } public TimeValuePair current() { - if (index >= rowCount || isNullValue(getValueIndex(index))) { + if (!hasCurrent()) { return null; } return getTimeValuePair(index); } + public boolean hasCurrent() { + if (bitMap == null) { + return index < rowCount; + } + return index < rowCount && !isNullValue(getValueIndex(index)); + } + + public long currentTime() { + if (!hasCurrent()) { + return Long.MIN_VALUE; + } + return currentTime; + } + public int getIndex() { return index; } public void setIndex(int index) { this.index = index; + currentTime = index < rowCount ? getTime(index) : Long.MIN_VALUE; + } + + protected void step() { + index++; + currentTime = index < rowCount ? getTime(index) : Long.MIN_VALUE; } } } From 45db409ec2ea9b35c8b4a79fe7fefde3e075c20b Mon Sep 17 00:00:00 2001 From: shizy Date: Fri, 13 Dec 2024 08:35:30 +0800 Subject: [PATCH 15/54] retrofit encode when tvlist_sort_threshold is zero --- .../dataregion/memtable/WritableMemChunk.java | 53 +++++++++++++++++++ 1 file changed, 53 insertions(+) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/WritableMemChunk.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/WritableMemChunk.java index 209bb652aa03c..bb8e782fb91ee 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/WritableMemChunk.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/WritableMemChunk.java @@ -437,6 +437,54 @@ public String toString() { return out.toString(); } + public void encodeWorkingTVList(IChunkWriter chunkWriter) { + ChunkWriterImpl chunkWriterImpl = (ChunkWriterImpl) chunkWriter; + + for (int sortedRowIndex = 0; sortedRowIndex < list.rowCount(); sortedRowIndex++) { + long time = list.getTime(sortedRowIndex); + + TSDataType tsDataType = schema.getType(); + + // skip duplicated data + if ((sortedRowIndex + 1 < list.rowCount() && (time == list.getTime(sortedRowIndex + 1)))) { + continue; + } + + // store last point for SDT + if (sortedRowIndex + 1 == list.rowCount()) { + chunkWriterImpl.setLastPoint(true); + } + + switch (tsDataType) { + case BOOLEAN: + chunkWriterImpl.write(time, list.getBoolean(sortedRowIndex)); + break; + case INT32: + case DATE: + chunkWriterImpl.write(time, list.getInt(sortedRowIndex)); + break; + case INT64: + case TIMESTAMP: + chunkWriterImpl.write(time, list.getLong(sortedRowIndex)); + break; + case FLOAT: + chunkWriterImpl.write(time, list.getFloat(sortedRowIndex)); + break; + case DOUBLE: + chunkWriterImpl.write(time, list.getDouble(sortedRowIndex)); + break; + case TEXT: + case BLOB: + case STRING: + chunkWriterImpl.write(time, list.getBinary(sortedRowIndex)); + break; + default: + LOGGER.error("WritableMemChunk does not support data type: {}", tsDataType); + break; + } + } + } + private void writeData(ChunkWriterImpl chunkWriterImpl, TimeValuePair tvPair) { switch (schema.getType()) { case BOOLEAN: @@ -469,6 +517,11 @@ private void writeData(ChunkWriterImpl chunkWriterImpl, TimeValuePair tvPair) { @Override public void encode(IChunkWriter chunkWriter) { + if (TVLIST_SORT_THRESHOLD == 0) { + encodeWorkingTVList(chunkWriter); + return; + } + ChunkWriterImpl chunkWriterImpl = (ChunkWriterImpl) chunkWriter; // create MergeSortTvListIterator. It need not handle float/double precision here. From 9411feabd4ef4e86397d192da8e4bbc201486c13 Mon Sep 17 00:00:00 2001 From: shizy Date: Wed, 11 Dec 2024 11:46:30 +0800 Subject: [PATCH 16/54] delay sort & statistic generation to query execution --- .../utils/ResourceByPathUtils.java | 2 + .../dataregion/memtable/ReadOnlyMemChunk.java | 30 ++--- .../dataregion/memtable/WritableMemChunk.java | 114 +++++++++--------- .../dataregion/tsfile/TsFileResource.java | 39 +++--- 4 files changed, 97 insertions(+), 88 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/utils/ResourceByPathUtils.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/utils/ResourceByPathUtils.java index 3efcc42aaa9ae..4fad585df542e 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/utils/ResourceByPathUtils.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/utils/ResourceByPathUtils.java @@ -362,6 +362,8 @@ public ITimeSeriesMetadata generateTimeSeriesMetadata( for (ReadOnlyMemChunk memChunk : readOnlyMemChunk) { if (!memChunk.isEmpty()) { + memChunk.sortTvLists(); + memChunk.initChunkMetaFromTvLists(); seriesStatistics.mergeStatistics(memChunk.getChunkMetaData().getStatistics()); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/ReadOnlyMemChunk.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/ReadOnlyMemChunk.java index ac856b45b6066..7edde04874d20 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/ReadOnlyMemChunk.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/ReadOnlyMemChunk.java @@ -125,24 +125,9 @@ public ReadOnlyMemChunk( this.pageStatisticsList = new ArrayList<>(); this.pageOffsetsList = new ArrayList<>(); this.context.addTVListToSet(tvListQueryMap); - - initChunkAndPageStatistics(); - } - - private void initChunkAndPageStatistics() { - // create chunk metadata - Statistics chunkStatistics = Statistics.getStatsByType(dataType); - IChunkMetadata metaData = - new ChunkMetadata(measurementUid, dataType, null, null, 0, chunkStatistics); - metaData.setChunkLoader(new MemChunkLoader(context, this)); - metaData.setVersion(Long.MAX_VALUE); - cachedMetaData = metaData; - - sortTvLists(); - updateChunkAndPageStatisticsFromTvLists(); } - private void sortTvLists() { + public void sortTvLists() { for (Map.Entry entry : getTvListQueryMap().entrySet()) { TVList tvList = entry.getKey(); int queryRowCount = entry.getValue(); @@ -152,9 +137,9 @@ private void sortTvLists() { } } - private void updateChunkAndPageStatisticsFromTvLists() { - Statistics chunkStatistics = cachedMetaData.getStatistics(); - + public void initChunkMetaFromTvLists() { + // create chunk statistics + Statistics chunkStatistics = Statistics.getStatsByType(dataType); int cnt = 0; int[] deleteCursor = {0}; List tvLists = new ArrayList<>(tvListQueryMap.keySet()); @@ -209,6 +194,13 @@ private void updateChunkAndPageStatisticsFromTvLists() { } pageOffsetsList.add(Arrays.copyOf(tvListOffsets, tvListOffsets.length)); chunkStatistics.setEmpty(cnt == 0); + + // chunk meta + IChunkMetadata metaData = + new ChunkMetadata(measurementUid, dataType, null, null, 0, chunkStatistics); + metaData.setChunkLoader(new MemChunkLoader(context, this)); + metaData.setVersion(Long.MAX_VALUE); + cachedMetaData = metaData; } public TSDataType getDataType() { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/WritableMemChunk.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/WritableMemChunk.java index bb8e782fb91ee..7ec9a95a44ccd 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/WritableMemChunk.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/WritableMemChunk.java @@ -99,31 +99,33 @@ protected void handoverTvList() { @Override public boolean writeWithFlushCheck(long insertTime, Object objectValue) { boolean shouldFlush; - switch (schema.getType()) { - case BOOLEAN: - shouldFlush = putBooleanWithFlushCheck(insertTime, (boolean) objectValue); - break; - case INT32: - case DATE: - shouldFlush = putIntWithFlushCheck(insertTime, (int) objectValue); - break; - case INT64: - case TIMESTAMP: - shouldFlush = putLongWithFlushCheck(insertTime, (long) objectValue); - break; - case FLOAT: - shouldFlush = putFloatWithFlushCheck(insertTime, (float) objectValue); - break; - case DOUBLE: - shouldFlush = putDoubleWithFlushCheck(insertTime, (double) objectValue); - break; - case TEXT: - case BLOB: - case STRING: - shouldFlush = putBinaryWithFlushCheck(insertTime, (Binary) objectValue); - break; - default: - throw new UnSupportedDataTypeException(UNSUPPORTED_TYPE + schema.getType().name()); + synchronized (list) { + switch (schema.getType()) { + case BOOLEAN: + shouldFlush = putBooleanWithFlushCheck(insertTime, (boolean) objectValue); + break; + case INT32: + case DATE: + shouldFlush = putIntWithFlushCheck(insertTime, (int) objectValue); + break; + case INT64: + case TIMESTAMP: + shouldFlush = putLongWithFlushCheck(insertTime, (long) objectValue); + break; + case FLOAT: + shouldFlush = putFloatWithFlushCheck(insertTime, (float) objectValue); + break; + case DOUBLE: + shouldFlush = putDoubleWithFlushCheck(insertTime, (double) objectValue); + break; + case TEXT: + case BLOB: + case STRING: + shouldFlush = putBinaryWithFlushCheck(insertTime, (Binary) objectValue); + break; + default: + throw new UnSupportedDataTypeException(UNSUPPORTED_TYPE + schema.getType().name()); + } } if (shouldFlush) { return true; @@ -145,36 +147,38 @@ public boolean writeAlignedValueWithFlushCheck( public boolean writeWithFlushCheck( long[] times, Object valueList, BitMap bitMap, TSDataType dataType, int start, int end) { boolean shouldFlush; - switch (dataType) { - case BOOLEAN: - boolean[] boolValues = (boolean[]) valueList; - shouldFlush = putBooleansWithFlushCheck(times, boolValues, bitMap, start, end); - break; - case INT32: - case DATE: - int[] intValues = (int[]) valueList; - shouldFlush = putIntsWithFlushCheck(times, intValues, bitMap, start, end); - break; - case INT64: - case TIMESTAMP: - long[] longValues = (long[]) valueList; - return putLongsWithFlushCheck(times, longValues, bitMap, start, end); - case FLOAT: - float[] floatValues = (float[]) valueList; - shouldFlush = putFloatsWithFlushCheck(times, floatValues, bitMap, start, end); - break; - case DOUBLE: - double[] doubleValues = (double[]) valueList; - shouldFlush = putDoublesWithFlushCheck(times, doubleValues, bitMap, start, end); - break; - case TEXT: - case BLOB: - case STRING: - Binary[] binaryValues = (Binary[]) valueList; - shouldFlush = putBinariesWithFlushCheck(times, binaryValues, bitMap, start, end); - break; - default: - throw new UnSupportedDataTypeException(UNSUPPORTED_TYPE + dataType.name()); + synchronized (list) { + switch (dataType) { + case BOOLEAN: + boolean[] boolValues = (boolean[]) valueList; + shouldFlush = putBooleansWithFlushCheck(times, boolValues, bitMap, start, end); + break; + case INT32: + case DATE: + int[] intValues = (int[]) valueList; + shouldFlush = putIntsWithFlushCheck(times, intValues, bitMap, start, end); + break; + case INT64: + case TIMESTAMP: + long[] longValues = (long[]) valueList; + return putLongsWithFlushCheck(times, longValues, bitMap, start, end); + case FLOAT: + float[] floatValues = (float[]) valueList; + shouldFlush = putFloatsWithFlushCheck(times, floatValues, bitMap, start, end); + break; + case DOUBLE: + double[] doubleValues = (double[]) valueList; + shouldFlush = putDoublesWithFlushCheck(times, doubleValues, bitMap, start, end); + break; + case TEXT: + case BLOB: + case STRING: + Binary[] binaryValues = (Binary[]) valueList; + shouldFlush = putBinariesWithFlushCheck(times, binaryValues, bitMap, start, end); + break; + default: + throw new UnSupportedDataTypeException(UNSUPPORTED_TYPE + dataType.name()); + } } if (shouldFlush) { return true; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/TsFileResource.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/TsFileResource.java index 8c3d8e76a852c..3422d822595b6 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/TsFileResource.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/TsFileResource.java @@ -70,6 +70,7 @@ import java.io.FileOutputStream; import java.io.IOException; import java.io.InputStream; +import java.io.UncheckedIOException; import java.nio.ByteBuffer; import java.nio.file.Files; import java.util.ArrayList; @@ -83,6 +84,7 @@ import java.util.Objects; import java.util.Set; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.Future; @@ -181,7 +183,8 @@ public class TsFileResource implements PersistentResource { private Map> pathToReadOnlyMemChunkMap = new HashMap<>(); /** used for unsealed file to get TimeseriesMetadata */ - private Map pathToTimeSeriesMetadataMap = new HashMap<>(); + private Map pathToTimeSeriesMetadataMap = + new ConcurrentHashMap<>(); /** * If it is not null, it indicates that the current tsfile resource is a snapshot of the @@ -242,7 +245,6 @@ public TsFileResource( this.timeIndex = originTsFileResource.timeIndex; this.pathToReadOnlyMemChunkMap = pathToReadOnlyMemChunkMap; this.pathToChunkMetadataListMap = pathToChunkMetadataListMap; - generatePathToTimeSeriesMetadataMap(); this.originTsFileResource = originTsFileResource; this.tsFileID = originTsFileResource.tsFileID; this.isSeq = originTsFileResource.isSeq; @@ -996,8 +998,27 @@ public void setProcessor(TsFileProcessor processor) { * * @return TimeseriesMetadata or the first ValueTimeseriesMetadata in VectorTimeseriesMetadata */ - public ITimeSeriesMetadata getTimeSeriesMetadata(IFullPath seriesPath) { - return pathToTimeSeriesMetadataMap.get(seriesPath); + public ITimeSeriesMetadata getTimeSeriesMetadata(IFullPath seriesPath) throws IOException { + try { + return pathToTimeSeriesMetadataMap.computeIfAbsent( + seriesPath, + k -> { + if (pathToChunkMetadataListMap.containsKey(k)) { + try { + return ResourceByPathUtils.getResourceInstance(seriesPath) + .generateTimeSeriesMetadata( + pathToReadOnlyMemChunkMap.get(seriesPath), + pathToChunkMetadataListMap.get(seriesPath)); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } else { + return null; + } + }); + } catch (UncheckedIOException e) { + throw e.getCause(); + } } public DataRegion.SettleTsFileCallBack getSettleTsFileCallBack() { @@ -1272,16 +1293,6 @@ public long degradeTimeIndex() { return beforeRamSize - ramSize; } - private void generatePathToTimeSeriesMetadataMap() throws IOException { - for (IFullPath path : pathToChunkMetadataListMap.keySet()) { - pathToTimeSeriesMetadataMap.put( - path, - ResourceByPathUtils.getResourceInstance(path) - .generateTimeSeriesMetadata( - pathToReadOnlyMemChunkMap.get(path), pathToChunkMetadataListMap.get(path))); - } - } - public void deleteRemovedDeviceAndUpdateEndTime(Map lastTimeForEachDevice) { ITimeIndex newTimeIndex = CONFIG.getTimeIndexLevel().getTimeIndex(); for (Map.Entry entry : lastTimeForEachDevice.entrySet()) { From e3346d3508a40bc5c2a4b299b234a66c7dfae9b3 Mon Sep 17 00:00:00 2001 From: shizy Date: Fri, 13 Dec 2024 14:50:54 +0800 Subject: [PATCH 17/54] fix: skip deleted data during encode --- .../storageengine/dataregion/memtable/WritableMemChunk.java | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/WritableMemChunk.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/WritableMemChunk.java index 7ec9a95a44ccd..ab9267f3a24cd 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/WritableMemChunk.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/WritableMemChunk.java @@ -449,6 +449,10 @@ public void encodeWorkingTVList(IChunkWriter chunkWriter) { TSDataType tsDataType = schema.getType(); + // skip deleted data + if (list.isNullValue(list.getValueIndex(sortedRowIndex))) { + continue; + } // skip duplicated data if ((sortedRowIndex + 1 < list.rowCount() && (time == list.getTime(sortedRowIndex + 1)))) { continue; From 8e0f8e5e91f23a30ae09c36357b25ee293d9372b Mon Sep 17 00:00:00 2001 From: shizy Date: Mon, 16 Dec 2024 08:56:38 +0800 Subject: [PATCH 18/54] aligned time series part --- .../fragment/FragmentInstanceContext.java | 2 +- .../execution/fragment/QueryContext.java | 5 + .../utils/ResourceByPathUtils.java | 88 +++- .../dataregion/memtable/AbstractMemTable.java | 4 +- .../memtable/AlignedReadOnlyMemChunk.java | 408 +++++++++++++----- .../memtable/AlignedWritableMemChunk.java | 265 ++++++++++-- .../AlignedWritableMemChunkGroup.java | 4 +- .../dataregion/memtable/IMemTable.java | 2 +- .../memtable/IWritableMemChunk.java | 4 +- .../memtable/IWritableMemChunkGroup.java | 2 +- .../dataregion/memtable/ReadOnlyMemChunk.java | 36 +- .../dataregion/memtable/TsFileProcessor.java | 6 +- .../dataregion/memtable/WritableMemChunk.java | 6 +- .../memtable/WritableMemChunkGroup.java | 2 +- .../reader/chunk/MemAlignedChunkReader.java | 153 ++++++- .../reader/chunk/MemAlignedPageReader.java | 144 ++++++- .../read/reader/chunk/MemChunkReader.java | 5 +- .../read/reader/chunk/MemPageReader.java | 8 +- .../chunk/metadata/AlignedPageMetadata.java | 75 ++++ .../db/utils/datastructure/AlignedTVList.java | 256 ++++++++++- .../MergeSortAlignedTVListIterator.java | 143 ++++++ .../MergeSortTvListIterator.java | 45 +- .../iotdb/db/utils/datastructure/TVList.java | 60 ++- .../chunk/AlignedMemPageReaderTest.java | 35 +- .../chunk/MemAlignedChunkLoaderTest.java | 130 +++--- 25 files changed, 1565 insertions(+), 323 deletions(-) create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/metadata/AlignedPageMetadata.java create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MergeSortAlignedTVListIterator.java diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceContext.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceContext.java index 187c0a9065562..73e4b0b5f28b0 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceContext.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceContext.java @@ -699,7 +699,7 @@ public synchronized void releaseResource() { unClosedFilePaths = null; } - // release TVList owned by current query + // release TVList/AlignedTVList owned by current query releaseTVListOwnedByQuery(); dataRegion = null; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/QueryContext.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/QueryContext.java index 242d92a3d5ae2..e0abc8e424786 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/QueryContext.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/QueryContext.java @@ -26,6 +26,7 @@ import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileID; import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; import org.apache.iotdb.db.utils.ModificationUtils; +import org.apache.iotdb.db.utils.datastructure.AlignedTVList; import org.apache.iotdb.db.utils.datastructure.PatternTreeMapFactory; import org.apache.iotdb.db.utils.datastructure.PatternTreeMapFactory.ModsSerializer; import org.apache.iotdb.db.utils.datastructure.TVList; @@ -215,4 +216,8 @@ public void setIgnoreAllNullRows(boolean ignoreAllNullRows) { public void addTVListToSet(Map tvListMap) { tvListSet.addAll(tvListMap.keySet()); } + + public void addAlignedTVListToSet(Map alignedTvListMap) { + tvListSet.addAll(alignedTvListMap.keySet()); + } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/utils/ResourceByPathUtils.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/utils/ResourceByPathUtils.java index 4fad585df542e..91793daaa4cda 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/utils/ResourceByPathUtils.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/utils/ResourceByPathUtils.java @@ -35,6 +35,7 @@ import org.apache.iotdb.db.storageengine.dataregion.modification.ModEntry; import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; import org.apache.iotdb.db.utils.ModificationUtils; +import org.apache.iotdb.db.utils.datastructure.AlignedTVList; import org.apache.iotdb.db.utils.datastructure.TVList; import org.apache.tsfile.enums.TSDataType; @@ -102,6 +103,7 @@ public abstract List getVisibleMetadataListFromWriter( } class AlignedResourceByPathUtils extends ResourceByPathUtils { + private static final Logger LOGGER = LoggerFactory.getLogger(AlignedResourceByPathUtils.class); AlignedFullPath alignedFullPath; @@ -155,6 +157,8 @@ public AlignedTimeSeriesMetadata generateTimeSeriesMetadata( for (ReadOnlyMemChunk memChunk : readOnlyMemChunk) { if (!memChunk.isEmpty()) { + memChunk.sortTvLists(); + memChunk.initChunkMetaFromTvLists(); AlignedChunkMetadata alignedChunkMetadata = (AlignedChunkMetadata) memChunk.getChunkMetaData(); timeStatistics.mergeStatistics(alignedChunkMetadata.getTimeChunkMetadata().getStatistics()); @@ -183,6 +187,73 @@ public AlignedTimeSeriesMetadata generateTimeSeriesMetadata( return new AlignedTimeSeriesMetadata(timeTimeSeriesMetadata, valueTimeSeriesMetadataList); } + private Map prepareAlignedTvListMapForQuery( + QueryContext context, + AlignedWritableMemChunk alignedMemChunk, + boolean isWorkMemTable, + Filter globalTimeFilter) { + Map alignedTvListQueryMap = new LinkedHashMap<>(); + // immutable aligned TVList + for (AlignedTVList alignedTvList : alignedMemChunk.getSortedList()) { + if (globalTimeFilter != null + && !globalTimeFilter.satisfyStartEndTime( + alignedTvList.getMinTime(), alignedTvList.getMaxTime())) { + continue; + } + alignedTvList.lockQueryList(); + try { + LOGGER.debug( + "Flushing/Working MemTable - add current query context to immutable AlignedTVList's query list"); + alignedTvList.getQueryContextList().add(context); + alignedTvListQueryMap.put(alignedTvList, alignedTvList.rowCount()); + } finally { + alignedTvList.unlockQueryList(); + } + } + + // mutable aligned TVList + AlignedTVList list = alignedMemChunk.getWorkingTVList(); + list.lockQueryList(); + try { + if (!isWorkMemTable) { + if (globalTimeFilter == null + || globalTimeFilter.satisfyStartEndTime(list.getMinTime(), list.getMaxTime())) { + LOGGER.debug( + "Flushing MemTable - add current query context to mutable AlignedTVList's query list"); + list.getQueryContextList().add(context); + alignedTvListQueryMap.put(list, list.rowCount()); + } + } else { + if (list.isSorted() || list.getQueryContextList().isEmpty()) { + LOGGER.debug( + "Working MemTable - add current query context to mutable AlignedTVList's query list when it's sorted or no other query on it"); + list.getQueryContextList().add(context); + alignedTvListQueryMap.put(list, list.rowCount()); + } else { + LOGGER.debug( + "Working MemTable - clone mutable AlignedTVList and replace old AlignedTVList in working MemTable"); + QueryContext firstQuery = list.getQueryContextList().get(0); + // reserve query memory + if (firstQuery instanceof FragmentInstanceContext) { + MemoryReservationManager memoryReservationManager = + ((FragmentInstanceContext) firstQuery).getMemoryReservationContext(); + memoryReservationManager.reserveMemoryCumulatively(list.calculateRamSize()); + } + list.setOwnerQuery(firstQuery); + + // clone TVList + AlignedTVList cloneList = list.clone(); + cloneList.getQueryContextList().add(context); + alignedTvListQueryMap.put(cloneList, cloneList.rowCount()); + alignedMemChunk.setWorkingTVList(cloneList); + } + } + } finally { + list.unlockQueryList(); + } + return alignedTvListQueryMap; + } + @Override public ReadOnlyMemChunk getReadOnlyMemChunkFromMemTable( QueryContext context, @@ -214,10 +285,16 @@ public ReadOnlyMemChunk getReadOnlyMemChunkFromMemTable( } } - // get sorted tv list is synchronized so different query can get right sorted list reference - TVList alignedTvListCopy = - alignedMemChunk.getSortedTvListForQuery( - alignedFullPath.getSchemaList(), context.isIgnoreAllNullRows()); + // prepare AlignedTVList for query. It should clone and sort TVList if necessary. + // Also, the map keeps AlignedTVList length at this moment. + Map alignedTvListQueryMap = + prepareAlignedTvListMapForQuery( + context, alignedMemChunk, modsToMemtable == null, globalTimeFilter); + + // column index list for the query + List columnIndexList = + alignedMemChunk.getColumnIndexList(alignedFullPath.getSchemaList()); + List timeColumnDeletion = null; List> valueColumnsDeletionList = null; if (modsToMemtable != null) { @@ -239,8 +316,9 @@ public ReadOnlyMemChunk getReadOnlyMemChunkFromMemTable( } return new AlignedReadOnlyMemChunk( context, + columnIndexList, getMeasurementSchema(), - alignedTvListCopy, + alignedTvListQueryMap, timeColumnDeletion, valueColumnsDeletionList); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AbstractMemTable.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AbstractMemTable.java index f256db485dc53..3a343243c851a 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AbstractMemTable.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AbstractMemTable.java @@ -401,12 +401,12 @@ public boolean chunkNotExist(IDeviceID deviceId, String measurement) { } @Override - public long getCurrentTVListSize(IDeviceID deviceId, String measurement) { + public long getMeasurementSize(IDeviceID deviceId, String measurement) { IWritableMemChunkGroup memChunkGroup = memTableMap.get(deviceId); if (null == memChunkGroup) { return 0; } - return memChunkGroup.getCurrentTVListSize(measurement); + return memChunkGroup.getMeasurementSize(measurement); } @Override diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedReadOnlyMemChunk.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedReadOnlyMemChunk.java index 569db7347ab56..944bbc5fd6b7f 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedReadOnlyMemChunk.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedReadOnlyMemChunk.java @@ -23,8 +23,10 @@ import org.apache.iotdb.db.queryengine.execution.fragment.QueryContext; import org.apache.iotdb.db.storageengine.dataregion.read.reader.chunk.MemAlignedChunkLoader; import org.apache.iotdb.db.utils.datastructure.AlignedTVList; +import org.apache.iotdb.db.utils.datastructure.MergeSortAlignedTVListIterator; import org.apache.iotdb.db.utils.datastructure.TVList; +import org.apache.tsfile.block.column.ColumnBuilder; import org.apache.tsfile.common.conf.TSFileDescriptor; import org.apache.tsfile.enums.TSDataType; import org.apache.tsfile.file.metadata.AlignedChunkMetadata; @@ -32,12 +34,21 @@ import org.apache.tsfile.file.metadata.IChunkMetadata; import org.apache.tsfile.file.metadata.enums.TSEncoding; import org.apache.tsfile.file.metadata.statistics.Statistics; +import org.apache.tsfile.read.TimeValuePair; import org.apache.tsfile.read.common.TimeRange; +import org.apache.tsfile.read.common.block.TsBlock; +import org.apache.tsfile.read.common.block.TsBlockBuilder; import org.apache.tsfile.read.reader.IPointReader; +import org.apache.tsfile.utils.TsPrimitiveType; +import org.apache.tsfile.write.UnSupportedDataTypeException; import org.apache.tsfile.write.schema.IMeasurementSchema; +import java.io.IOException; +import java.io.Serializable; import java.util.ArrayList; +import java.util.Arrays; import java.util.List; +import java.util.Map; public class AlignedReadOnlyMemChunk extends ReadOnlyMemChunk { private final String timeChunkName; @@ -46,133 +57,336 @@ public class AlignedReadOnlyMemChunk extends ReadOnlyMemChunk { private final List dataTypes; + private final int floatPrecision; + private final List encodingList; + + private final List timeColumnDeletion; + private final List> valueColumnsDeletionList; + + // time & values statistics + private final List> timeStatisticsList; + private final List>> valueStatisticsList; + + // AlignedTVList rowCount during query + protected Map alignedTvListQueryMap; + + private final List columnIndexList; + /** * The constructor for Aligned type. * + * @param context query context * @param schema VectorMeasurementSchema - * @param tvList VectorTvList - * @param deletionList The timeRange of deletionList + * @param alignedTvListQueryMap VectorTvList + * @param timeColumnDeletion The timeRange of deletionList + * @param valueColumnsDeletionList time value column deletionList * @throws QueryProcessException if there is unsupported data type. */ public AlignedReadOnlyMemChunk( QueryContext context, + List columnIndexList, IMeasurementSchema schema, - TVList tvList, + Map alignedTvListQueryMap, List timeColumnDeletion, - List> valueColumnsDeletionList) - throws QueryProcessException { + List> valueColumnsDeletionList) { super(context); + this.pageOffsetsList = new ArrayList<>(); this.timeChunkName = schema.getMeasurementName(); this.valueChunkNames = schema.getSubMeasurementsList(); this.dataTypes = schema.getSubMeasurementsTSDataTypeList(); - int floatPrecision = TSFileDescriptor.getInstance().getConfig().getFloatPrecision(); - List encodingList = schema.getSubMeasurementsTSEncodingList(); - this.tsBlock = - ((AlignedTVList) tvList) - .buildTsBlock( - floatPrecision, - encodingList, - timeColumnDeletion, - valueColumnsDeletionList, - context.isIgnoreAllNullRows()); - initAlignedChunkMetaFromTsBlock(); - } - - private void initAlignedChunkMetaFromTsBlock() throws QueryProcessException { - // Time chunk - Statistics timeStatistics = Statistics.getStatsByType(TSDataType.VECTOR); - IChunkMetadata timeChunkMetadata = - new ChunkMetadata(timeChunkName, TSDataType.VECTOR, null, null, 0, timeStatistics); - List valueChunkMetadataList = new ArrayList<>(); - // Update time chunk - for (int row = 0; row < tsBlock.getPositionCount(); row++) { - timeStatistics.update(tsBlock.getTimeColumn().getLong(row)); + this.floatPrecision = TSFileDescriptor.getInstance().getConfig().getFloatPrecision(); + this.encodingList = schema.getSubMeasurementsTSEncodingList(); + this.timeColumnDeletion = timeColumnDeletion; + this.valueColumnsDeletionList = valueColumnsDeletionList; + this.timeStatisticsList = new ArrayList<>(); + this.valueStatisticsList = new ArrayList<>(); + this.alignedTvListQueryMap = alignedTvListQueryMap; + this.columnIndexList = columnIndexList; + this.context.addAlignedTVListToSet(alignedTvListQueryMap); + } + + @Override + public void sortTvLists() { + for (Map.Entry entry : getAligendTvListQueryMap().entrySet()) { + AlignedTVList alignedTvList = entry.getKey(); + int queryRowCount = entry.getValue(); + if (!alignedTvList.isSorted() && queryRowCount > alignedTvList.seqRowCount()) { + alignedTvList.safelySort(); + } } - timeStatistics.setEmpty(false); - // Update value chunk - for (int column = 0; column < tsBlock.getValueColumnCount(); column++) { - Statistics valueStatistics = Statistics.getStatsByType(dataTypes.get(column)); - valueStatistics.setEmpty(true); - switch (dataTypes.get(column)) { - case BOOLEAN: - for (int row = 0; row < tsBlock.getPositionCount(); row++) { - if (!tsBlock.getColumn(column).isNull(row)) { - long time = tsBlock.getTimeColumn().getLong(row); - valueStatistics.update(time, tsBlock.getColumn(column).getBoolean(row)); - } - } - break; - case TEXT: - case BLOB: - case STRING: - for (int row = 0; row < tsBlock.getPositionCount(); row++) { - if (!tsBlock.getColumn(column).isNull(row)) { - long time = tsBlock.getTimeColumn().getLong(row); - valueStatistics.update(time, tsBlock.getColumn(column).getBinary(row)); - } - } - break; - case FLOAT: - for (int row = 0; row < tsBlock.getPositionCount(); row++) { - if (!tsBlock.getColumn(column).isNull(row)) { - long time = tsBlock.getTimeColumn().getLong(row); - valueStatistics.update(time, tsBlock.getColumn(column).getFloat(row)); - } - } - break; - case INT32: - case DATE: - for (int row = 0; row < tsBlock.getPositionCount(); row++) { - if (!tsBlock.getColumn(column).isNull(row)) { - long time = tsBlock.getTimeColumn().getLong(row); - valueStatistics.update(time, tsBlock.getColumn(column).getInt(row)); - } - } - break; - case INT64: - case TIMESTAMP: - for (int row = 0; row < tsBlock.getPositionCount(); row++) { - if (!tsBlock.getColumn(column).isNull(row)) { - long time = tsBlock.getTimeColumn().getLong(row); - valueStatistics.update(time, tsBlock.getColumn(column).getLong(row)); - } - } - break; - case DOUBLE: - for (int row = 0; row < tsBlock.getPositionCount(); row++) { - if (!tsBlock.getColumn(column).isNull(row)) { - long time = tsBlock.getTimeColumn().getLong(row); - valueStatistics.update(time, tsBlock.getColumn(column).getDouble(row)); - } - } - break; - default: - throw new QueryProcessException("Unsupported data type:" + dataTypes.get(column)); + } + + @Override + public void initChunkMetaFromTvLists() { + // init chunk meta + Statistics chunkTimeStatistics = + Statistics.getStatsByType(TSDataType.VECTOR); + IChunkMetadata chunkTimeMetadata = + new ChunkMetadata(timeChunkName, TSDataType.VECTOR, null, null, 0, chunkTimeStatistics); + List> chunkValueStatistics = new ArrayList<>(); + List chunkValueMetadataList = new ArrayList<>(); + for (int column = 0; column < valueChunkNames.size(); column++) { + Statistics valueStatistics = + Statistics.getStatsByType(dataTypes.get(column)); + chunkValueStatistics.add(valueStatistics); + IChunkMetadata valueChunkMetadata = + new ChunkMetadata( + valueChunkNames.get(column), dataTypes.get(column), null, null, 0, valueStatistics); + chunkValueMetadataList.add(valueChunkMetadata); + } + + int cnt = 0; + List alignedTvLists = new ArrayList<>(alignedTvListQueryMap.keySet()); + MergeSortAlignedTVListIterator timeValuePairIterator = + new MergeSortAlignedTVListIterator( + alignedTvLists, + columnIndexList, + floatPrecision, + encodingList, + timeColumnDeletion, + valueColumnsDeletionList, + context.isIgnoreAllNullRows()); + int[] alignedTvListOffsets = timeValuePairIterator.getAlignedTVListOffsets(); + while (timeValuePairIterator.hasNextTimeValuePair()) { + // Split pages + if (cnt % MAX_NUMBER_OF_POINTS_IN_PAGE == 0) { + Statistics pageTimeStatistics = + Statistics.getStatsByType(TSDataType.VECTOR); + pageTimeStatistics.setEmpty(false); + timeStatisticsList.add(pageTimeStatistics); + List> pageValueStatistics = new ArrayList<>(); + for (int column = 0; column < valueChunkNames.size(); column++) { + Statistics valueStatistics = + Statistics.getStatsByType(dataTypes.get(column)); + pageValueStatistics.add(valueStatistics); + } + valueStatisticsList.add(pageValueStatistics); + pageOffsetsList.add(Arrays.copyOf(alignedTvListOffsets, alignedTvListOffsets.length)); } - if (valueStatistics.getCount() > 0) { - IChunkMetadata valueChunkMetadata = - new ChunkMetadata( - valueChunkNames.get(column), dataTypes.get(column), null, null, 0, valueStatistics); - valueChunkMetadataList.add(valueChunkMetadata); - valueStatistics.setEmpty(false); - } else { - valueChunkMetadataList.add(null); + + // Update Page & Chunk Statistics + TimeValuePair tvPair = timeValuePairIterator.nextTimeValuePair(); + Statistics pageTimeStats = + timeStatisticsList.get(timeStatisticsList.size() - 1); + pageTimeStats.update(tvPair.getTimestamp()); + chunkTimeStatistics.update(tvPair.getTimestamp()); + + List> pageValuesStats = + valueStatisticsList.get(valueStatisticsList.size() - 1); + TsPrimitiveType[] primitiveValues = tvPair.getValue().getVector(); + for (int column = 0; column < primitiveValues.length; column++) { + if (primitiveValues[column] == null) { + continue; + } + switch (dataTypes.get(column)) { + case BOOLEAN: + pageValuesStats + .get(column) + .update(tvPair.getTimestamp(), primitiveValues[column].getBoolean()); + chunkValueStatistics + .get(column) + .update(tvPair.getTimestamp(), primitiveValues[column].getBoolean()); + break; + case INT32: + case DATE: + pageValuesStats + .get(column) + .update(tvPair.getTimestamp(), primitiveValues[column].getInt()); + chunkValueStatistics + .get(column) + .update(tvPair.getTimestamp(), primitiveValues[column].getInt()); + break; + case INT64: + case TIMESTAMP: + pageValuesStats + .get(column) + .update(tvPair.getTimestamp(), primitiveValues[column].getLong()); + chunkValueStatistics + .get(column) + .update(tvPair.getTimestamp(), primitiveValues[column].getLong()); + break; + case FLOAT: + pageValuesStats + .get(column) + .update(tvPair.getTimestamp(), primitiveValues[column].getFloat()); + chunkValueStatistics + .get(column) + .update(tvPair.getTimestamp(), primitiveValues[column].getFloat()); + break; + case DOUBLE: + pageValuesStats + .get(column) + .update(tvPair.getTimestamp(), primitiveValues[column].getDouble()); + chunkValueStatistics + .get(column) + .update(tvPair.getTimestamp(), primitiveValues[column].getDouble()); + break; + case TEXT: + case BLOB: + case STRING: + pageValuesStats + .get(column) + .update(tvPair.getTimestamp(), primitiveValues[column].getBinary()); + chunkValueStatistics + .get(column) + .update(tvPair.getTimestamp(), primitiveValues[column].getBinary()); + break; + default: + throw new UnSupportedDataTypeException( + String.format("Data type %s is not supported.", dataTypes.get(column))); + } + } + cnt++; + } + pageOffsetsList.add(Arrays.copyOf(alignedTvListOffsets, alignedTvListOffsets.length)); + chunkTimeStatistics.setEmpty(cnt == 0); + + // statistics should be set null if there is no data + for (int column = 0; column < chunkValueMetadataList.size(); column++) { + if (chunkValueMetadataList.get(column).getStatistics().isEmpty()) { + chunkValueMetadataList.set(column, null); + } + } + for (List> pageValueStats : valueStatisticsList) { + for (int column = 0; column < pageValueStats.size(); column++) { + if (pageValueStats.get(column).isEmpty()) { + pageValueStats.set(column, null); + } } } + + // aligned chunk meta IChunkMetadata alignedChunkMetadata = - new AlignedChunkMetadata(timeChunkMetadata, valueChunkMetadataList); + new AlignedChunkMetadata(chunkTimeMetadata, chunkValueMetadataList); alignedChunkMetadata.setChunkLoader(new MemAlignedChunkLoader(context, this)); alignedChunkMetadata.setVersion(Long.MAX_VALUE); cachedMetaData = alignedChunkMetadata; } + private int count() { + int count = 0; + for (TVList list : alignedTvListQueryMap.keySet()) { + count += list.count(); + } + return count; + } + @Override public boolean isEmpty() { - return tsBlock.isEmpty(); + return count() == 0; } @Override public IPointReader getPointReader() { + for (Map.Entry entry : alignedTvListQueryMap.entrySet()) { + AlignedTVList tvList = entry.getKey(); + int queryLength = entry.getValue(); + if (!tvList.isSorted() && queryLength > tvList.seqRowCount()) { + tvList.safelySort(); + } + } + TsBlock tsBlock = buildTsBlock(); return tsBlock.getTsBlockAlignedRowIterator(); } + + private TsBlock buildTsBlock() { + try { + TsBlockBuilder builder = new TsBlockBuilder(dataTypes); + writeValidValuesIntoTsBlock(builder); + return builder.build(); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + private void writeValidValuesIntoTsBlock(TsBlockBuilder builder) throws IOException { + List alignedTvLists = new ArrayList<>(alignedTvListQueryMap.keySet()); + IPointReader timeValuePairIterator = + new MergeSortAlignedTVListIterator( + alignedTvLists, + columnIndexList, + floatPrecision, + encodingList, + timeColumnDeletion, + valueColumnsDeletionList, + context.isIgnoreAllNullRows()); + while (timeValuePairIterator.hasNextTimeValuePair()) { + TimeValuePair tvPair = timeValuePairIterator.nextTimeValuePair(); + builder.getTimeColumnBuilder().writeLong(tvPair.getTimestamp()); + + // value columns + TsPrimitiveType[] values = tvPair.getValue().getVector(); + for (int columnIndex = 0; columnIndex < values.length; columnIndex++) { + if (values[columnIndex] == null) { + builder.getColumnBuilder(columnIndex).appendNull(); + continue; + } + ColumnBuilder valueBuilder = builder.getColumnBuilder(columnIndex); + switch (dataTypes.get(columnIndex)) { + case BOOLEAN: + valueBuilder.writeBoolean(values[columnIndex].getBoolean()); + break; + case INT32: + case DATE: + valueBuilder.writeInt(values[columnIndex].getInt()); + break; + case INT64: + case TIMESTAMP: + valueBuilder.writeLong(values[columnIndex].getLong()); + break; + case FLOAT: + valueBuilder.writeFloat(values[columnIndex].getFloat()); + break; + case DOUBLE: + valueBuilder.writeDouble(values[columnIndex].getDouble()); + break; + case TEXT: + case BLOB: + case STRING: + valueBuilder.writeBinary(values[columnIndex].getBinary()); + break; + default: + break; + } + } + builder.declarePosition(); + } + } + + public Map getAligendTvListQueryMap() { + return alignedTvListQueryMap; + } + + @Override + public int getFloatPrecision() { + return floatPrecision; + } + + public List getColumnIndexList() { + return columnIndexList; + } + + public List getTimeColumnDeletion() { + return timeColumnDeletion; + } + + public List> getValueColumnsDeletionList() { + return valueColumnsDeletionList; + } + + public List getEncodingList() { + return encodingList; + } + + public List getDataTypes() { + return dataTypes; + } + + public List> getTimeStatisticsList() { + return timeStatisticsList; + } + + public List>> getValuesStatisticsList() { + return valueStatisticsList; + } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedWritableMemChunk.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedWritableMemChunk.java index 8cabea95fe2df..8e00c21be07b8 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedWritableMemChunk.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedWritableMemChunk.java @@ -20,19 +20,26 @@ package org.apache.iotdb.db.storageengine.dataregion.memtable; import org.apache.iotdb.common.rpc.thrift.TSStatus; +import org.apache.iotdb.db.queryengine.execution.fragment.FragmentInstanceContext; +import org.apache.iotdb.db.queryengine.execution.fragment.QueryContext; +import org.apache.iotdb.db.queryengine.plan.planner.memory.MemoryReservationManager; import org.apache.iotdb.db.storageengine.dataregion.wal.buffer.IWALByteBufferView; import org.apache.iotdb.db.storageengine.dataregion.wal.utils.WALWriteUtils; import org.apache.iotdb.db.utils.datastructure.AlignedTVList; +import org.apache.iotdb.db.utils.datastructure.MergeSortAlignedTVListIterator; import org.apache.iotdb.db.utils.datastructure.TVList; import org.apache.tsfile.common.conf.TSFileDescriptor; import org.apache.tsfile.enums.TSDataType; +import org.apache.tsfile.read.TimeValuePair; import org.apache.tsfile.utils.Binary; import org.apache.tsfile.utils.BitMap; import org.apache.tsfile.utils.Pair; +import org.apache.tsfile.utils.TsPrimitiveType; import org.apache.tsfile.write.UnSupportedDataTypeException; import org.apache.tsfile.write.chunk.AlignedChunkWriterImpl; import org.apache.tsfile.write.chunk.IChunkWriter; +import org.apache.tsfile.write.chunk.ValueChunkWriter; import org.apache.tsfile.write.schema.IMeasurementSchema; import org.apache.tsfile.write.schema.MeasurementSchema; @@ -52,6 +59,7 @@ public class AlignedWritableMemChunk implements IWritableMemChunk { private final Map measurementIndexMap; private final List schemaList; private AlignedTVList list; + private List sortedList; private boolean ignoreAllNullRows; private static final int MAX_NUMBER_OF_POINTS_IN_PAGE = @@ -68,6 +76,7 @@ public AlignedWritableMemChunk(List schemaList, boolean isTa dataTypeList.add(schemaList.get(i).getType()); } this.list = AlignedTVList.newAlignedList(dataTypeList); + this.sortedList = new ArrayList<>(); this.ignoreAllNullRows = !isTableModel; } @@ -79,6 +88,7 @@ private AlignedWritableMemChunk( measurementIndexMap.put(schemaList.get(i).getMeasurementName(), i); } this.list = list; + this.sortedList = new ArrayList<>(); this.ignoreAllNullRows = !isTableModel; } @@ -122,8 +132,10 @@ public boolean putBooleanWithFlushCheck(long t, boolean v) { @Override public boolean putAlignedValueWithFlushCheck(long t, Object[] v) { - list.putAlignedValue(t, v); - return list.reachChunkSizeOrPointNumThreshold(); + synchronized (list) { + list.putAlignedValue(t, v); + return list.reachChunkSizeOrPointNumThreshold(); + } } @Override @@ -161,8 +173,10 @@ public boolean putBooleansWithFlushCheck( @Override public boolean putAlignedValuesWithFlushCheck( long[] t, Object[] v, BitMap[] bitMaps, int start, int end, TSStatus[] results) { - list.putAlignedValues(t, v, bitMaps, start, end, results); - return list.reachChunkSizeOrPointNumThreshold(); + synchronized (list) { + list.putAlignedValues(t, v, bitMaps, start, end, results); + return list.reachChunkSizeOrPointNumThreshold(); + } } @Override @@ -176,12 +190,54 @@ public boolean writeWithFlushCheck( throw new UnSupportedDataTypeException(UNSUPPORTED_TYPE + TSDataType.VECTOR); } + protected void handoverAlignedTvList() { + // ensure query contexts won't be removed from list during handover process. + list.lockQueryList(); + try { + if (list.isSorted()) { + sortedList.add(list); + return; + } + + if (list.getQueryContextList().isEmpty()) { + list.safelySort(); + sortedList.add(list); + } else { + QueryContext firstQuery = list.getQueryContextList().get(0); + // reserve query memory + if (firstQuery instanceof FragmentInstanceContext) { + MemoryReservationManager memoryReservationManager = + ((FragmentInstanceContext) firstQuery).getMemoryReservationContext(); + memoryReservationManager.reserveMemoryCumulatively(list.calculateRamSize()); + } + // update current TVList owner to first query in the list + list.setOwnerQuery(firstQuery); + // clone tv list + AlignedTVList cloneList = list.clone(); + sortedList.add(cloneList); + } + } finally { + list.unlockQueryList(); + } + List dataTypeList = new ArrayList<>(); + for (IMeasurementSchema schema : schemaList) { + dataTypeList.add(schema.getType()); + } + this.list = AlignedTVList.newAlignedList(dataTypeList); + } + @Override public boolean writeAlignedValueWithFlushCheck( long insertTime, Object[] objectValue, List schemaList) { Object[] reorderedValue = checkAndReorderColumnValuesInInsertPlan(schemaList, objectValue, null).left; - return putAlignedValueWithFlushCheck(insertTime, reorderedValue); + if (putAlignedValueWithFlushCheck(insertTime, reorderedValue)) { + return true; + } + if (TVLIST_SORT_THRESHOLD > 0 && list.rowCount() >= TVLIST_SORT_THRESHOLD) { + handoverAlignedTvList(); + } + return false; } @Override @@ -197,8 +253,14 @@ public boolean writeAlignedValuesWithFlushCheck( checkAndReorderColumnValuesInInsertPlan(schemaList, valueList, bitMaps); Object[] reorderedColumnValues = pair.left; BitMap[] reorderedBitMaps = pair.right; - return putAlignedValuesWithFlushCheck( - times, reorderedColumnValues, reorderedBitMaps, start, end, results); + if (putAlignedValuesWithFlushCheck( + times, reorderedColumnValues, reorderedBitMaps, start, end, results)) { + return true; + } + if (TVLIST_SORT_THRESHOLD > 0 && list.rowCount() >= TVLIST_SORT_THRESHOLD) { + handoverAlignedTvList(); + } + return false; } /** @@ -241,20 +303,33 @@ private Pair checkAndReorderColumnValuesInInsertPlan( } @Override - public TVList getWorkingTVList() { + public AlignedTVList getWorkingTVList() { return list; } + public void setWorkingTVList(AlignedTVList list) { + this.list = list; + } + @Override public long count() { if (!ignoreAllNullRows && measurementIndexMap.isEmpty()) { - return list.rowCount(); + return rowCount(); } - return (long) list.rowCount() * measurementIndexMap.size(); + return rowCount() * measurementIndexMap.size(); + } + + @Override + public long rowCount() { + return alignedListSize(); } public int alignedListSize() { - return list.rowCount(); + int rowCount = list.rowCount(); + for (AlignedTVList alignedTvList : sortedList) { + rowCount += alignedTvList.rowCount(); + } + return rowCount; } @Override @@ -264,7 +339,20 @@ public IMeasurementSchema getSchema() { @Override public long getMaxTime() { - return list.getMaxTime(); + long maxTime = list.getMaxTime(); + for (AlignedTVList alignedTvList : sortedList) { + maxTime = Math.max(maxTime, alignedTvList.getMaxTime()); + } + return maxTime; + } + + @Override + public long getMinTime() { + long minTime = list.getMinTime(); + for (AlignedTVList alignedTvList : sortedList) { + minTime = Math.min(minTime, alignedTvList.getMinTime()); + } + return minTime; } @Override @@ -304,25 +392,46 @@ private void sortTVList() { @Override public synchronized void sortTvListForFlush() { - sortTVList(); + if (!list.isSorted()) { + list.safelySort(); + } } @Override public int delete(long lowerBound, long upperBound) { - return list.delete(lowerBound, upperBound); + int deletedNumber = list.delete(lowerBound, upperBound); + for (AlignedTVList alignedTvList : sortedList) { + deletedNumber += alignedTvList.delete(lowerBound, upperBound); + } + return deletedNumber; } public int deleteTime(long lowerBound, long upperBound) { - return list.deleteTime(lowerBound, upperBound); + int deletedNumber = list.deleteTime(lowerBound, upperBound); + for (AlignedTVList alignedTvList : sortedList) { + deletedNumber += alignedTvList.deleteTime(lowerBound, upperBound); + } + return deletedNumber; } public Pair deleteDataFromAColumn( long lowerBound, long upperBound, String measurementId) { - return list.delete(lowerBound, upperBound, measurementIndexMap.get(measurementId)); + Pair deletePair = + list.delete(lowerBound, upperBound, measurementIndexMap.get(measurementId)); + for (AlignedTVList alignedTvList : sortedList) { + Pair p = + alignedTvList.delete(lowerBound, upperBound, measurementIndexMap.get(measurementId)); + deletePair.left += p.left; + deletePair.right = deletePair.right && p.right; + } + return deletePair; } public void removeColumn(String measurementId) { list.deleteColumn(measurementIndexMap.get(measurementId)); + for (AlignedTVList alignedTvList : sortedList) { + alignedTvList.deleteColumn(measurementIndexMap.get(measurementId)); + } IMeasurementSchema schemaToBeRemoved = schemaList.get(measurementIndexMap.get(measurementId)); schemaList.remove(schemaToBeRemoved); measurementIndexMap.clear(); @@ -336,9 +445,7 @@ public IChunkWriter createIChunkWriter() { return new AlignedChunkWriterImpl(schemaList); } - @SuppressWarnings({"squid:S6541", "squid:S3776"}) - @Override - public void encode(IChunkWriter chunkWriter) { + private void encodeWorkingAlignedTVList(IChunkWriter chunkWriter) { AlignedChunkWriterImpl alignedChunkWriter = (AlignedChunkWriterImpl) chunkWriter; BitMap allValueColDeletedMap; @@ -484,34 +591,116 @@ public void encode(IChunkWriter chunkWriter) { } } + @SuppressWarnings({"squid:S6541", "squid:S3776"}) + @Override + public void encode(IChunkWriter chunkWriter) { + if (TVLIST_SORT_THRESHOLD == 0) { + encodeWorkingAlignedTVList(chunkWriter); + return; + } + + AlignedChunkWriterImpl alignedChunkWriter = (AlignedChunkWriterImpl) chunkWriter; + // create MergeSortAlignedTVListIterator. + List alignedTvLists = new ArrayList<>(sortedList); + alignedTvLists.add(list); + MergeSortAlignedTVListIterator timeValuePairIterator = + new MergeSortAlignedTVListIterator( + alignedTvLists, null, null, null, null, null, ignoreAllNullRows); + + int pointsInPage = 0; + long[] times = new long[MAX_NUMBER_OF_POINTS_IN_PAGE]; + while (timeValuePairIterator.hasNextTimeValuePair()) { + TimeValuePair tvPair = timeValuePairIterator.nextTimeValuePair(); + times[pointsInPage] = tvPair.getTimestamp(); + TsPrimitiveType[] values = tvPair.getValue().getVector(); + for (int columnIndex = 0; columnIndex < values.length; columnIndex++) { + ValueChunkWriter valueChunkWriter = + alignedChunkWriter.getValueChunkWriterByIndex(columnIndex); + boolean isNull = values[columnIndex].getValue() == null; + switch (schemaList.get(columnIndex).getType()) { + case BOOLEAN: + valueChunkWriter.write(tvPair.getTimestamp(), values[columnIndex].getBoolean(), isNull); + break; + case INT32: + case DATE: + valueChunkWriter.write(tvPair.getTimestamp(), values[columnIndex].getInt(), isNull); + break; + case INT64: + case TIMESTAMP: + valueChunkWriter.write(tvPair.getTimestamp(), values[columnIndex].getLong(), isNull); + break; + case FLOAT: + valueChunkWriter.write(tvPair.getTimestamp(), values[columnIndex].getFloat(), isNull); + break; + case DOUBLE: + valueChunkWriter.write(tvPair.getTimestamp(), values[columnIndex].getDouble(), isNull); + break; + case TEXT: + case BLOB: + case STRING: + valueChunkWriter.write(tvPair.getTimestamp(), values[columnIndex].getBinary(), isNull); + break; + default: + break; + } + } + pointsInPage++; + if (pointsInPage % MAX_NUMBER_OF_POINTS_IN_PAGE == 0) { + alignedChunkWriter.write(times, pointsInPage, 0); + pointsInPage = 0; + } + } + } + + private void maybeReleaseTvList(AlignedTVList alignedTvList) { + alignedTvList.lockQueryList(); + try { + if (alignedTvList.getQueryContextList().isEmpty()) { + alignedTvList.clear(); + } else { + QueryContext firstQuery = alignedTvList.getQueryContextList().get(0); + // transfer memory from write process to read process. Here it reserves read memory and + // releaseFlushedMemTable will release write memory. + if (firstQuery instanceof FragmentInstanceContext) { + MemoryReservationManager memoryReservationManager = + ((FragmentInstanceContext) firstQuery).getMemoryReservationContext(); + memoryReservationManager.reserveMemoryCumulatively(alignedTvList.calculateRamSize()); + } + // update current TVList owner to first query in the list + alignedTvList.setOwnerQuery(firstQuery); + } + } finally { + alignedTvList.unlockQueryList(); + } + } + @Override public void release() { - if (list.getReferenceCount() == 0) { - list.clear(); + maybeReleaseTvList(list); + for (AlignedTVList alignedTvList : sortedList) { + maybeReleaseTvList(alignedTvList); } } @Override public long getFirstPoint() { - if (list.rowCount() == 0) { + if (rowCount() == 0) { return Long.MAX_VALUE; } - return getSortedTvListForQuery().getTimeValuePair(0).getTimestamp(); + return getMinTime(); } @Override public long getLastPoint() { - if (list.rowCount() == 0) { + if (rowCount() == 0) { return Long.MIN_VALUE; } - return getSortedTvListForQuery() - .getTimeValuePair(getSortedTvListForQuery().rowCount() - 1) - .getTimestamp(); + return getMaxTime(); } @Override public boolean isEmpty() { - return list.rowCount() == 0; + return rowCount() == 0; } @Override @@ -522,6 +711,9 @@ public int serializedSize() { size += schema.serializedSize(); } + for (AlignedTVList alignedTvList : sortedList) { + size += alignedTvList.serializedSize(); + } size += list.serializedSize(); return size; } @@ -534,7 +726,9 @@ public void serializeToWAL(IWALByteBufferView buffer) { schema.serializeTo(ByteBuffer.wrap(bytes)); buffer.put(bytes); } - + for (AlignedTVList alignedTvList : sortedList) { + alignedTvList.serializeToWAL(buffer); + } list.serializeToWAL(buffer); } @@ -554,4 +748,17 @@ public static AlignedWritableMemChunk deserialize(DataInputStream stream, boolea public List getSchemaList() { return schemaList; } + + public List getSortedList() { + return sortedList; + } + + public List getColumnIndexList(List schemaList) { + List columnIndexList = new ArrayList<>(); + for (IMeasurementSchema measurementSchema : schemaList) { + columnIndexList.add( + measurementIndexMap.getOrDefault(measurementSchema.getMeasurementName(), -1)); + } + return columnIndexList; + } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedWritableMemChunkGroup.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedWritableMemChunkGroup.java index efe64bf5b6e1a..99fc37b273611 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedWritableMemChunkGroup.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedWritableMemChunkGroup.java @@ -128,8 +128,8 @@ public long deleteTime(ModEntry modEntry) { } @Override - public long getCurrentTVListSize(String measurement) { - return memChunk.getWorkingTVList().rowCount(); + public long getMeasurementSize(String measurement) { + return memChunk.rowCount(); } @Override diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/IMemTable.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/IMemTable.java index fbd24aea16f1d..9c4581292fd68 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/IMemTable.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/IMemTable.java @@ -171,7 +171,7 @@ void queryForDeviceRegionScan( boolean chunkNotExist(IDeviceID deviceId, String measurement); /** only used when mem control enabled */ - long getCurrentTVListSize(IDeviceID deviceId, String measurement); + long getMeasurementSize(IDeviceID deviceId, String measurement); /** only used when mem control enabled */ void addTextDataSize(long textDataIncrement); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/IWritableMemChunk.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/IWritableMemChunk.java index b081e4b65dfef..38008be773e56 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/IWritableMemChunk.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/IWritableMemChunk.java @@ -86,9 +86,7 @@ boolean writeAlignedValuesWithFlushCheck( long count(); - default long rowCount() { - return 0; - } + long rowCount(); IMeasurementSchema getSchema(); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/IWritableMemChunkGroup.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/IWritableMemChunkGroup.java index 353da05bf283d..4a06060dc65a0 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/IWritableMemChunkGroup.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/IWritableMemChunkGroup.java @@ -55,7 +55,7 @@ boolean writeWithFlushCheck( long deleteTime(ModEntry modEntry); - long getCurrentTVListSize(String measurement); + long getMeasurementSize(String measurement); long getMaxTime(); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/ReadOnlyMemChunk.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/ReadOnlyMemChunk.java index 7edde04874d20..49b08fa72b489 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/ReadOnlyMemChunk.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/ReadOnlyMemChunk.java @@ -68,8 +68,6 @@ public class ReadOnlyMemChunk { protected IChunkMetadata cachedMetaData; - protected TsBlock tsBlock; - private int floatPrecision; private TSEncoding encoding; private List deletionList; @@ -77,12 +75,14 @@ public class ReadOnlyMemChunk { // Read only chunk is now regarded as multiple pages. Apart from chunk statistics, // we need to collect page statistic and MergeSortTvListIterator offset for each page. private List pageStatisticsList; - private List pageOffsetsList; + + // page offsets + protected List pageOffsetsList; // tvlist rowCount during query - protected Map tvListQueryMap; + private Map tvListQueryMap; - public static final int MAX_NUMBER_OF_POINTS_IN_PAGE = + protected static final int MAX_NUMBER_OF_POINTS_IN_PAGE = TSFileDescriptor.getInstance().getConfig().getMaxNumberOfPointsInPage(); protected ReadOnlyMemChunk(QueryContext context) { @@ -144,7 +144,7 @@ public void initChunkMetaFromTvLists() { int[] deleteCursor = {0}; List tvLists = new ArrayList<>(tvListQueryMap.keySet()); MergeSortTvListIterator timeValuePairIterator = - new MergeSortTvListIterator(dataType, encoding, floatPrecision, tvLists); + new MergeSortTvListIterator(tvLists, floatPrecision, encoding); int[] tvListOffsets = timeValuePairIterator.getTVListOffsets(); while (timeValuePairIterator.hasNextTimeValuePair()) { if (cnt % MAX_NUMBER_OF_POINTS_IN_PAGE == 0) { @@ -188,12 +188,10 @@ public void initChunkMetaFromTvLists() { throw new UnSupportedDataTypeException( String.format("Data type %s is not supported.", dataType)); } - pageStatistics.setEmpty(false); } cnt++; } pageOffsetsList.add(Arrays.copyOf(tvListOffsets, tvListOffsets.length)); - chunkStatistics.setEmpty(cnt == 0); // chunk meta IChunkMetadata metaData = @@ -208,10 +206,7 @@ public TSDataType getDataType() { } public boolean isEmpty() { - if (tsBlock == null) { - return count() == 0; - } - return tsBlock.isEmpty(); + return count() == 0; } public IChunkMetadata getChunkMetaData() { @@ -246,7 +241,7 @@ private void writeValidValuesIntoTsBlock(TsBlockBuilder builder) throws IOExcept int[] deleteCursor = {0}; List tvLists = new ArrayList<>(tvListQueryMap.keySet()); IPointReader timeValuePairIterator = - new MergeSortTvListIterator(dataType, encoding, floatPrecision, tvLists); + new MergeSortTvListIterator(tvLists, floatPrecision, encoding); while (timeValuePairIterator.hasNextTimeValuePair()) { TimeValuePair tvPair = timeValuePairIterator.nextTimeValuePair(); @@ -293,15 +288,11 @@ private void writeValidValuesIntoTsBlock(TsBlockBuilder builder) throws IOExcept } } - public TsBlock getTsBlock() { - return tsBlock; - } - public Map getTvListQueryMap() { return tvListQueryMap; } - public int count() { + private int count() { int count = 0; for (TVList list : tvListQueryMap.keySet()) { count += list.count(); @@ -328,4 +319,13 @@ public List getPageStatisticsList() { public List getPageOffsetsList() { return pageOffsetsList; } + + public QueryContext getContext() { + return context; + } + + @TestOnly + public TsBlock getTsBlock() { + return null; + } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/TsFileProcessor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/TsFileProcessor.java index 7992c1debf3ad..6883b3d6562ae 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/TsFileProcessor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/TsFileProcessor.java @@ -655,7 +655,7 @@ private long[] checkMemCostAndAddToTspInfoForRow( memTableIncrement += TVList.tvListArrayMemCost(dataTypes[i]); } else { // here currentChunkPointNum >= 1 - long currentChunkPointNum = workMemTable.getCurrentTVListSize(deviceId, measurements[i]); + long currentChunkPointNum = workMemTable.getMeasurementSize(deviceId, measurements[i]); memTableIncrement += (currentChunkPointNum % PrimitiveArrayManager.ARRAY_SIZE) == 0 ? TVList.tvListArrayMemCost(dataTypes[i]) @@ -700,7 +700,7 @@ private long[] checkMemCostAndAddToTspInfoForRows(List insertRowN .putIfAbsent(measurements[i], 1); } else { // here currentChunkPointNum >= 1 - long currentChunkPointNum = workMemTable.getCurrentTVListSize(deviceId, measurements[i]); + long currentChunkPointNum = workMemTable.getMeasurementSize(deviceId, measurements[i]); int addingPointNum = increasingMemTableInfo .computeIfAbsent(deviceId, k -> new HashMap<>()) @@ -966,7 +966,7 @@ private void updateMemCost( ((end - start) / PrimitiveArrayManager.ARRAY_SIZE + 1) * TVList.tvListArrayMemCost(dataType); } else { - long currentChunkPointNum = workMemTable.getCurrentTVListSize(deviceId, measurement); + long currentChunkPointNum = workMemTable.getMeasurementSize(deviceId, measurement); if (currentChunkPointNum % PrimitiveArrayManager.ARRAY_SIZE == 0) { memIncrements[0] += ((end - start) / PrimitiveArrayManager.ARRAY_SIZE + 1) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/WritableMemChunk.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/WritableMemChunk.java index ab9267f3a24cd..967ed485a9e12 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/WritableMemChunk.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/WritableMemChunk.java @@ -535,11 +535,11 @@ public void encode(IChunkWriter chunkWriter) { // create MergeSortTvListIterator. It need not handle float/double precision here. List tvLists = new ArrayList<>(sortedList); tvLists.add(list); - MergeSortTvListIterator iterator = new MergeSortTvListIterator(schema.getType(), tvLists); + MergeSortTvListIterator timeValuePairIterator = new MergeSortTvListIterator(tvLists); TimeValuePair prevTvPair = null; - while (iterator.hasNextTimeValuePair()) { - TimeValuePair currTvPair = iterator.nextTimeValuePair(); + while (timeValuePairIterator.hasNextTimeValuePair()) { + TimeValuePair currTvPair = timeValuePairIterator.nextTimeValuePair(); if (prevTvPair == null) { prevTvPair = currTvPair; continue; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/WritableMemChunkGroup.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/WritableMemChunkGroup.java index 9ecd6a46d0fb7..e814b9f59b728 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/WritableMemChunkGroup.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/WritableMemChunkGroup.java @@ -147,7 +147,7 @@ public long deleteTime(ModEntry modEntry) { } @Override - public long getCurrentTVListSize(String measurement) { + public long getMeasurementSize(String measurement) { if (!memChunkMap.containsKey(measurement)) { return 0; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemAlignedChunkReader.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemAlignedChunkReader.java index 33c85408f0d38..0c23bf0751411 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemAlignedChunkReader.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemAlignedChunkReader.java @@ -20,30 +20,75 @@ package org.apache.iotdb.db.storageengine.dataregion.read.reader.chunk; import org.apache.iotdb.db.storageengine.dataregion.memtable.AlignedReadOnlyMemChunk; +import org.apache.iotdb.db.utils.datastructure.AlignedTVList; +import org.apache.iotdb.db.utils.datastructure.MergeSortAlignedTVListIterator; -import org.apache.tsfile.file.metadata.AlignedChunkMetadata; +import org.apache.tsfile.block.column.ColumnBuilder; +import org.apache.tsfile.enums.TSDataType; +import org.apache.tsfile.file.metadata.statistics.Statistics; +import org.apache.tsfile.read.TimeValuePair; import org.apache.tsfile.read.common.BatchData; +import org.apache.tsfile.read.common.block.TsBlock; +import org.apache.tsfile.read.common.block.TsBlockBuilder; import org.apache.tsfile.read.filter.basic.Filter; import org.apache.tsfile.read.reader.IChunkReader; import org.apache.tsfile.read.reader.IPageReader; +import org.apache.tsfile.utils.TsPrimitiveType; import java.io.IOException; -import java.util.Collections; +import java.io.Serializable; +import java.util.ArrayList; import java.util.List; +import java.util.function.Supplier; /** To read aligned chunk data in memory. */ public class MemAlignedChunkReader implements IChunkReader { - + private final AlignedReadOnlyMemChunk readableChunk; + private final MergeSortAlignedTVListIterator timeValuePairIterator; + private final Filter globalTimeFilter; private final List pageReaderList; public MemAlignedChunkReader(AlignedReadOnlyMemChunk readableChunk, Filter globalTimeFilter) { - // we treat one ReadOnlyMemChunk as one Page - this.pageReaderList = - Collections.singletonList( - new MemAlignedPageReader( - readableChunk.getTsBlock(), - (AlignedChunkMetadata) readableChunk.getChunkMetaData(), - globalTimeFilter)); + this.readableChunk = readableChunk; + List alignedTVLists = + new ArrayList<>(readableChunk.getAligendTvListQueryMap().keySet()); + timeValuePairIterator = + new MergeSortAlignedTVListIterator( + alignedTVLists, + readableChunk.getColumnIndexList(), + readableChunk.getFloatPrecision(), + readableChunk.getEncodingList(), + readableChunk.getTimeColumnDeletion(), + readableChunk.getValueColumnsDeletionList(), + readableChunk.getContext().isIgnoreAllNullRows()); + this.globalTimeFilter = globalTimeFilter; + this.pageReaderList = new ArrayList<>(); + initAllPageReaders( + readableChunk.getDataTypes(), + readableChunk.getTimeStatisticsList(), + readableChunk.getValuesStatisticsList(), + readableChunk.getPageOffsetsList()); + } + + private void initAllPageReaders( + List tsDataTypes, + List> timeStatistics, + List>> valuesStatistics, + List pageOffsetsList) { + Supplier tsBlockSupplier = new MemAlignedChunkReader.TsBlockSupplier(); + for (int i = 0; i < timeStatistics.size(); i++) { + MemAlignedPageReader pageReader = + new MemAlignedPageReader( + tsBlockSupplier, + timeValuePairIterator, + pageOffsetsList.get(i), + i + 1 < timeStatistics.size() ? pageOffsetsList.get(i + 1) : null, + tsDataTypes, + timeStatistics.get(i), + valuesStatistics.get(i), + globalTimeFilter); + this.pageReaderList.add(pageReader); + } } @Override @@ -65,4 +110,92 @@ public void close() { public List loadPageReaderList() { return this.pageReaderList; } + + class TsBlockSupplier implements Supplier { + private int[] pageEndOffsets; + + public TsBlockSupplier() {} + + public void setPageEndOffsets(int[] pageEndOffsets) { + this.pageEndOffsets = pageEndOffsets; + } + + @Override + public TsBlock get() { + return buildTsBlock(); + } + + private TsBlock buildTsBlock() { + try { + List tsDataTypes = readableChunk.getDataTypes(); + TsBlockBuilder builder = new TsBlockBuilder(tsDataTypes); + writeValidValuesIntoTsBlock(builder); + return builder.build(); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + private boolean isOutOfMemPageBounds() { + if (pageEndOffsets == null) { + return false; + } + int[] currTvListOffsets = timeValuePairIterator.getAlignedTVListOffsets(); + for (int i = 0; i < pageEndOffsets.length; i++) { + if (currTvListOffsets[i] < pageEndOffsets[i]) { + return false; + } + } + return true; + } + + // read one page and write to tsblock + private synchronized void writeValidValuesIntoTsBlock(TsBlockBuilder builder) { + List tsDataTypes = readableChunk.getDataTypes(); + while (timeValuePairIterator.hasNextTimeValuePair()) { + if (isOutOfMemPageBounds()) { + break; + } + TimeValuePair tvPair = timeValuePairIterator.nextTimeValuePair(); + builder.getTimeColumnBuilder().writeLong(tvPair.getTimestamp()); + + // value columns + TsPrimitiveType[] values = tvPair.getValue().getVector(); + for (int columnIndex = 0; columnIndex < values.length; columnIndex++) { + if (values[columnIndex] == null) { + builder.getColumnBuilder(columnIndex).appendNull(); + continue; + } + ColumnBuilder valueBuilder = builder.getColumnBuilder(columnIndex); + switch (tsDataTypes.get(columnIndex)) { + case BOOLEAN: + valueBuilder.writeBoolean(values[columnIndex].getBoolean()); + break; + case INT32: + case DATE: + valueBuilder.writeInt(values[columnIndex].getInt()); + break; + case INT64: + case TIMESTAMP: + valueBuilder.writeLong(values[columnIndex].getLong()); + break; + case FLOAT: + valueBuilder.writeFloat(values[columnIndex].getFloat()); + break; + case DOUBLE: + valueBuilder.writeDouble(values[columnIndex].getDouble()); + break; + case TEXT: + case BLOB: + case STRING: + valueBuilder.writeBinary(values[columnIndex].getBinary()); + break; + default: + break; + } + } + builder.declarePosition(); + } + } + } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemAlignedPageReader.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemAlignedPageReader.java index 473cd7366c33c..973652d19e5d4 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemAlignedPageReader.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemAlignedPageReader.java @@ -19,10 +19,12 @@ package org.apache.iotdb.db.storageengine.dataregion.read.reader.chunk; +import org.apache.iotdb.db.storageengine.dataregion.read.reader.chunk.metadata.AlignedPageMetadata; +import org.apache.iotdb.db.utils.datastructure.MergeSortAlignedTVListIterator; + import org.apache.tsfile.block.column.Column; import org.apache.tsfile.block.column.ColumnBuilder; import org.apache.tsfile.enums.TSDataType; -import org.apache.tsfile.file.metadata.AlignedChunkMetadata; import org.apache.tsfile.file.metadata.statistics.Statistics; import org.apache.tsfile.read.common.BatchData; import org.apache.tsfile.read.common.BatchDataFactory; @@ -33,19 +35,28 @@ import org.apache.tsfile.read.reader.IPageReader; import org.apache.tsfile.read.reader.series.PaginationController; import org.apache.tsfile.utils.TsPrimitiveType; +import org.apache.tsfile.write.UnSupportedDataTypeException; import java.io.IOException; import java.io.Serializable; +import java.util.ArrayList; import java.util.Arrays; import java.util.List; import java.util.Optional; +import java.util.function.Supplier; import static org.apache.tsfile.read.reader.series.PaginationController.UNLIMITED_PAGINATION_CONTROLLER; public class MemAlignedPageReader implements IPageReader { - private final TsBlock tsBlock; - private final AlignedChunkMetadata chunkMetadata; + private TsBlock tsBlock; + private final AlignedPageMetadata pageMetadata; + + private final MergeSortAlignedTVListIterator mergeSortAlignedTVListIterator; + private final int[] pageStartOffsets; + private final int[] pageEndOffsets; + private final Supplier tsBlockSupplier; + private final List tsDataTypes; private Filter recordFilter; private PaginationController paginationController = UNLIMITED_PAGINATION_CONTROLLER; @@ -53,10 +64,21 @@ public class MemAlignedPageReader implements IPageReader { private TsBlockBuilder builder; public MemAlignedPageReader( - TsBlock tsBlock, AlignedChunkMetadata chunkMetadata, Filter recordFilter) { - this.tsBlock = tsBlock; - this.chunkMetadata = chunkMetadata; + Supplier tsBlockSupplier, + MergeSortAlignedTVListIterator mergeSortAlignedTVListIterator, + int[] pageStartOffsets, + int[] pageEndOffSets, + List tsDataTypes, + Statistics timeStatistics, + List> valueStatistics, + Filter recordFilter) { + this.tsBlockSupplier = tsBlockSupplier; + this.mergeSortAlignedTVListIterator = mergeSortAlignedTVListIterator; + this.pageStartOffsets = pageStartOffsets; + this.pageEndOffsets = pageEndOffSets; this.recordFilter = recordFilter; + this.tsDataTypes = tsDataTypes; + this.pageMetadata = new AlignedPageMetadata(timeStatistics, valueStatistics); } @Override @@ -66,6 +88,8 @@ public BatchData getAllSatisfiedPageData() throws IOException { @Override public BatchData getAllSatisfiedPageData(boolean ascending) throws IOException { + getTsBlock(); + BatchData batchData = BatchDataFactory.createBatchData(TSDataType.VECTOR, ascending, false); boolean[] satisfyInfo = buildSatisfyInfoArray(); @@ -87,6 +111,8 @@ public BatchData getAllSatisfiedPageData(boolean ascending) throws IOException { @Override public TsBlock getAllSatisfiedData() { + getTsBlock(); + builder.reset(); boolean[] satisfyInfo = buildSatisfyInfoArray(); @@ -158,23 +184,23 @@ private void buildValueColumns(boolean[] satisfyInfo, int readEndIndex) { @Override public Statistics getStatistics() { - return chunkMetadata.getStatistics(); + return pageMetadata.getStatistics(); } @Override public Statistics getTimeStatistics() { - return chunkMetadata.getTimeStatistics(); + return pageMetadata.getTimeStatistics(); } @Override public Optional> getMeasurementStatistics( int measurementIndex) { - return chunkMetadata.getMeasurementStatistics(measurementIndex); + return pageMetadata.getMeasurementStatistics(measurementIndex); } @Override public boolean hasNullValue(int measurementIndex) { - return chunkMetadata.hasNullValue(measurementIndex); + return pageMetadata.hasNullValue(measurementIndex); } @Override @@ -196,4 +222,102 @@ public boolean isModified() { public void initTsBlockBuilder(List dataTypes) { builder = new TsBlockBuilder(dataTypes); } + + private void getTsBlock() { + if (tsBlock == null) { + initializeOffsets(); + tsBlock = tsBlockSupplier.get(); + if (pageMetadata.getStatistics() == null) { + initPageStatistics(); + } + } + } + + private void initializeOffsets() { + if (pageStartOffsets != null) { + mergeSortAlignedTVListIterator.setAlignedTVListOffsets(pageStartOffsets); + } + if (tsBlockSupplier instanceof MemAlignedChunkReader.TsBlockSupplier) { + ((MemAlignedChunkReader.TsBlockSupplier) tsBlockSupplier).setPageEndOffsets(pageEndOffsets); + } + } + + private void initPageStatistics() { + Statistics pageTimeStatistics = + Statistics.getStatsByType(TSDataType.VECTOR); + List> pageValueStatistics = new ArrayList<>(); + for (int column = 0; column < tsDataTypes.size(); column++) { + Statistics valueStatistics = + Statistics.getStatsByType(tsDataTypes.get(column)); + pageValueStatistics.add(valueStatistics); + } + updatePageStatisticsFromTsBlock(pageTimeStatistics, pageValueStatistics); + pageMetadata.setStatistics(pageTimeStatistics, pageValueStatistics); + } + + private void updatePageStatisticsFromTsBlock( + Statistics timeStatistics, + List> valueStatistics) { + if (!tsBlock.isEmpty()) { + // update time statistics + for (int i = 0; i < tsBlock.getPositionCount(); i++) { + timeStatistics.update(tsBlock.getTimeByIndex(i)); + } + timeStatistics.setEmpty(false); + + for (int column = 0; column < tsDataTypes.size(); column++) { + switch (tsDataTypes.get(column)) { + case BOOLEAN: + for (int i = 0; i < tsBlock.getPositionCount(); i++) { + valueStatistics + .get(column) + .update(tsBlock.getTimeByIndex(i), tsBlock.getColumn(column).getBoolean(i)); + } + break; + case INT32: + case DATE: + for (int i = 0; i < tsBlock.getPositionCount(); i++) { + valueStatistics + .get(column) + .update(tsBlock.getTimeByIndex(i), tsBlock.getColumn(column).getInt(i)); + } + break; + case INT64: + case TIMESTAMP: + for (int i = 0; i < tsBlock.getPositionCount(); i++) { + valueStatistics + .get(column) + .update(tsBlock.getTimeByIndex(i), tsBlock.getColumn(column).getLong(i)); + } + break; + case FLOAT: + for (int i = 0; i < tsBlock.getPositionCount(); i++) { + valueStatistics + .get(column) + .update(tsBlock.getTimeByIndex(i), tsBlock.getColumn(column).getFloat(i)); + } + break; + case DOUBLE: + for (int i = 0; i < tsBlock.getPositionCount(); i++) { + valueStatistics + .get(column) + .update(tsBlock.getTimeByIndex(i), tsBlock.getColumn(column).getDouble(i)); + } + break; + case TEXT: + case BLOB: + case STRING: + for (int i = 0; i < tsBlock.getPositionCount(); i++) { + valueStatistics + .get(column) + .update(tsBlock.getTimeByIndex(i), tsBlock.getColumn(column).getBinary(i)); + } + break; + default: + throw new UnSupportedDataTypeException( + String.format("Data type %s is not supported.", tsDataTypes.get(column))); + } + } + } + } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemChunkReader.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemChunkReader.java index db107d4b2f586..bee00bebcee10 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemChunkReader.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemChunkReader.java @@ -59,10 +59,7 @@ public MemChunkReader(ReadOnlyMemChunk readableChunk, Filter globalTimeFilter) { List tvLists = new ArrayList<>(readableChunk.getTvListQueryMap().keySet()); timeValuePairIterator = new MergeSortTvListIterator( - readableChunk.getDataType(), - readableChunk.getEncoding(), - readableChunk.getFloatPrecision(), - tvLists); + tvLists, readableChunk.getFloatPrecision(), readableChunk.getEncoding()); this.globalTimeFilter = globalTimeFilter; this.pageReaderList = new ArrayList<>(); initAllPageReaders( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemPageReader.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemPageReader.java index 21668df320e9a..ac46d6c5ff422 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemPageReader.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemPageReader.java @@ -261,14 +261,14 @@ private void initializeOffsets() { // We do the initialization if it is not set, especially in test cases. private void initPageStatistics() { Statistics statistics = Statistics.getStatsByType(tsDataType); - updatePageStatisticsFromTsBlock(statistics, tsDataType); + updatePageStatisticsFromTsBlock(statistics); statistics.setEmpty(tsBlock.isEmpty()); pageMetadata.setStatistics(statistics); } - private void updatePageStatisticsFromTsBlock(Statistics statistics, TSDataType dataType) { + private void updatePageStatisticsFromTsBlock(Statistics statistics) { if (!tsBlock.isEmpty()) { - switch (dataType) { + switch (tsDataType) { case BOOLEAN: for (int i = 0; i < tsBlock.getPositionCount(); i++) { statistics.update(tsBlock.getTimeByIndex(i), tsBlock.getColumn(0).getBoolean(i)); @@ -305,7 +305,7 @@ private void updatePageStatisticsFromTsBlock(Statistics statistics, TSDataType d break; default: throw new UnSupportedDataTypeException( - String.format("Data type %s is not supported.", dataType)); + String.format("Data type %s is not supported.", tsDataType)); } } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/metadata/AlignedPageMetadata.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/metadata/AlignedPageMetadata.java new file mode 100644 index 0000000000000..5ab43857522df --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/metadata/AlignedPageMetadata.java @@ -0,0 +1,75 @@ +/* + * 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.storageengine.dataregion.read.reader.chunk.metadata; + +import org.apache.tsfile.file.metadata.IMetadata; +import org.apache.tsfile.file.metadata.statistics.Statistics; + +import java.io.Serializable; +import java.util.List; +import java.util.Optional; + +public class AlignedPageMetadata implements IMetadata { + private Statistics timeStatistics; + private List> valueStatistics; + + public AlignedPageMetadata( + Statistics timeStatistics, + List> valuesStatistics) { + this.timeStatistics = timeStatistics; + this.valueStatistics = valuesStatistics; + } + + @Override + public Statistics getStatistics() { + return valueStatistics.size() == 1 && valueStatistics.get(0) != null + ? valueStatistics.get(0) + : timeStatistics; + } + + @Override + public Statistics getTimeStatistics() { + return timeStatistics; + } + + @Override + public Optional> getMeasurementStatistics( + int measurementIndex) { + return Optional.ofNullable( + measurementIndex >= valueStatistics.size() ? null : valueStatistics.get(measurementIndex)); + } + + @Override + public boolean hasNullValue(int measurementIndex) { + if (measurementIndex >= valueStatistics.size()) { + return false; + } + long rowCount = getTimeStatistics().getCount(); + Statistics stats = valueStatistics.get(measurementIndex); + return stats != null && stats.hasNullValue(rowCount); + } + + public void setStatistics( + Statistics timeStatistics, + List> valueStatistics) { + this.timeStatistics = timeStatistics; + this.valueStatistics = valueStatistics; + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/AlignedTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/AlignedTVList.java index 2c00e3c78d086..594ed3afce729 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/AlignedTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/AlignedTVList.java @@ -182,6 +182,7 @@ public void putAlignedValue(long timestamp, Object[] value) { int arrayIndex = rowCount / ARRAY_SIZE; int elementIndex = rowCount % ARRAY_SIZE; maxTime = Math.max(maxTime, timestamp); + minTime = Math.min(minTime, timestamp); timestamps.get(arrayIndex)[elementIndex] = timestamp; for (int i = 0; i < values.size(); i++) { Object columnValue = value[i]; @@ -242,7 +243,7 @@ public void putAlignedValue(long timestamp, Object[] value) { @Override public Object getAlignedValue(int index) { - return getAlignedValueForQuery(index, null, null); + return getAlignedValueForQuery(index, null, null, null); } @Override @@ -254,30 +255,41 @@ protected TimeValuePair getTimeValuePair( @Override public TimeValuePair getTimeValuePair(int index) { return new TimeValuePair( - getTime(index), (TsPrimitiveType) getAlignedValueForQuery(index, null, null)); + getTime(index), (TsPrimitiveType) getAlignedValueForQuery(index, null, null, null)); } private Object getAlignedValueForQuery( - int index, Integer floatPrecision, List encodingList) { + int index, + Integer floatPrecision, + List encodingList, + List columnIndexList) { if (index >= rowCount) { throw new ArrayIndexOutOfBoundsException(index); } int arrayIndex = index / ARRAY_SIZE; int elementIndex = index % ARRAY_SIZE; int valueIndex = indices.get(arrayIndex)[elementIndex]; - return getAlignedValueByValueIndex(valueIndex, null, floatPrecision, encodingList); + return getAlignedValueByValueIndex( + valueIndex, null, floatPrecision, encodingList, columnIndexList); } private TsPrimitiveType getAlignedValueByValueIndex( int valueIndex, int[] validIndexesForTimeDuplicatedRows, Integer floatPrecision, - List encodingList) { + List encodingList, + List columnIndexList) { if (valueIndex >= rowCount) { throw new ArrayIndexOutOfBoundsException(valueIndex); } - TsPrimitiveType[] vector = new TsPrimitiveType[values.size()]; - for (int columnIndex = 0; columnIndex < values.size(); columnIndex++) { + + int columns = columnIndexList == null ? values.size() : columnIndexList.size(); + TsPrimitiveType[] vector = new TsPrimitiveType[columns]; + for (int i = 0; i < columns; i++) { + int columnIndex = columnIndexList == null ? i : columnIndexList.get(i); + if (columnIndex < 0 || columnIndex >= values.size()) { + continue; + } List columnValues = values.get(columnIndex); int validValueIndex; if (validIndexesForTimeDuplicatedRows != null) { @@ -295,43 +307,43 @@ private TsPrimitiveType getAlignedValueByValueIndex( case BLOB: case STRING: Binary valueT = ((Binary[]) columnValues.get(arrayIndex))[elementIndex]; - vector[columnIndex] = TsPrimitiveType.getByType(TSDataType.TEXT, valueT); + vector[i] = TsPrimitiveType.getByType(TSDataType.TEXT, valueT); break; case FLOAT: float valueF = ((float[]) columnValues.get(arrayIndex))[elementIndex]; if (floatPrecision != null && encodingList != null && !Float.isNaN(valueF) - && (encodingList.get(columnIndex) == TSEncoding.RLE - || encodingList.get(columnIndex) == TSEncoding.TS_2DIFF)) { + && (encodingList.get(i) == TSEncoding.RLE + || encodingList.get(i) == TSEncoding.TS_2DIFF)) { valueF = MathUtils.roundWithGivenPrecision(valueF, floatPrecision); } - vector[columnIndex] = TsPrimitiveType.getByType(TSDataType.FLOAT, valueF); + vector[i] = TsPrimitiveType.getByType(TSDataType.FLOAT, valueF); break; case INT32: case DATE: int valueI = ((int[]) columnValues.get(arrayIndex))[elementIndex]; - vector[columnIndex] = TsPrimitiveType.getByType(TSDataType.INT32, valueI); + vector[i] = TsPrimitiveType.getByType(TSDataType.INT32, valueI); break; case INT64: case TIMESTAMP: long valueL = ((long[]) columnValues.get(arrayIndex))[elementIndex]; - vector[columnIndex] = TsPrimitiveType.getByType(TSDataType.INT64, valueL); + vector[i] = TsPrimitiveType.getByType(TSDataType.INT64, valueL); break; case DOUBLE: double valueD = ((double[]) columnValues.get(arrayIndex))[elementIndex]; if (floatPrecision != null && encodingList != null && !Double.isNaN(valueD) - && (encodingList.get(columnIndex) == TSEncoding.RLE - || encodingList.get(columnIndex) == TSEncoding.TS_2DIFF)) { + && (encodingList.get(i) == TSEncoding.RLE + || encodingList.get(i) == TSEncoding.TS_2DIFF)) { valueD = MathUtils.roundWithGivenPrecision(valueD, floatPrecision); } - vector[columnIndex] = TsPrimitiveType.getByType(TSDataType.DOUBLE, valueD); + vector[i] = TsPrimitiveType.getByType(TSDataType.DOUBLE, valueD); break; case BOOLEAN: boolean valueB = ((boolean[]) columnValues.get(arrayIndex))[elementIndex]; - vector[columnIndex] = TsPrimitiveType.getByType(TSDataType.BOOLEAN, valueB); + vector[i] = TsPrimitiveType.getByType(TSDataType.BOOLEAN, valueB); break; default: throw new UnsupportedOperationException(ERR_DATATYPE_NOT_CONSISTENT); @@ -549,6 +561,10 @@ public int deleteTime(long lowerBound, long upperBound) { if (needUpdateMaxTime) { updateMaxTime(); } + boolean needUpdateMinTime = lowerBound <= minTime && minTime <= upperBound; + if (needUpdateMinTime) { + updateMinTime(); + } return deletedNumber; } @@ -562,6 +578,16 @@ private void updateMaxTime() { this.maxTime = maxTime; } + private void updateMinTime() { + long minTime = Long.MAX_VALUE; + for (int i = 0; i < rowCount; i++) { + if (!isTimeDeleted(i)) { + minTime = Math.min(minTime, getTime(i)); + } + } + this.minTime = minTime; + } + /** * Delete points in a specific column. * @@ -727,9 +753,15 @@ public int getValidRowIndexForTimeDuplicatedRows( } protected TimeValuePair getTimeValuePair( - int index, long time, Integer floatPrecision, List encodingList) { + int index, + long time, + Integer floatPrecision, + List encodingList, + List columnIndexList) { return new TimeValuePair( - time, (TsPrimitiveType) getAlignedValueForQuery(index, floatPrecision, encodingList)); + time, + (TsPrimitiveType) + getAlignedValueForQuery(index, floatPrecision, encodingList, columnIndexList)); } @Override @@ -883,6 +915,11 @@ public TSDataType getDataType() { return TSDataType.VECTOR; } + @Override + public long calculateRamSize() { + return timestamps.size() * alignedTvListArrayMemCost(dataTypes); + } + /** * Get the single alignedTVList array mem cost by give types. * @@ -1448,4 +1485,185 @@ public BitMap getAllValueColDeletedMap() { public List> getBitMaps() { return bitMaps; } + + public AlignedTVListIterator iterator( + List columnIndexList, + boolean ignoreAllNullRows, + Integer floatPrecision, + List encodingList, + List timeColumnDeletion, + List> valueColumnsDeletionList) { + return new AlignedTVListIterator( + columnIndexList, + ignoreAllNullRows, + floatPrecision, + encodingList, + timeColumnDeletion, + valueColumnsDeletionList); + } + + /* AlignedTVList Iterator */ + public class AlignedTVListIterator extends TVListIterator { + private final BitMap allValueColDeletedMap; + private TimeValuePair currTvPair; + + private final Integer floatPrecision; + private final List encodingList; + + List columnIndexList; + + private final boolean ignoreAllNullRows; + private final List timeColumnDeletion; + private int[] timeDeleteCursor; + private final List> valueColumnsDeletionList; + private List valueColumnDeleteCursor; + + public AlignedTVListIterator( + List columnIndexList, + boolean ignoreAllNullRows, + Integer floatPrecision, + List encodingList, + List timeColumnDeletion, + List> valueColumnsDeletionList) { + super(null, null); + this.columnIndexList = columnIndexList; + this.ignoreAllNullRows = ignoreAllNullRows; + this.allValueColDeletedMap = ignoreAllNullRows ? getAllValueColDeletedMap() : null; + this.floatPrecision = floatPrecision; + this.encodingList = encodingList; + this.timeColumnDeletion = timeColumnDeletion; + this.valueColumnsDeletionList = valueColumnsDeletionList; + if (timeColumnDeletion != null) { + this.timeDeleteCursor = new int[] {0}; + } + if (valueColumnsDeletionList != null) { + this.valueColumnDeleteCursor = new ArrayList<>(); + valueColumnsDeletionList.forEach(x -> valueColumnDeleteCursor.add(new int[] {0})); + } + } + + private boolean isAllColumnNull(TimeValuePair tvPair) { + TsPrimitiveType[] primitiveValues = tvPair.getValue().getVector(); + for (TsPrimitiveType primitiveValue : primitiveValues) { + if (primitiveValue != null) { + return false; + } + } + return true; + } + + private void prepareNext() { + currTvPair = null; + // find the first row that is neither deleted nor empty (all NULL values) + boolean findValidRow = false; + while (index < rowCount && !findValidRow) { + // all columns values are deleted + if ((allValueColDeletedMap != null && allValueColDeletedMap.isMarked(getValueIndex(index))) + || (timeColumnDeletion != null + && isPointDeleted(currentTime, timeColumnDeletion, timeDeleteCursor))) { + index++; + currentTime = index < rowCount ? getTime(index) : Long.MIN_VALUE; + continue; + } + + // does not find any valid row + if (index >= rowCount) { + probeNext = true; + return; + } + + // check whether null column exits + currTvPair = + getTimeValuePair(index, currentTime, floatPrecision, encodingList, columnIndexList); + TsPrimitiveType[] primitiveValues = currTvPair.getValue().getVector(); + for (int columnIndex = 0; columnIndex < primitiveValues.length; columnIndex++) { + if (valueColumnsDeletionList != null + && isPointDeleted( + currentTime, + valueColumnsDeletionList.get(columnIndex), + valueColumnDeleteCursor.get(columnIndex))) { + primitiveValues[columnIndex] = null; + } + } + if (ignoreAllNullRows && isAllColumnNull(currTvPair)) { + currTvPair = null; + index++; + currentTime = index < rowCount ? getTime(index) : Long.MIN_VALUE; + } else { + findValidRow = true; + } + } + + // handle duplicated timestamp + while (index + 1 < rowCount && getTime(index + 1) == currentTime) { + index++; + // skip all-Null rows if allValueColDeletedMap exits + if (allValueColDeletedMap == null + || !allValueColDeletedMap.isMarked(getValueIndex(index))) { + TimeValuePair tvPair = + getTimeValuePair(index, currentTime, floatPrecision, encodingList, columnIndexList); + TsPrimitiveType[] primitiveValues = tvPair.getValue().getVector(); + for (int columnIndex = 0; columnIndex < primitiveValues.length; columnIndex++) { + // update currTvPair if the column is not null + if (primitiveValues[columnIndex] != null) { + currTvPair.getValue().getVector()[columnIndex] = primitiveValues[columnIndex]; + } + } + } + } + probeNext = true; + } + + @Override + public boolean hasNext() { + if (!probeNext) { + prepareNext(); + } + return index < rowCount && currTvPair != null; + } + + @Override + public boolean hasCurrent() { + return index < rowCount && currTvPair != null; + } + + @Override + public TimeValuePair next() { + if (!hasNext()) { + return null; + } + TimeValuePair ret = currTvPair; + index++; + currentTime = index < rowCount ? getTime(index) : Long.MIN_VALUE; + currTvPair = null; + probeNext = false; + return ret; + } + + @Override + public TimeValuePair current() { + if (!hasCurrent()) { + return null; + } + return currTvPair; + } + + // private TimeValuePair currentTimeValuePair() { + // if (currTvPair == null) { + // return null; + // } + // + // TsPrimitiveType[] currentVector = currTvPair.getValue().getVector(); + // if (currentVector.length == dataTypes.size()) { + // return currTvPair; + // } + // + // TsPrimitiveType[] vector = new TsPrimitiveType[dataTypes.size()]; + // for (int i = 0; i < currentVector.length; i++) { + // vector[i] = currentVector[i]; + // } + // TsPrimitiveType value = TsPrimitiveType.getByType(TSDataType.VECTOR, vector); + // return new TimeValuePair(currentTime, value); + // } + } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MergeSortAlignedTVListIterator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MergeSortAlignedTVListIterator.java new file mode 100644 index 0000000000000..d2c281db4e54e --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MergeSortAlignedTVListIterator.java @@ -0,0 +1,143 @@ +/* + * 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.tsfile.file.metadata.enums.TSEncoding; +import org.apache.tsfile.read.TimeValuePair; +import org.apache.tsfile.read.common.TimeRange; +import org.apache.tsfile.read.reader.IPointReader; +import org.apache.tsfile.utils.TsPrimitiveType; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +public class MergeSortAlignedTVListIterator implements IPointReader { + private final List alignedTvListIterators; + private boolean probeNext = false; + private TimeValuePair currentTvPair; + + private final int[] alignedTvListOffsets; + + public MergeSortAlignedTVListIterator( + List alignedTvLists, + List columnIndexList, + Integer floatPrecision, + List encodingList, + List timeColumnDeletion, + List> valueColumnsDeletionList, + boolean ignoreAllNullRows) { + this.alignedTvListIterators = new ArrayList<>(); + for (AlignedTVList alignedTvList : alignedTvLists) { + alignedTvListIterators.add( + alignedTvList.iterator( + columnIndexList, + ignoreAllNullRows, + floatPrecision, + encodingList, + timeColumnDeletion, + valueColumnsDeletionList)); + } + this.alignedTvListOffsets = new int[alignedTvLists.size()]; + } + + private void prepareNextRow() { + currentTvPair = null; + long time = Long.MAX_VALUE; + for (int i = 0; i < alignedTvListIterators.size(); i++) { + AlignedTVList.AlignedTVListIterator iterator = alignedTvListIterators.get(i); + if (iterator.hasNext() && iterator.currentTime() <= time) { + TimeValuePair tvPair = iterator.current(); + // check valueColumnsDeletionList + TsPrimitiveType[] primitiveValues = tvPair.getValue().getVector(); + if (currentTvPair == null || iterator.currentTime() < time) { + currentTvPair = tvPair; + } else { + for (int columnIndex = 0; columnIndex < primitiveValues.length; columnIndex++) { + // update currentTvPair if the column is not null + if (primitiveValues[columnIndex] != null) { + currentTvPair.getValue().getVector()[columnIndex] = primitiveValues[columnIndex]; + } + } + } + time = iterator.currentTime(); + } + } + probeNext = true; + } + + @Override + public boolean hasNextTimeValuePair() { + if (!probeNext) { + prepareNextRow(); + } + return currentTvPair != null; + } + + @Override + public TimeValuePair nextTimeValuePair() { + if (!hasNextTimeValuePair()) { + return null; + } + + for (int i = 0; i < alignedTvListIterators.size(); i++) { + AlignedTVList.AlignedTVListIterator iterator = alignedTvListIterators.get(i); + if (iterator.hasCurrent() && iterator.currentTime() == currentTvPair.getTimestamp()) { + alignedTvListIterators.get(i).step(); + alignedTvListOffsets[i] = alignedTvListIterators.get(i).getIndex(); + } + } + + TimeValuePair ret = currentTvPair; + probeNext = false; + return ret; + } + + @Override + public TimeValuePair currentTimeValuePair() { + if (!hasNextTimeValuePair()) { + return null; + } + return currentTvPair; + } + + @Override + public long getUsedMemorySize() { + // not used + return 0; + } + + @Override + public void close() throws IOException { + alignedTvListIterators.clear(); + } + + public int[] getAlignedTVListOffsets() { + return alignedTvListOffsets; + } + + public void setAlignedTVListOffsets(int[] alignedTvListOffsets) { + for (int i = 0; i < alignedTvListIterators.size(); i++) { + alignedTvListIterators.get(i).setIndex(alignedTvListOffsets[i]); + this.alignedTvListOffsets[i] = alignedTvListOffsets[i]; + } + probeNext = false; + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MergeSortTvListIterator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MergeSortTvListIterator.java index 24a3a07029064..81534cffeb4fb 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MergeSortTvListIterator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MergeSortTvListIterator.java @@ -19,9 +19,6 @@ package org.apache.iotdb.db.utils.datastructure; -import org.apache.iotdb.db.utils.MathUtils; - -import org.apache.tsfile.enums.TSDataType; import org.apache.tsfile.file.metadata.enums.TSEncoding; import org.apache.tsfile.read.TimeValuePair; import org.apache.tsfile.read.reader.IPointReader; @@ -32,29 +29,27 @@ public class MergeSortTvListIterator implements IPointReader { private final List tvListIterators; - private final TSDataType tsDataType; - private TSEncoding encoding; - private int floatPrecision = -1; private int selectedTVListIndex = -1; private TimeValuePair currentTvPair; private final int[] tvListOffsets; - public MergeSortTvListIterator(TSDataType tsDataType, List tvLists) { - this.tsDataType = tsDataType; + public MergeSortTvListIterator(List tvLists) { tvListIterators = new ArrayList<>(); for (TVList tvList : tvLists) { - tvListIterators.add(tvList.iterator()); + tvListIterators.add(tvList.iterator(null, null)); } this.tvListOffsets = new int[tvLists.size()]; } public MergeSortTvListIterator( - TSDataType tsDataType, TSEncoding encoding, int floatPrecision, List tvLists) { - this(tsDataType, tvLists); - this.encoding = encoding; - this.floatPrecision = floatPrecision; + List tvLists, Integer floatPrecision, TSEncoding encoding) { + tvListIterators = new ArrayList<>(); + for (TVList tvList : tvLists) { + tvListIterators.add(tvList.iterator(floatPrecision, encoding)); + } + this.tvListOffsets = new int[tvLists.size()]; } private void prepareNextRow() { @@ -100,29 +95,13 @@ public TimeValuePair nextTimeValuePair() { } selectedTVListIndex = -1; - return currentTimeValuePair(); + return currentTvPair; } @Override public TimeValuePair currentTimeValuePair() { - if (encoding != null && floatPrecision != -1) { - if (tsDataType == TSDataType.FLOAT) { - float value = currentTvPair.getValue().getFloat(); - if (!Float.isNaN(value) - && (encoding == TSEncoding.RLE || encoding == TSEncoding.TS_2DIFF)) { - currentTvPair - .getValue() - .setFloat(MathUtils.roundWithGivenPrecision(value, floatPrecision)); - } - } else if (tsDataType == TSDataType.DOUBLE) { - double value = currentTvPair.getValue().getDouble(); - if (!Double.isNaN(value) - && (encoding == TSEncoding.RLE || encoding == TSEncoding.TS_2DIFF)) { - currentTvPair - .getValue() - .setDouble(MathUtils.roundWithGivenPrecision(value, floatPrecision)); - } - } + if (!hasNextTimeValuePair()) { + return null; } return currentTvPair; } @@ -146,7 +125,7 @@ public void setTVListOffsets(int[] tvListOffsets) { for (int i = 0; i < tvListIterators.size(); i++) { tvListIterators.get(i).setIndex(tvListOffsets[i]); this.tvListOffsets[i] = tvListOffsets[i]; - selectedTVListIndex = -1; } + selectedTVListIndex = -1; } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java index 8f13b3a39722d..0ff5a3fe8a07f 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java @@ -603,33 +603,46 @@ public void unlockQueryList() { queryListLock.unlock(); } - public TVListIterator iterator() { - return new TVListIterator(); + public TVListIterator iterator(Integer floatPrecision, TSEncoding encoding) { + return new TVListIterator(floatPrecision, encoding); } /* TVList Iterator */ public class TVListIterator { - private int index; - private long currentTime; + protected int index; + protected long currentTime; + protected boolean probeNext; + private final Integer floatPrecision; + private final TSEncoding encoding; + + public TVListIterator(Integer floatPrecision, TSEncoding encoding) { + this.index = 0; + this.currentTime = index < rowCount ? getTime(index) : Long.MIN_VALUE; + this.floatPrecision = floatPrecision; + this.encoding = encoding; + } + + private void prepareNext() { + // skip deleted rows + int prevIndex = index; + while (index < rowCount && (bitMap != null && isNullValue(getValueIndex(index)))) { + index++; + } + // update current timestamp if needed + if (index > prevIndex) { + currentTime = index < rowCount ? getTime(index) : Long.MIN_VALUE; + } - public TVListIterator() { - index = 0; - currentTime = index < rowCount ? getTime(index) : Long.MIN_VALUE; + // skip duplicated timestamp + while (index + 1 < rowCount && getTime(index + 1) == currentTime) { + index++; + } + probeNext = true; } public boolean hasNext() { - if (bitMap != null) { - // skip deleted & duplicated timestamp - while ((index < rowCount && isNullValue(getValueIndex(index))) - || (index + 1 < rowCount && getTime(index + 1) == currentTime)) { - index++; - } - currentTime = index < rowCount ? getTime(index) : Long.MIN_VALUE; - } else { - // skip duplicated timestamp - while (index + 1 < rowCount && getTime(index + 1) == currentTime) { - index++; - } + if (!probeNext) { + prepareNext(); } return index < rowCount; } @@ -638,8 +651,9 @@ public TimeValuePair next() { if (!hasNext()) { return null; } - TimeValuePair ret = getTimeValuePair(index++); + TimeValuePair ret = getTimeValuePair(index++, currentTime, floatPrecision, encoding); currentTime = index < rowCount ? getTime(index) : Long.MIN_VALUE; + probeNext = false; return ret; } @@ -647,7 +661,7 @@ public TimeValuePair current() { if (!hasCurrent()) { return null; } - return getTimeValuePair(index); + return getTimeValuePair(index, currentTime, floatPrecision, encoding); } public boolean hasCurrent() { @@ -670,11 +684,13 @@ public int getIndex() { public void setIndex(int index) { this.index = index; - currentTime = index < rowCount ? getTime(index) : Long.MIN_VALUE; + this.probeNext = false; + this.currentTime = index < rowCount ? getTime(index) : Long.MIN_VALUE; } protected void step() { index++; + probeNext = false; currentTime = index < rowCount ? getTime(index) : Long.MIN_VALUE; } } diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/AlignedMemPageReaderTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/AlignedMemPageReaderTest.java index 2bc7166b71fa8..59e2fd2a2223b 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/AlignedMemPageReaderTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/AlignedMemPageReaderTest.java @@ -21,6 +21,7 @@ import org.apache.tsfile.enums.TSDataType; import org.apache.tsfile.file.metadata.AlignedChunkMetadata; +import org.apache.tsfile.file.metadata.IMetadata; import org.apache.tsfile.file.metadata.statistics.IntegerStatistics; import org.apache.tsfile.file.metadata.statistics.Statistics; import org.apache.tsfile.file.metadata.statistics.TimeStatistics; @@ -34,9 +35,13 @@ import org.junit.Test; import org.mockito.Mockito; +import java.io.Serializable; import java.util.Arrays; import java.util.Collections; +import java.util.List; import java.util.Optional; +import java.util.function.Supplier; +import java.util.stream.Collectors; import static org.apache.tsfile.read.filter.factory.ValueFilterApi.DEFAULT_MEASUREMENT_INDEX; @@ -94,15 +99,41 @@ public class AlignedMemPageReaderTest { } private MemAlignedPageReader generateAlignedPageReader() { + Supplier tsBlockSupplier = () -> tsBlock1; + List> valueStatistcsList = + chunkMetadata1.getValueChunkMetadataList().stream() + .map(IMetadata::getStatistics) + .collect(Collectors.toList()); MemAlignedPageReader alignedPageReader = - new MemAlignedPageReader(tsBlock1, chunkMetadata1, null); + new MemAlignedPageReader( + tsBlockSupplier, + null, + null, + null, + Arrays.asList(TSDataType.INT32, TSDataType.INT32), + chunkMetadata1.getTimeStatistics(), + valueStatistcsList, + null); alignedPageReader.initTsBlockBuilder(Arrays.asList(TSDataType.INT32, TSDataType.INT32)); return alignedPageReader; } private MemAlignedPageReader generateSingleColumnAlignedPageReader() { + Supplier tsBlockSupplier = () -> tsBlock2; + List> valueStatistcsList = + chunkMetadata2.getValueChunkMetadataList().stream() + .map(IMetadata::getStatistics) + .collect(Collectors.toList()); MemAlignedPageReader alignedPageReader = - new MemAlignedPageReader(tsBlock2, chunkMetadata2, null); + new MemAlignedPageReader( + tsBlockSupplier, + null, + null, + null, + Arrays.asList(TSDataType.INT32), + chunkMetadata2.getTimeStatistics(), + valueStatistcsList, + null); alignedPageReader.initTsBlockBuilder(Collections.singletonList(TSDataType.INT32)); return alignedPageReader; } diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemAlignedChunkLoaderTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemAlignedChunkLoaderTest.java index e94a75f55a701..1a23760554677 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemAlignedChunkLoaderTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemAlignedChunkLoaderTest.java @@ -21,24 +21,28 @@ import org.apache.iotdb.db.queryengine.execution.fragment.QueryContext; import org.apache.iotdb.db.storageengine.dataregion.memtable.AlignedReadOnlyMemChunk; +import org.apache.iotdb.db.utils.datastructure.AlignedTVList; import org.apache.tsfile.common.conf.TSFileConfig; import org.apache.tsfile.enums.TSDataType; import org.apache.tsfile.file.metadata.AlignedChunkMetadata; import org.apache.tsfile.file.metadata.ChunkMetadata; +import org.apache.tsfile.file.metadata.enums.TSEncoding; import org.apache.tsfile.file.metadata.statistics.Statistics; import org.apache.tsfile.read.common.BatchData; import org.apache.tsfile.read.common.block.TsBlock; -import org.apache.tsfile.read.common.block.TsBlockBuilder; import org.apache.tsfile.read.reader.IPageReader; import org.apache.tsfile.utils.Binary; import org.junit.Test; import org.mockito.Mockito; import java.io.IOException; +import java.io.Serializable; +import java.util.ArrayList; import java.util.Arrays; +import java.util.LinkedHashMap; import java.util.List; -import java.util.Optional; +import java.util.Map; import static org.apache.tsfile.read.reader.series.PaginationController.UNLIMITED_PAGINATION_CONTROLLER; import static org.junit.Assert.assertEquals; @@ -54,9 +58,9 @@ public class MemAlignedChunkLoaderTest { public void testMemAlignedChunkLoader() throws IOException { AlignedReadOnlyMemChunk chunk = Mockito.mock(AlignedReadOnlyMemChunk.class); ChunkMetadata chunkMetadata = Mockito.mock(ChunkMetadata.class); + QueryContext ctx = new QueryContext(); + MemAlignedChunkLoader memAlignedChunkLoader = new MemAlignedChunkLoader(ctx, chunk); - MemAlignedChunkLoader memAlignedChunkLoader = - new MemAlignedChunkLoader(new QueryContext(), chunk); try { memAlignedChunkLoader.loadChunk(chunkMetadata); fail(); @@ -64,29 +68,53 @@ public void testMemAlignedChunkLoader() throws IOException { assertNull(e.getMessage()); } - AlignedChunkMetadata chunkMetadata1 = Mockito.mock(AlignedChunkMetadata.class); + // Mock getTimeStatisticsList & getValuesStatisticsList + List> timeStatitsticsList = new ArrayList<>(); + Statistics timeStatistics = Mockito.mock(Statistics.class); + Mockito.when(timeStatistics.getCount()).thenReturn(2L); + timeStatitsticsList.add(timeStatistics); + Mockito.when(chunk.getTimeStatisticsList()).thenReturn(timeStatitsticsList); - Mockito.when(chunk.getTsBlock()).thenReturn(buildTsBlock()); - Mockito.when(chunk.getChunkMetaData()).thenReturn(chunkMetadata1); - Statistics statistics1 = Mockito.mock(Statistics.class); + List>> valuesStatitsticsList = new ArrayList<>(); + List> valuesStatistics = new ArrayList<>(); + Statistics statistics1 = Mockito.mock(Statistics.class); Mockito.when(statistics1.hasNullValue(2)).thenReturn(true); - Statistics statistics2 = Mockito.mock(Statistics.class); + valuesStatistics.add(statistics1); + Statistics statistics2 = Mockito.mock(Statistics.class); Mockito.when(statistics2.hasNullValue(2)).thenReturn(true); - Statistics statistics3 = Mockito.mock(Statistics.class); + valuesStatistics.add(statistics2); + Statistics statistics3 = Mockito.mock(Statistics.class); Mockito.when(statistics3.hasNullValue(2)).thenReturn(true); - Statistics statistics4 = Mockito.mock(Statistics.class); + valuesStatistics.add(statistics3); + Statistics statistics4 = Mockito.mock(Statistics.class); Mockito.when(statistics4.hasNullValue(2)).thenReturn(true); - Statistics statistics5 = Mockito.mock(Statistics.class); + valuesStatistics.add(statistics4); + Statistics statistics5 = Mockito.mock(Statistics.class); Mockito.when(statistics5.hasNullValue(2)).thenReturn(true); - Statistics statistics6 = Mockito.mock(Statistics.class); + valuesStatistics.add(statistics5); + Statistics statistics6 = Mockito.mock(Statistics.class); Mockito.when(statistics6.hasNullValue(2)).thenReturn(true); + valuesStatistics.add(statistics6); + valuesStatitsticsList.add(valuesStatistics); + Mockito.when(chunk.getValuesStatisticsList()).thenReturn(valuesStatitsticsList); - Statistics timeStatistics = Mockito.mock(Statistics.class); - Mockito.when(timeStatistics.getCount()).thenReturn(2L); + // Mock AlignedReadOnlyMemChunk Getter + List pageOffsets = Arrays.asList(new int[] {0, 0}, new int[] {2, 1}); + Mockito.when(chunk.getPageOffsetsList()).thenReturn(pageOffsets); - Mockito.when(chunkMetadata1.getStatistics()).thenReturn(timeStatistics); - Mockito.when(chunkMetadata1.getTimeStatistics()).thenReturn(timeStatistics); - Mockito.when(chunkMetadata1.getMeasurementStatistics(0)).thenReturn(Optional.of(statistics1)); + List encodingList = + Arrays.asList(new TSEncoding[] {null, null, null, null, null, null}); + + Mockito.when(chunk.getEncodingList()).thenReturn(encodingList); + Mockito.when(chunk.getDataTypes()).thenReturn(buildTsDataTypes()); + Mockito.when(chunk.getColumnIndexList()).thenReturn(null); + Mockito.when(chunk.getTimeColumnDeletion()).thenReturn(null); + Mockito.when(chunk.getValueColumnsDeletionList()).thenReturn(null); + Mockito.when(chunk.getAligendTvListQueryMap()).thenReturn(buildAlignedTvListMap()); + Mockito.when(chunk.getContext()).thenReturn(ctx); + + AlignedChunkMetadata chunkMetadata1 = Mockito.mock(AlignedChunkMetadata.class); + Mockito.when(chunk.getChunkMetaData()).thenReturn(chunkMetadata1); MemAlignedChunkReader chunkReader = (MemAlignedChunkReader) memAlignedChunkLoader.getChunkReader(chunkMetadata1, null); @@ -110,14 +138,7 @@ public void testMemAlignedChunkLoader() throws IOException { MemAlignedPageReader pageReader = (MemAlignedPageReader) pageReaderList.get(0); - pageReader.initTsBlockBuilder( - Arrays.asList( - TSDataType.BOOLEAN, - TSDataType.INT32, - TSDataType.INT64, - TSDataType.FLOAT, - TSDataType.DOUBLE, - TSDataType.TEXT)); + pageReader.initTsBlockBuilder(buildTsDataTypes()); BatchData batchData = pageReader.getAllSatisfiedPageData(); assertEquals(2, batchData.length()); @@ -145,32 +166,35 @@ public void testMemAlignedChunkLoader() throws IOException { memAlignedChunkLoader.close(); } - private TsBlock buildTsBlock() { - TsBlockBuilder builder = - new TsBlockBuilder( - Arrays.asList( - TSDataType.BOOLEAN, - TSDataType.INT32, - TSDataType.INT64, - TSDataType.FLOAT, - TSDataType.DOUBLE, - TSDataType.TEXT)); - builder.getTimeColumnBuilder().writeLong(1L); - builder.getColumnBuilder(0).writeBoolean(true); - builder.getColumnBuilder(1).writeInt(1); - builder.getColumnBuilder(2).writeLong(1L); - builder.getColumnBuilder(3).writeFloat(1.1f); - builder.getColumnBuilder(4).appendNull(); - builder.getColumnBuilder(5).writeBinary(new Binary(BINARY_STR, TSFileConfig.STRING_CHARSET)); - builder.declarePosition(); - builder.getTimeColumnBuilder().writeLong(2L); - builder.getColumnBuilder(0).appendNull(); - builder.getColumnBuilder(1).appendNull(); - builder.getColumnBuilder(2).appendNull(); - builder.getColumnBuilder(3).appendNull(); - builder.getColumnBuilder(4).writeDouble(3.14d); - builder.getColumnBuilder(5).appendNull(); - builder.declarePosition(); - return builder.build(); + private List buildTsDataTypes() { + return Arrays.asList( + TSDataType.BOOLEAN, + TSDataType.INT32, + TSDataType.INT64, + TSDataType.FLOAT, + TSDataType.DOUBLE, + TSDataType.TEXT); + } + + private Map buildAlignedTvListMap() { + List dataTypes = buildTsDataTypes(); + AlignedTVList tvList1 = AlignedTVList.newAlignedList(dataTypes); + tvList1.putAlignedValue( + 1L, + new Object[] { + true, 1, 2L, 1.2f, null, new Binary(BINARY_STR, TSFileConfig.STRING_CHARSET) + }); + tvList1.putAlignedValue(2L, new Object[] {null, null, null, null, 3.14d, null}); + AlignedTVList tvList2 = AlignedTVList.newAlignedList(dataTypes); + tvList2.putAlignedValue( + 1L, + new Object[] { + true, 1, 1L, 1.1f, null, new Binary(BINARY_STR, TSFileConfig.STRING_CHARSET) + }); + + Map tvListMap = new LinkedHashMap<>(); + tvListMap.put(tvList1, 2); + tvListMap.put(tvList2, 1); + return tvListMap; } } From 6eedcf655b69607aee32156c83cc692d6bb20f76 Mon Sep 17 00:00:00 2001 From: shizy Date: Wed, 18 Dec 2024 15:27:58 +0800 Subject: [PATCH 19/54] fix: MemAlignedChunkReader page offset --- .../dataregion/read/reader/chunk/MemAlignedChunkReader.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemAlignedChunkReader.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemAlignedChunkReader.java index 0c23bf0751411..ce163741990d2 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemAlignedChunkReader.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemAlignedChunkReader.java @@ -82,7 +82,7 @@ private void initAllPageReaders( tsBlockSupplier, timeValuePairIterator, pageOffsetsList.get(i), - i + 1 < timeStatistics.size() ? pageOffsetsList.get(i + 1) : null, + pageOffsetsList.get(i + 1), tsDataTypes, timeStatistics.get(i), valuesStatistics.get(i), From 0a9c4ed0d58d2e263258bf256f95bb043fc00737 Mon Sep 17 00:00:00 2001 From: shizy Date: Tue, 17 Dec 2024 13:56:43 +0800 Subject: [PATCH 20/54] performance issue: * change some list to array * remember row count in tvlist iterator --- .../memtable/AlignedReadOnlyMemChunk.java | 134 ++++++++---------- .../reader/chunk/MemAlignedChunkReader.java | 2 +- .../reader/chunk/MemAlignedPageReader.java | 39 +++-- .../chunk/metadata/AlignedPageMetadata.java | 17 ++- .../db/utils/datastructure/AlignedTVList.java | 34 ++--- .../MergeSortAlignedTVListIterator.java | 47 +++--- .../MergeSortTvListIterator.java | 41 +++--- .../iotdb/db/utils/datastructure/TVList.java | 22 +-- .../chunk/AlignedMemPageReaderTest.java | 11 +- .../chunk/MemAlignedChunkLoaderTest.java | 16 +-- 10 files changed, 162 insertions(+), 201 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedReadOnlyMemChunk.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedReadOnlyMemChunk.java index 944bbc5fd6b7f..56c925b3d7b58 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedReadOnlyMemChunk.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedReadOnlyMemChunk.java @@ -65,7 +65,7 @@ public class AlignedReadOnlyMemChunk extends ReadOnlyMemChunk { // time & values statistics private final List> timeStatisticsList; - private final List>> valueStatisticsList; + private final List[]> valueStatisticsList; // AlignedTVList rowCount during query protected Map alignedTvListQueryMap; @@ -123,17 +123,9 @@ public void initChunkMetaFromTvLists() { Statistics.getStatsByType(TSDataType.VECTOR); IChunkMetadata chunkTimeMetadata = new ChunkMetadata(timeChunkName, TSDataType.VECTOR, null, null, 0, chunkTimeStatistics); - List> chunkValueStatistics = new ArrayList<>(); - List chunkValueMetadataList = new ArrayList<>(); - for (int column = 0; column < valueChunkNames.size(); column++) { - Statistics valueStatistics = - Statistics.getStatsByType(dataTypes.get(column)); - chunkValueStatistics.add(valueStatistics); - IChunkMetadata valueChunkMetadata = - new ChunkMetadata( - valueChunkNames.get(column), dataTypes.get(column), null, null, 0, valueStatistics); - chunkValueMetadataList.add(valueChunkMetadata); - } + Statistics[] chunkValueStatistics = + new Statistics[valueChunkNames.size()]; + Arrays.fill(chunkValueStatistics, null); int cnt = 0; List alignedTvLists = new ArrayList<>(alignedTvListQueryMap.keySet()); @@ -147,6 +139,8 @@ public void initChunkMetaFromTvLists() { valueColumnsDeletionList, context.isIgnoreAllNullRows()); int[] alignedTvListOffsets = timeValuePairIterator.getAlignedTVListOffsets(); + + Statistics pageTimeStats = null; while (timeValuePairIterator.hasNextTimeValuePair()) { // Split pages if (cnt % MAX_NUMBER_OF_POINTS_IN_PAGE == 0) { @@ -154,82 +148,78 @@ public void initChunkMetaFromTvLists() { Statistics.getStatsByType(TSDataType.VECTOR); pageTimeStatistics.setEmpty(false); timeStatisticsList.add(pageTimeStatistics); - List> pageValueStatistics = new ArrayList<>(); - for (int column = 0; column < valueChunkNames.size(); column++) { - Statistics valueStatistics = - Statistics.getStatsByType(dataTypes.get(column)); - pageValueStatistics.add(valueStatistics); - } + pageTimeStats = timeStatisticsList.get(timeStatisticsList.size() - 1); + + Statistics[] pageValueStatistics = + new Statistics[valueChunkNames.size()]; + Arrays.fill(pageValueStatistics, null); valueStatisticsList.add(pageValueStatistics); pageOffsetsList.add(Arrays.copyOf(alignedTvListOffsets, alignedTvListOffsets.length)); } // Update Page & Chunk Statistics TimeValuePair tvPair = timeValuePairIterator.nextTimeValuePair(); - Statistics pageTimeStats = - timeStatisticsList.get(timeStatisticsList.size() - 1); pageTimeStats.update(tvPair.getTimestamp()); chunkTimeStatistics.update(tvPair.getTimestamp()); - List> pageValuesStats = + Statistics[] pageValuesStats = valueStatisticsList.get(valueStatisticsList.size() - 1); TsPrimitiveType[] primitiveValues = tvPair.getValue().getVector(); for (int column = 0; column < primitiveValues.length; column++) { if (primitiveValues[column] == null) { continue; } + + if (pageValuesStats[column] == null) { + Statistics valueStatistics = + Statistics.getStatsByType(dataTypes.get(column)); + pageValuesStats[column] = valueStatistics; + } + if (chunkValueStatistics[column] == null) { + Statistics chunkValueStats = + Statistics.getStatsByType(dataTypes.get(column)); + chunkValueStatistics[column] = chunkValueStats; + } + switch (dataTypes.get(column)) { case BOOLEAN: - pageValuesStats - .get(column) - .update(tvPair.getTimestamp(), primitiveValues[column].getBoolean()); - chunkValueStatistics - .get(column) - .update(tvPair.getTimestamp(), primitiveValues[column].getBoolean()); + pageValuesStats[column].update( + tvPair.getTimestamp(), primitiveValues[column].getBoolean()); + chunkValueStatistics[column].update( + tvPair.getTimestamp(), primitiveValues[column].getBoolean()); break; case INT32: case DATE: - pageValuesStats - .get(column) - .update(tvPair.getTimestamp(), primitiveValues[column].getInt()); - chunkValueStatistics - .get(column) - .update(tvPair.getTimestamp(), primitiveValues[column].getInt()); + pageValuesStats[column].update(tvPair.getTimestamp(), primitiveValues[column].getInt()); + chunkValueStatistics[column].update( + tvPair.getTimestamp(), primitiveValues[column].getInt()); break; case INT64: case TIMESTAMP: - pageValuesStats - .get(column) - .update(tvPair.getTimestamp(), primitiveValues[column].getLong()); - chunkValueStatistics - .get(column) - .update(tvPair.getTimestamp(), primitiveValues[column].getLong()); + pageValuesStats[column].update( + tvPair.getTimestamp(), primitiveValues[column].getLong()); + chunkValueStatistics[column].update( + tvPair.getTimestamp(), primitiveValues[column].getLong()); break; case FLOAT: - pageValuesStats - .get(column) - .update(tvPair.getTimestamp(), primitiveValues[column].getFloat()); - chunkValueStatistics - .get(column) - .update(tvPair.getTimestamp(), primitiveValues[column].getFloat()); + pageValuesStats[column].update( + tvPair.getTimestamp(), primitiveValues[column].getFloat()); + chunkValueStatistics[column].update( + tvPair.getTimestamp(), primitiveValues[column].getFloat()); break; case DOUBLE: - pageValuesStats - .get(column) - .update(tvPair.getTimestamp(), primitiveValues[column].getDouble()); - chunkValueStatistics - .get(column) - .update(tvPair.getTimestamp(), primitiveValues[column].getDouble()); + pageValuesStats[column].update( + tvPair.getTimestamp(), primitiveValues[column].getDouble()); + chunkValueStatistics[column].update( + tvPair.getTimestamp(), primitiveValues[column].getDouble()); break; case TEXT: case BLOB: case STRING: - pageValuesStats - .get(column) - .update(tvPair.getTimestamp(), primitiveValues[column].getBinary()); - chunkValueStatistics - .get(column) - .update(tvPair.getTimestamp(), primitiveValues[column].getBinary()); + pageValuesStats[column].update( + tvPair.getTimestamp(), primitiveValues[column].getBinary()); + chunkValueStatistics[column].update( + tvPair.getTimestamp(), primitiveValues[column].getBinary()); break; default: throw new UnSupportedDataTypeException( @@ -241,21 +231,23 @@ public void initChunkMetaFromTvLists() { pageOffsetsList.add(Arrays.copyOf(alignedTvListOffsets, alignedTvListOffsets.length)); chunkTimeStatistics.setEmpty(cnt == 0); - // statistics should be set null if there is no data - for (int column = 0; column < chunkValueMetadataList.size(); column++) { - if (chunkValueMetadataList.get(column).getStatistics().isEmpty()) { - chunkValueMetadataList.set(column, null); - } - } - for (List> pageValueStats : valueStatisticsList) { - for (int column = 0; column < pageValueStats.size(); column++) { - if (pageValueStats.get(column).isEmpty()) { - pageValueStats.set(column, null); - } + // aligned chunk meta + List chunkValueMetadataList = new ArrayList<>(); + for (int column = 0; column < valueChunkNames.size(); column++) { + if (chunkValueStatistics[column] != null) { + IChunkMetadata valueChunkMetadata = + new ChunkMetadata( + valueChunkNames.get(column), + dataTypes.get(column), + null, + null, + 0, + chunkValueStatistics[column]); + chunkValueMetadataList.add(valueChunkMetadata); + } else { + chunkValueMetadataList.add(null); } } - - // aligned chunk meta IChunkMetadata alignedChunkMetadata = new AlignedChunkMetadata(chunkTimeMetadata, chunkValueMetadataList); alignedChunkMetadata.setChunkLoader(new MemAlignedChunkLoader(context, this)); @@ -386,7 +378,7 @@ public List> getTimeStatisticsList() { return timeStatisticsList; } - public List>> getValuesStatisticsList() { + public List[]> getValuesStatisticsList() { return valueStatisticsList; } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemAlignedChunkReader.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemAlignedChunkReader.java index ce163741990d2..e5a8bd87cd3c6 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemAlignedChunkReader.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemAlignedChunkReader.java @@ -73,7 +73,7 @@ public MemAlignedChunkReader(AlignedReadOnlyMemChunk readableChunk, Filter globa private void initAllPageReaders( List tsDataTypes, List> timeStatistics, - List>> valuesStatistics, + List[]> valuesStatistics, List pageOffsetsList) { Supplier tsBlockSupplier = new MemAlignedChunkReader.TsBlockSupplier(); for (int i = 0; i < timeStatistics.size(); i++) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemAlignedPageReader.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemAlignedPageReader.java index 973652d19e5d4..c0d388bed9f26 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemAlignedPageReader.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemAlignedPageReader.java @@ -39,7 +39,6 @@ import java.io.IOException; import java.io.Serializable; -import java.util.ArrayList; import java.util.Arrays; import java.util.List; import java.util.Optional; @@ -70,7 +69,7 @@ public MemAlignedPageReader( int[] pageEndOffSets, List tsDataTypes, Statistics timeStatistics, - List> valueStatistics, + Statistics[] valueStatistics, Filter recordFilter) { this.tsBlockSupplier = tsBlockSupplier; this.mergeSortAlignedTVListIterator = mergeSortAlignedTVListIterator; @@ -245,11 +244,11 @@ private void initializeOffsets() { private void initPageStatistics() { Statistics pageTimeStatistics = Statistics.getStatsByType(TSDataType.VECTOR); - List> pageValueStatistics = new ArrayList<>(); + Statistics[] pageValueStatistics = new Statistics[tsDataTypes.size()]; for (int column = 0; column < tsDataTypes.size(); column++) { Statistics valueStatistics = Statistics.getStatsByType(tsDataTypes.get(column)); - pageValueStatistics.add(valueStatistics); + pageValueStatistics[column] = valueStatistics; } updatePageStatisticsFromTsBlock(pageTimeStatistics, pageValueStatistics); pageMetadata.setStatistics(pageTimeStatistics, pageValueStatistics); @@ -257,7 +256,7 @@ private void initPageStatistics() { private void updatePageStatisticsFromTsBlock( Statistics timeStatistics, - List> valueStatistics) { + Statistics[] valueStatistics) { if (!tsBlock.isEmpty()) { // update time statistics for (int i = 0; i < tsBlock.getPositionCount(); i++) { @@ -269,48 +268,42 @@ private void updatePageStatisticsFromTsBlock( switch (tsDataTypes.get(column)) { case BOOLEAN: for (int i = 0; i < tsBlock.getPositionCount(); i++) { - valueStatistics - .get(column) - .update(tsBlock.getTimeByIndex(i), tsBlock.getColumn(column).getBoolean(i)); + valueStatistics[column].update( + tsBlock.getTimeByIndex(i), tsBlock.getColumn(column).getBoolean(i)); } break; case INT32: case DATE: for (int i = 0; i < tsBlock.getPositionCount(); i++) { - valueStatistics - .get(column) - .update(tsBlock.getTimeByIndex(i), tsBlock.getColumn(column).getInt(i)); + valueStatistics[column].update( + tsBlock.getTimeByIndex(i), tsBlock.getColumn(column).getInt(i)); } break; case INT64: case TIMESTAMP: for (int i = 0; i < tsBlock.getPositionCount(); i++) { - valueStatistics - .get(column) - .update(tsBlock.getTimeByIndex(i), tsBlock.getColumn(column).getLong(i)); + valueStatistics[column].update( + tsBlock.getTimeByIndex(i), tsBlock.getColumn(column).getLong(i)); } break; case FLOAT: for (int i = 0; i < tsBlock.getPositionCount(); i++) { - valueStatistics - .get(column) - .update(tsBlock.getTimeByIndex(i), tsBlock.getColumn(column).getFloat(i)); + valueStatistics[column].update( + tsBlock.getTimeByIndex(i), tsBlock.getColumn(column).getFloat(i)); } break; case DOUBLE: for (int i = 0; i < tsBlock.getPositionCount(); i++) { - valueStatistics - .get(column) - .update(tsBlock.getTimeByIndex(i), tsBlock.getColumn(column).getDouble(i)); + valueStatistics[column].update( + tsBlock.getTimeByIndex(i), tsBlock.getColumn(column).getDouble(i)); } break; case TEXT: case BLOB: case STRING: for (int i = 0; i < tsBlock.getPositionCount(); i++) { - valueStatistics - .get(column) - .update(tsBlock.getTimeByIndex(i), tsBlock.getColumn(column).getBinary(i)); + valueStatistics[column].update( + tsBlock.getTimeByIndex(i), tsBlock.getColumn(column).getBinary(i)); } break; default: diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/metadata/AlignedPageMetadata.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/metadata/AlignedPageMetadata.java index 5ab43857522df..c06ec112be038 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/metadata/AlignedPageMetadata.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/metadata/AlignedPageMetadata.java @@ -23,24 +23,23 @@ import org.apache.tsfile.file.metadata.statistics.Statistics; import java.io.Serializable; -import java.util.List; import java.util.Optional; public class AlignedPageMetadata implements IMetadata { private Statistics timeStatistics; - private List> valueStatistics; + private Statistics[] valueStatistics; public AlignedPageMetadata( Statistics timeStatistics, - List> valuesStatistics) { + Statistics[] valuesStatistics) { this.timeStatistics = timeStatistics; this.valueStatistics = valuesStatistics; } @Override public Statistics getStatistics() { - return valueStatistics.size() == 1 && valueStatistics.get(0) != null - ? valueStatistics.get(0) + return valueStatistics.length == 1 && valueStatistics[0] != null + ? valueStatistics[0] : timeStatistics; } @@ -53,22 +52,22 @@ public Statistics getTimeStatistics() { public Optional> getMeasurementStatistics( int measurementIndex) { return Optional.ofNullable( - measurementIndex >= valueStatistics.size() ? null : valueStatistics.get(measurementIndex)); + measurementIndex >= valueStatistics.length ? null : valueStatistics[measurementIndex]); } @Override public boolean hasNullValue(int measurementIndex) { - if (measurementIndex >= valueStatistics.size()) { + if (measurementIndex >= valueStatistics.length) { return false; } long rowCount = getTimeStatistics().getCount(); - Statistics stats = valueStatistics.get(measurementIndex); + Statistics stats = valueStatistics[measurementIndex]; return stats != null && stats.hasNullValue(rowCount); } public void setStatistics( Statistics timeStatistics, - List> valueStatistics) { + Statistics[] valueStatistics) { this.timeStatistics = timeStatistics; this.valueStatistics = valueStatistics; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/AlignedTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/AlignedTVList.java index 594ed3afce729..e506c62d1f9c7 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/AlignedTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/AlignedTVList.java @@ -1556,18 +1556,18 @@ private void prepareNext() { currTvPair = null; // find the first row that is neither deleted nor empty (all NULL values) boolean findValidRow = false; - while (index < rowCount && !findValidRow) { + while (index < rows && !findValidRow) { // all columns values are deleted if ((allValueColDeletedMap != null && allValueColDeletedMap.isMarked(getValueIndex(index))) || (timeColumnDeletion != null && isPointDeleted(currentTime, timeColumnDeletion, timeDeleteCursor))) { index++; - currentTime = index < rowCount ? getTime(index) : Long.MIN_VALUE; + currentTime = index < rows ? getTime(index) : Long.MIN_VALUE; continue; } // does not find any valid row - if (index >= rowCount) { + if (index >= rows) { probeNext = true; return; } @@ -1588,14 +1588,14 @@ && isPointDeleted( if (ignoreAllNullRows && isAllColumnNull(currTvPair)) { currTvPair = null; index++; - currentTime = index < rowCount ? getTime(index) : Long.MIN_VALUE; + currentTime = index < rows ? getTime(index) : Long.MIN_VALUE; } else { findValidRow = true; } } // handle duplicated timestamp - while (index + 1 < rowCount && getTime(index + 1) == currentTime) { + while (index + 1 < rows && getTime(index + 1) == currentTime) { index++; // skip all-Null rows if allValueColDeletedMap exits if (allValueColDeletedMap == null @@ -1619,12 +1619,12 @@ public boolean hasNext() { if (!probeNext) { prepareNext(); } - return index < rowCount && currTvPair != null; + return index < rows; } @Override public boolean hasCurrent() { - return index < rowCount && currTvPair != null; + return index < rows; } @Override @@ -1634,7 +1634,7 @@ public TimeValuePair next() { } TimeValuePair ret = currTvPair; index++; - currentTime = index < rowCount ? getTime(index) : Long.MIN_VALUE; + currentTime = index < rows ? getTime(index) : Long.MIN_VALUE; currTvPair = null; probeNext = false; return ret; @@ -1647,23 +1647,5 @@ public TimeValuePair current() { } return currTvPair; } - - // private TimeValuePair currentTimeValuePair() { - // if (currTvPair == null) { - // return null; - // } - // - // TsPrimitiveType[] currentVector = currTvPair.getValue().getVector(); - // if (currentVector.length == dataTypes.size()) { - // return currTvPair; - // } - // - // TsPrimitiveType[] vector = new TsPrimitiveType[dataTypes.size()]; - // for (int i = 0; i < currentVector.length; i++) { - // vector[i] = currentVector[i]; - // } - // TsPrimitiveType value = TsPrimitiveType.getByType(TSDataType.VECTOR, vector); - // return new TimeValuePair(currentTime, value); - // } } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MergeSortAlignedTVListIterator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MergeSortAlignedTVListIterator.java index d2c281db4e54e..2243d19c81691 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MergeSortAlignedTVListIterator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MergeSortAlignedTVListIterator.java @@ -26,11 +26,10 @@ import org.apache.tsfile.utils.TsPrimitiveType; import java.io.IOException; -import java.util.ArrayList; import java.util.List; public class MergeSortAlignedTVListIterator implements IPointReader { - private final List alignedTvListIterators; + private final AlignedTVList.AlignedTVListIterator[] alignedTvListIterators; private boolean probeNext = false; private TimeValuePair currentTvPair; @@ -44,16 +43,18 @@ public MergeSortAlignedTVListIterator( List timeColumnDeletion, List> valueColumnsDeletionList, boolean ignoreAllNullRows) { - this.alignedTvListIterators = new ArrayList<>(); - for (AlignedTVList alignedTvList : alignedTvLists) { - alignedTvListIterators.add( - alignedTvList.iterator( - columnIndexList, - ignoreAllNullRows, - floatPrecision, - encodingList, - timeColumnDeletion, - valueColumnsDeletionList)); + this.alignedTvListIterators = new AlignedTVList.AlignedTVListIterator[alignedTvLists.size()]; + for (int i = 0; i < alignedTvLists.size(); i++) { + alignedTvListIterators[i] = + alignedTvLists + .get(i) + .iterator( + columnIndexList, + ignoreAllNullRows, + floatPrecision, + encodingList, + timeColumnDeletion, + valueColumnsDeletionList); } this.alignedTvListOffsets = new int[alignedTvLists.size()]; } @@ -61,15 +62,15 @@ public MergeSortAlignedTVListIterator( private void prepareNextRow() { currentTvPair = null; long time = Long.MAX_VALUE; - for (int i = 0; i < alignedTvListIterators.size(); i++) { - AlignedTVList.AlignedTVListIterator iterator = alignedTvListIterators.get(i); + for (int i = 0; i < alignedTvListIterators.length; i++) { + AlignedTVList.AlignedTVListIterator iterator = alignedTvListIterators[i]; if (iterator.hasNext() && iterator.currentTime() <= time) { TimeValuePair tvPair = iterator.current(); // check valueColumnsDeletionList - TsPrimitiveType[] primitiveValues = tvPair.getValue().getVector(); if (currentTvPair == null || iterator.currentTime() < time) { currentTvPair = tvPair; } else { + TsPrimitiveType[] primitiveValues = tvPair.getValue().getVector(); for (int columnIndex = 0; columnIndex < primitiveValues.length; columnIndex++) { // update currentTvPair if the column is not null if (primitiveValues[columnIndex] != null) { @@ -97,11 +98,11 @@ public TimeValuePair nextTimeValuePair() { return null; } - for (int i = 0; i < alignedTvListIterators.size(); i++) { - AlignedTVList.AlignedTVListIterator iterator = alignedTvListIterators.get(i); + for (int i = 0; i < alignedTvListIterators.length; i++) { + AlignedTVList.AlignedTVListIterator iterator = alignedTvListIterators[i]; if (iterator.hasCurrent() && iterator.currentTime() == currentTvPair.getTimestamp()) { - alignedTvListIterators.get(i).step(); - alignedTvListOffsets[i] = alignedTvListIterators.get(i).getIndex(); + alignedTvListIterators[i].step(); + alignedTvListOffsets[i] = alignedTvListIterators[i].getIndex(); } } @@ -125,17 +126,15 @@ public long getUsedMemorySize() { } @Override - public void close() throws IOException { - alignedTvListIterators.clear(); - } + public void close() throws IOException {} public int[] getAlignedTVListOffsets() { return alignedTvListOffsets; } public void setAlignedTVListOffsets(int[] alignedTvListOffsets) { - for (int i = 0; i < alignedTvListIterators.size(); i++) { - alignedTvListIterators.get(i).setIndex(alignedTvListOffsets[i]); + for (int i = 0; i < alignedTvListIterators.length; i++) { + alignedTvListIterators[i].setIndex(alignedTvListOffsets[i]); this.alignedTvListOffsets[i] = alignedTvListOffsets[i]; } probeNext = false; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MergeSortTvListIterator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MergeSortTvListIterator.java index 81534cffeb4fb..5e928f30d114a 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MergeSortTvListIterator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MergeSortTvListIterator.java @@ -24,11 +24,10 @@ import org.apache.tsfile.read.reader.IPointReader; import java.io.IOException; -import java.util.ArrayList; import java.util.List; public class MergeSortTvListIterator implements IPointReader { - private final List tvListIterators; + private final TVList.TVListIterator[] tvListIterators; private int selectedTVListIndex = -1; private TimeValuePair currentTvPair; @@ -36,18 +35,18 @@ public class MergeSortTvListIterator implements IPointReader { private final int[] tvListOffsets; public MergeSortTvListIterator(List tvLists) { - tvListIterators = new ArrayList<>(); - for (TVList tvList : tvLists) { - tvListIterators.add(tvList.iterator(null, null)); + tvListIterators = new TVList.TVListIterator[tvLists.size()]; + for (int i = 0; i < tvLists.size(); i++) { + tvListIterators[i] = tvLists.get(i).iterator(null, null); } this.tvListOffsets = new int[tvLists.size()]; } public MergeSortTvListIterator( List tvLists, Integer floatPrecision, TSEncoding encoding) { - tvListIterators = new ArrayList<>(); - for (TVList tvList : tvLists) { - tvListIterators.add(tvList.iterator(floatPrecision, encoding)); + tvListIterators = new TVList.TVListIterator[tvLists.size()]; + for (int i = 0; i < tvLists.size(); i++) { + tvListIterators[i] = tvLists.get(i).iterator(floatPrecision, encoding); } this.tvListOffsets = new int[tvLists.size()]; } @@ -55,8 +54,8 @@ public MergeSortTvListIterator( private void prepareNextRow() { long time = Long.MAX_VALUE; selectedTVListIndex = -1; - for (int i = 0; i < tvListIterators.size(); i++) { - TVList.TVListIterator iterator = tvListIterators.get(i); + for (int i = 0; i < tvListIterators.length; i++) { + TVList.TVListIterator iterator = tvListIterators[i]; boolean hasNext = iterator.hasNext(); // update minimum time and remember selected TVList if (hasNext && iterator.currentTime() <= time) { @@ -71,7 +70,7 @@ public boolean hasNextTimeValuePair() { if (selectedTVListIndex == -1) { prepareNextRow(); } - return selectedTVListIndex >= 0 && selectedTVListIndex < tvListIterators.size(); + return selectedTVListIndex >= 0 && selectedTVListIndex < tvListIterators.length; } @Override @@ -79,18 +78,18 @@ public TimeValuePair nextTimeValuePair() { if (!hasNextTimeValuePair()) { return null; } - currentTvPair = tvListIterators.get(selectedTVListIndex).next(); - tvListOffsets[selectedTVListIndex] = tvListIterators.get(selectedTVListIndex).getIndex(); + currentTvPair = tvListIterators[selectedTVListIndex].next(); + tvListOffsets[selectedTVListIndex] = tvListIterators[selectedTVListIndex].getIndex(); // call next to skip identical timestamp in other iterators - for (int i = 0; i < tvListIterators.size(); i++) { + for (int i = 0; i < tvListIterators.length; i++) { if (selectedTVListIndex == i) { continue; } - TVList.TVListIterator iterator = tvListIterators.get(i); + TVList.TVListIterator iterator = tvListIterators[i]; if (iterator.hasCurrent() && iterator.currentTime() == currentTvPair.getTimestamp()) { - tvListIterators.get(i).step(); - tvListOffsets[i] = tvListIterators.get(i).getIndex(); + tvListIterators[i].step(); + tvListOffsets[i] = tvListIterators[i].getIndex(); } } @@ -113,17 +112,15 @@ public long getUsedMemorySize() { } @Override - public void close() throws IOException { - tvListIterators.clear(); - } + public void close() throws IOException {} public int[] getTVListOffsets() { return tvListOffsets; } public void setTVListOffsets(int[] tvListOffsets) { - for (int i = 0; i < tvListIterators.size(); i++) { - tvListIterators.get(i).setIndex(tvListOffsets[i]); + for (int i = 0; i < tvListIterators.length; i++) { + tvListIterators[i].setIndex(tvListOffsets[i]); this.tvListOffsets[i] = tvListOffsets[i]; } selectedTVListIndex = -1; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java index 0ff5a3fe8a07f..9ec092ac3566e 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java @@ -610,6 +610,7 @@ public TVListIterator iterator(Integer floatPrecision, TSEncoding encoding) { /* TVList Iterator */ public class TVListIterator { protected int index; + protected int rows; protected long currentTime; protected boolean probeNext; private final Integer floatPrecision; @@ -617,7 +618,8 @@ public class TVListIterator { public TVListIterator(Integer floatPrecision, TSEncoding encoding) { this.index = 0; - this.currentTime = index < rowCount ? getTime(index) : Long.MIN_VALUE; + this.rows = rowCount; + this.currentTime = index < rows ? getTime(index) : Long.MIN_VALUE; this.floatPrecision = floatPrecision; this.encoding = encoding; } @@ -625,16 +627,16 @@ public TVListIterator(Integer floatPrecision, TSEncoding encoding) { private void prepareNext() { // skip deleted rows int prevIndex = index; - while (index < rowCount && (bitMap != null && isNullValue(getValueIndex(index)))) { + while (index < rows && (bitMap != null && isNullValue(getValueIndex(index)))) { index++; } // update current timestamp if needed if (index > prevIndex) { - currentTime = index < rowCount ? getTime(index) : Long.MIN_VALUE; + currentTime = index < rows ? getTime(index) : Long.MIN_VALUE; } // skip duplicated timestamp - while (index + 1 < rowCount && getTime(index + 1) == currentTime) { + while (index + 1 < rows && getTime(index + 1) == currentTime) { index++; } probeNext = true; @@ -644,7 +646,7 @@ public boolean hasNext() { if (!probeNext) { prepareNext(); } - return index < rowCount; + return index < rows; } public TimeValuePair next() { @@ -652,7 +654,7 @@ public TimeValuePair next() { return null; } TimeValuePair ret = getTimeValuePair(index++, currentTime, floatPrecision, encoding); - currentTime = index < rowCount ? getTime(index) : Long.MIN_VALUE; + currentTime = index < rows ? getTime(index) : Long.MIN_VALUE; probeNext = false; return ret; } @@ -666,9 +668,9 @@ public TimeValuePair current() { public boolean hasCurrent() { if (bitMap == null) { - return index < rowCount; + return index < rows; } - return index < rowCount && !isNullValue(getValueIndex(index)); + return index < rows && !isNullValue(getValueIndex(index)); } public long currentTime() { @@ -685,13 +687,13 @@ public int getIndex() { public void setIndex(int index) { this.index = index; this.probeNext = false; - this.currentTime = index < rowCount ? getTime(index) : Long.MIN_VALUE; + this.currentTime = index < rows ? getTime(index) : Long.MIN_VALUE; } protected void step() { index++; probeNext = false; - currentTime = index < rowCount ? getTime(index) : Long.MIN_VALUE; + currentTime = index < rows ? getTime(index) : Long.MIN_VALUE; } } } diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/AlignedMemPageReaderTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/AlignedMemPageReaderTest.java index 59e2fd2a2223b..94fa6a1a09001 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/AlignedMemPageReaderTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/AlignedMemPageReaderTest.java @@ -35,13 +35,10 @@ import org.junit.Test; import org.mockito.Mockito; -import java.io.Serializable; import java.util.Arrays; import java.util.Collections; -import java.util.List; import java.util.Optional; import java.util.function.Supplier; -import java.util.stream.Collectors; import static org.apache.tsfile.read.filter.factory.ValueFilterApi.DEFAULT_MEASUREMENT_INDEX; @@ -100,10 +97,10 @@ public class AlignedMemPageReaderTest { private MemAlignedPageReader generateAlignedPageReader() { Supplier tsBlockSupplier = () -> tsBlock1; - List> valueStatistcsList = + Statistics[] valueStatistcsList = chunkMetadata1.getValueChunkMetadataList().stream() .map(IMetadata::getStatistics) - .collect(Collectors.toList()); + .toArray(Statistics[]::new); MemAlignedPageReader alignedPageReader = new MemAlignedPageReader( tsBlockSupplier, @@ -120,10 +117,10 @@ private MemAlignedPageReader generateAlignedPageReader() { private MemAlignedPageReader generateSingleColumnAlignedPageReader() { Supplier tsBlockSupplier = () -> tsBlock2; - List> valueStatistcsList = + Statistics[] valueStatistcsList = chunkMetadata2.getValueChunkMetadataList().stream() .map(IMetadata::getStatistics) - .collect(Collectors.toList()); + .toArray(Statistics[]::new); MemAlignedPageReader alignedPageReader = new MemAlignedPageReader( tsBlockSupplier, diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemAlignedChunkLoaderTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemAlignedChunkLoaderTest.java index 1a23760554677..c4297c9b071d2 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemAlignedChunkLoaderTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemAlignedChunkLoaderTest.java @@ -75,26 +75,26 @@ public void testMemAlignedChunkLoader() throws IOException { timeStatitsticsList.add(timeStatistics); Mockito.when(chunk.getTimeStatisticsList()).thenReturn(timeStatitsticsList); - List>> valuesStatitsticsList = new ArrayList<>(); - List> valuesStatistics = new ArrayList<>(); + List[]> valuesStatitsticsList = new ArrayList<>(); + Statistics[] valuesStatistics = new Statistics[6]; Statistics statistics1 = Mockito.mock(Statistics.class); Mockito.when(statistics1.hasNullValue(2)).thenReturn(true); - valuesStatistics.add(statistics1); + valuesStatistics[0] = statistics1; Statistics statistics2 = Mockito.mock(Statistics.class); Mockito.when(statistics2.hasNullValue(2)).thenReturn(true); - valuesStatistics.add(statistics2); + valuesStatistics[1] = statistics2; Statistics statistics3 = Mockito.mock(Statistics.class); Mockito.when(statistics3.hasNullValue(2)).thenReturn(true); - valuesStatistics.add(statistics3); + valuesStatistics[2] = statistics3; Statistics statistics4 = Mockito.mock(Statistics.class); Mockito.when(statistics4.hasNullValue(2)).thenReturn(true); - valuesStatistics.add(statistics4); + valuesStatistics[3] = statistics4; Statistics statistics5 = Mockito.mock(Statistics.class); Mockito.when(statistics5.hasNullValue(2)).thenReturn(true); - valuesStatistics.add(statistics5); + valuesStatistics[4] = statistics5; Statistics statistics6 = Mockito.mock(Statistics.class); Mockito.when(statistics6.hasNullValue(2)).thenReturn(true); - valuesStatistics.add(statistics6); + valuesStatistics[5] = statistics6; valuesStatitsticsList.add(valuesStatistics); Mockito.when(chunk.getValuesStatisticsList()).thenReturn(valuesStatitsticsList); From 9b0a6f8a823e973addb98c852db4d0e16d245a10 Mon Sep 17 00:00:00 2001 From: shizy Date: Mon, 23 Dec 2024 23:49:53 +0800 Subject: [PATCH 21/54] fix: memory chunk reader may read more points than expected in one page --- .../schemaregion/utils/ResourceByPathUtils.java | 2 +- .../dataregion/memtable/AlignedReadOnlyMemChunk.java | 9 +++++++++ .../dataregion/memtable/AlignedWritableMemChunk.java | 7 ++----- .../dataregion/memtable/IWritableMemChunk.java | 3 +++ .../dataregion/memtable/ReadOnlyMemChunk.java | 9 +++++++-- .../dataregion/memtable/WritableMemChunk.java | 5 +---- .../read/reader/chunk/MemAlignedChunkReader.java | 9 +++++++++ .../dataregion/read/reader/chunk/MemChunkReader.java | 8 ++++++++ .../datastructure/MergeSortAlignedTVListIterator.java | 8 ++++++++ .../db/utils/datastructure/MergeSortTvListIterator.java | 8 ++++++++ .../org/apache/iotdb/db/utils/datastructure/TVList.java | 9 +++++++++ 11 files changed, 65 insertions(+), 12 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/utils/ResourceByPathUtils.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/utils/ResourceByPathUtils.java index 91793daaa4cda..fff51bb95bbd2 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/utils/ResourceByPathUtils.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/utils/ResourceByPathUtils.java @@ -293,7 +293,7 @@ public ReadOnlyMemChunk getReadOnlyMemChunkFromMemTable( // column index list for the query List columnIndexList = - alignedMemChunk.getColumnIndexList(alignedFullPath.getSchemaList()); + alignedMemChunk.buildColumnIndexList(alignedFullPath.getSchemaList()); List timeColumnDeletion = null; List> valueColumnsDeletionList = null; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedReadOnlyMemChunk.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedReadOnlyMemChunk.java index 56c925b3d7b58..17d7c2f313d59 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedReadOnlyMemChunk.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedReadOnlyMemChunk.java @@ -50,6 +50,8 @@ import java.util.List; import java.util.Map; +import static org.apache.iotdb.db.storageengine.dataregion.memtable.IWritableMemChunk.MAX_NUMBER_OF_POINTS_IN_PAGE; + public class AlignedReadOnlyMemChunk extends ReadOnlyMemChunk { private final String timeChunkName; @@ -72,6 +74,8 @@ public class AlignedReadOnlyMemChunk extends ReadOnlyMemChunk { private final List columnIndexList; + private int workingTVListRows; + /** * The constructor for Aligned type. * @@ -138,6 +142,7 @@ public void initChunkMetaFromTvLists() { timeColumnDeletion, valueColumnsDeletionList, context.isIgnoreAllNullRows()); + this.workingTVListRows = timeValuePairIterator.getRowsForWorkingTVListIterator(); int[] alignedTvListOffsets = timeValuePairIterator.getAlignedTVListOffsets(); Statistics pageTimeStats = null; @@ -381,4 +386,8 @@ public List> getTimeStatisticsList() { public List[]> getValuesStatisticsList() { return valueStatisticsList; } + + public int workingTVListRows() { + return workingTVListRows; + } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedWritableMemChunk.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedWritableMemChunk.java index 8e00c21be07b8..7f1e85cd883a9 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedWritableMemChunk.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedWritableMemChunk.java @@ -196,10 +196,7 @@ protected void handoverAlignedTvList() { try { if (list.isSorted()) { sortedList.add(list); - return; - } - - if (list.getQueryContextList().isEmpty()) { + } else if (list.getQueryContextList().isEmpty()) { list.safelySort(); sortedList.add(list); } else { @@ -753,7 +750,7 @@ public List getSortedList() { return sortedList; } - public List getColumnIndexList(List schemaList) { + public List buildColumnIndexList(List schemaList) { List columnIndexList = new ArrayList<>(); for (IMeasurementSchema measurementSchema : schemaList) { columnIndexList.add( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/IWritableMemChunk.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/IWritableMemChunk.java index 38008be773e56..25749ad58aa54 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/IWritableMemChunk.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/IWritableMemChunk.java @@ -23,6 +23,7 @@ import org.apache.iotdb.db.storageengine.dataregion.wal.buffer.WALEntryValue; import org.apache.iotdb.db.utils.datastructure.TVList; +import org.apache.tsfile.common.conf.TSFileDescriptor; import org.apache.tsfile.enums.TSDataType; import org.apache.tsfile.utils.Binary; import org.apache.tsfile.utils.BitMap; @@ -33,6 +34,8 @@ public interface IWritableMemChunk extends WALEntryValue { int TVLIST_SORT_THRESHOLD = IoTDBDescriptor.getInstance().getConfig().getTvListSortThreshold(); + int MAX_NUMBER_OF_POINTS_IN_PAGE = + TSFileDescriptor.getInstance().getConfig().getMaxNumberOfPointsInPage(); boolean putLongWithFlushCheck(long t, long v); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/ReadOnlyMemChunk.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/ReadOnlyMemChunk.java index 49b08fa72b489..d16bea3d8192d 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/ReadOnlyMemChunk.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/ReadOnlyMemChunk.java @@ -50,6 +50,7 @@ import java.util.List; import java.util.Map; +import static org.apache.iotdb.db.storageengine.dataregion.memtable.IWritableMemChunk.MAX_NUMBER_OF_POINTS_IN_PAGE; import static org.apache.iotdb.db.utils.ModificationUtils.isPointDeleted; /** @@ -82,8 +83,7 @@ public class ReadOnlyMemChunk { // tvlist rowCount during query private Map tvListQueryMap; - protected static final int MAX_NUMBER_OF_POINTS_IN_PAGE = - TSFileDescriptor.getInstance().getConfig().getMaxNumberOfPointsInPage(); + private int workingTVListRows; protected ReadOnlyMemChunk(QueryContext context) { this.context = context; @@ -145,6 +145,7 @@ public void initChunkMetaFromTvLists() { List tvLists = new ArrayList<>(tvListQueryMap.keySet()); MergeSortTvListIterator timeValuePairIterator = new MergeSortTvListIterator(tvLists, floatPrecision, encoding); + this.workingTVListRows = timeValuePairIterator.getRowsForWorkingTVListIterator(); int[] tvListOffsets = timeValuePairIterator.getTVListOffsets(); while (timeValuePairIterator.hasNextTimeValuePair()) { if (cnt % MAX_NUMBER_OF_POINTS_IN_PAGE == 0) { @@ -328,4 +329,8 @@ public QueryContext getContext() { public TsBlock getTsBlock() { return null; } + + public int workingTVListRows() { + return workingTVListRows; + } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/WritableMemChunk.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/WritableMemChunk.java index 967ed485a9e12..1e513085a8e06 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/WritableMemChunk.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/WritableMemChunk.java @@ -70,10 +70,7 @@ protected void handoverTvList() { try { if (list.isSorted()) { sortedList.add(list); - return; - } - - if (list.getQueryContextList().isEmpty()) { + } else if (list.getQueryContextList().isEmpty()) { list.safelySort(); sortedList.add(list); } else { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemAlignedChunkReader.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemAlignedChunkReader.java index e5a8bd87cd3c6..a666d2e2ea089 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemAlignedChunkReader.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemAlignedChunkReader.java @@ -41,6 +41,8 @@ import java.util.List; import java.util.function.Supplier; +import static org.apache.iotdb.db.storageengine.dataregion.memtable.IWritableMemChunk.MAX_NUMBER_OF_POINTS_IN_PAGE; + /** To read aligned chunk data in memory. */ public class MemAlignedChunkReader implements IChunkReader { private final AlignedReadOnlyMemChunk readableChunk; @@ -61,6 +63,7 @@ public MemAlignedChunkReader(AlignedReadOnlyMemChunk readableChunk, Filter globa readableChunk.getTimeColumnDeletion(), readableChunk.getValueColumnsDeletionList(), readableChunk.getContext().isIgnoreAllNullRows()); + timeValuePairIterator.setRowsForWorkingTVListIterator(readableChunk.workingTVListRows()); this.globalTimeFilter = globalTimeFilter; this.pageReaderList = new ArrayList<>(); initAllPageReaders( @@ -196,6 +199,12 @@ private synchronized void writeValidValuesIntoTsBlock(TsBlockBuilder builder) { } builder.declarePosition(); } + if (builder.getPositionCount() > MAX_NUMBER_OF_POINTS_IN_PAGE) { + throw new RuntimeException( + String.format( + "Points in current page %d is larger than %d", + builder.getPositionCount(), MAX_NUMBER_OF_POINTS_IN_PAGE)); + } } } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemChunkReader.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemChunkReader.java index bee00bebcee10..e511432c8a85e 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemChunkReader.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemChunkReader.java @@ -41,6 +41,7 @@ import java.util.List; import java.util.function.Supplier; +import static org.apache.iotdb.db.storageengine.dataregion.memtable.IWritableMemChunk.MAX_NUMBER_OF_POINTS_IN_PAGE; import static org.apache.iotdb.db.utils.ModificationUtils.isPointDeleted; /** To read chunk data in memory. */ @@ -60,6 +61,7 @@ public MemChunkReader(ReadOnlyMemChunk readableChunk, Filter globalTimeFilter) { timeValuePairIterator = new MergeSortTvListIterator( tvLists, readableChunk.getFloatPrecision(), readableChunk.getEncoding()); + timeValuePairIterator.setRowsForWorkingTVListIterator(readableChunk.workingTVListRows()); this.globalTimeFilter = globalTimeFilter; this.pageReaderList = new ArrayList<>(); initAllPageReaders( @@ -232,6 +234,12 @@ private synchronized void writeValidValuesIntoTsBlock(TsBlockBuilder builder) builder.declarePosition(); } } + if (builder.getPositionCount() > MAX_NUMBER_OF_POINTS_IN_PAGE) { + throw new RuntimeException( + String.format( + "Points in current page %d is larger than %d", + builder.getPositionCount(), MAX_NUMBER_OF_POINTS_IN_PAGE)); + } } } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MergeSortAlignedTVListIterator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MergeSortAlignedTVListIterator.java index 2243d19c81691..56a7705644646 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MergeSortAlignedTVListIterator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MergeSortAlignedTVListIterator.java @@ -139,4 +139,12 @@ public void setAlignedTVListOffsets(int[] alignedTvListOffsets) { } probeNext = false; } + + public int getRowsForWorkingTVListIterator() { + return alignedTvListIterators[alignedTvListIterators.length - 1].getRows(); + } + + public void setRowsForWorkingTVListIterator(int rows) { + alignedTvListIterators[alignedTvListIterators.length - 1].setRows(rows); + } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MergeSortTvListIterator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MergeSortTvListIterator.java index 5e928f30d114a..6b9468936c9ce 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MergeSortTvListIterator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MergeSortTvListIterator.java @@ -125,4 +125,12 @@ public void setTVListOffsets(int[] tvListOffsets) { } selectedTVListIndex = -1; } + + public int getRowsForWorkingTVListIterator() { + return tvListIterators[tvListIterators.length - 1].getRows(); + } + + public void setRowsForWorkingTVListIterator(int rows) { + tvListIterators[tvListIterators.length - 1].setRows(rows); + } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java index 9ec092ac3566e..eab94a7c7d1c3 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java @@ -695,5 +695,14 @@ protected void step() { probeNext = false; currentTime = index < rows ? getTime(index) : Long.MIN_VALUE; } + + public int getRows() { + return rows; + } + + public void setRows(int rows) { + this.rows = rows; + this.currentTime = index < rows ? getTime(index) : Long.MIN_VALUE; + } } } From 23bf1f7079f84a6dea4eb819044dd26f63331190 Mon Sep 17 00:00:00 2001 From: shizy Date: Sat, 21 Dec 2024 17:12:28 +0800 Subject: [PATCH 22/54] update chunk & page statistic for aligend memchunk by column --- .../memtable/AlignedReadOnlyMemChunk.java | 340 +++++++++++++----- .../memtable/AlignedWritableMemChunk.java | 152 +++++--- .../reader/chunk/MemAlignedChunkReader.java | 200 ++++++++--- .../reader/chunk/MemAlignedPageReader.java | 2 +- .../db/utils/datastructure/AlignedTVList.java | 191 +++++----- .../MergeSortAlignedTVListIterator.java | 119 ++++-- .../datastructure/PageColumnAccessInfo.java | 49 +++ 7 files changed, 762 insertions(+), 291 deletions(-) create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/PageColumnAccessInfo.java diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedReadOnlyMemChunk.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedReadOnlyMemChunk.java index 17d7c2f313d59..5c9ade1f0aa29 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedReadOnlyMemChunk.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedReadOnlyMemChunk.java @@ -24,6 +24,7 @@ import org.apache.iotdb.db.storageengine.dataregion.read.reader.chunk.MemAlignedChunkLoader; import org.apache.iotdb.db.utils.datastructure.AlignedTVList; import org.apache.iotdb.db.utils.datastructure.MergeSortAlignedTVListIterator; +import org.apache.iotdb.db.utils.datastructure.PageColumnAccessInfo; import org.apache.iotdb.db.utils.datastructure.TVList; import org.apache.tsfile.block.column.ColumnBuilder; @@ -39,6 +40,7 @@ import org.apache.tsfile.read.common.block.TsBlock; import org.apache.tsfile.read.common.block.TsBlockBuilder; import org.apache.tsfile.read.reader.IPointReader; +import org.apache.tsfile.utils.BitMap; import org.apache.tsfile.utils.TsPrimitiveType; import org.apache.tsfile.write.UnSupportedDataTypeException; import org.apache.tsfile.write.schema.IMeasurementSchema; @@ -51,6 +53,7 @@ import java.util.Map; import static org.apache.iotdb.db.storageengine.dataregion.memtable.IWritableMemChunk.MAX_NUMBER_OF_POINTS_IN_PAGE; +import static org.apache.iotdb.db.utils.ModificationUtils.isPointDeleted; public class AlignedReadOnlyMemChunk extends ReadOnlyMemChunk { private final String timeChunkName; @@ -120,6 +123,135 @@ public void sortTvLists() { } } + private void updateTimeStatistics( + long[] time, + int count, + Statistics chunkTimeStatistics, + Statistics pageTimeStatistics) { + // update time statistics + for (int index = 0; index < count; index++) { + chunkTimeStatistics.update(time[index]); + pageTimeStatistics.update(time[index]); + } + pageTimeStatistics.setEmpty(count == 0); + } + + private void checkValueStatistics( + Statistics[] chunkValueStatistics, + Statistics[] pageValueStatistics, + List dataTypes, + int columnIndex) { + if (pageValueStatistics[columnIndex] == null) { + Statistics valueStatistics = + Statistics.getStatsByType(dataTypes.get(columnIndex)); + pageValueStatistics[columnIndex] = valueStatistics; + } + if (chunkValueStatistics[columnIndex] == null) { + Statistics chunkValueStats = + Statistics.getStatsByType(dataTypes.get(columnIndex)); + chunkValueStatistics[columnIndex] = chunkValueStats; + } + } + + private void updateValueStatistics( + long[] time, + PageColumnAccessInfo[] columnAccessInfo, + Statistics[] chunkValueStatistics, + Statistics[] pageValueStatistics, + MergeSortAlignedTVListIterator timeValuePairIterator) { + // update value statistics + for (int columnIndex = 0; columnIndex < dataTypes.size(); columnIndex++) { + PageColumnAccessInfo pageAccessInfo = columnAccessInfo[columnIndex]; + switch (dataTypes.get(columnIndex)) { + case BOOLEAN: + for (int index = 0; index < pageAccessInfo.count(); index++) { + int[] accessInfo = pageAccessInfo.get(index); + TsPrimitiveType value = + timeValuePairIterator.getPrimitiveObject(accessInfo, columnIndex); + if (value != null) { + checkValueStatistics( + pageValueStatistics, chunkValueStatistics, dataTypes, columnIndex); + pageValueStatistics[columnIndex].update(time[index], value.getBoolean()); + chunkValueStatistics[columnIndex].update(time[index], value.getBoolean()); + } + } + break; + case INT32: + case DATE: + for (int index = 0; index < pageAccessInfo.count(); index++) { + int[] accessInfo = pageAccessInfo.get(index); + TsPrimitiveType value = + timeValuePairIterator.getPrimitiveObject(accessInfo, columnIndex); + if (value != null) { + checkValueStatistics( + pageValueStatistics, chunkValueStatistics, dataTypes, columnIndex); + pageValueStatistics[columnIndex].update(time[index], value.getInt()); + chunkValueStatistics[columnIndex].update(time[index], value.getInt()); + } + } + break; + case INT64: + case TIMESTAMP: + for (int index = 0; index < pageAccessInfo.count(); index++) { + int[] accessInfo = pageAccessInfo.get(index); + TsPrimitiveType value = + timeValuePairIterator.getPrimitiveObject(accessInfo, columnIndex); + if (value != null) { + checkValueStatistics( + pageValueStatistics, chunkValueStatistics, dataTypes, columnIndex); + pageValueStatistics[columnIndex].update(time[index], value.getLong()); + chunkValueStatistics[columnIndex].update(time[index], value.getLong()); + } + } + break; + case FLOAT: + for (int index = 0; index < pageAccessInfo.count(); index++) { + int[] accessInfo = pageAccessInfo.get(index); + TsPrimitiveType value = + timeValuePairIterator.getPrimitiveObject(accessInfo, columnIndex); + if (value != null) { + checkValueStatistics( + pageValueStatistics, chunkValueStatistics, dataTypes, columnIndex); + pageValueStatistics[columnIndex].update(time[index], value.getFloat()); + chunkValueStatistics[columnIndex].update(time[index], value.getFloat()); + } + } + break; + case DOUBLE: + for (int index = 0; index < pageAccessInfo.count(); index++) { + int[] accessInfo = pageAccessInfo.get(index); + TsPrimitiveType value = + timeValuePairIterator.getPrimitiveObject(accessInfo, columnIndex); + if (value != null) { + checkValueStatistics( + pageValueStatistics, chunkValueStatistics, dataTypes, columnIndex); + pageValueStatistics[columnIndex].update(time[index], value.getDouble()); + chunkValueStatistics[columnIndex].update(time[index], value.getDouble()); + } + } + break; + case TEXT: + case BLOB: + case STRING: + for (int index = 0; index < pageAccessInfo.count(); index++) { + int[] accessInfo = pageAccessInfo.get(index); + TsPrimitiveType value = + timeValuePairIterator.getPrimitiveObject(accessInfo, columnIndex); + if (value != null) { + checkValueStatistics( + pageValueStatistics, chunkValueStatistics, dataTypes, columnIndex); + pageValueStatistics[columnIndex].update(time[index], value.getBinary()); + chunkValueStatistics[columnIndex].update(time[index], value.getBinary()); + } + } + break; + default: + throw new UnSupportedDataTypeException( + String.format("Data type %s is not supported.", dataTypes.get(columnIndex))); + } + } + } + @Override public void initChunkMetaFromTvLists() { // init chunk meta @@ -129,112 +261,121 @@ public void initChunkMetaFromTvLists() { new ChunkMetadata(timeChunkName, TSDataType.VECTOR, null, null, 0, chunkTimeStatistics); Statistics[] chunkValueStatistics = new Statistics[valueChunkNames.size()]; - Arrays.fill(chunkValueStatistics, null); - int cnt = 0; + // create MergeSortAlignedTVListIterator List alignedTvLists = new ArrayList<>(alignedTvListQueryMap.keySet()); MergeSortAlignedTVListIterator timeValuePairIterator = new MergeSortAlignedTVListIterator( alignedTvLists, + dataTypes, columnIndexList, floatPrecision, encodingList, - timeColumnDeletion, - valueColumnsDeletionList, context.isIgnoreAllNullRows()); this.workingTVListRows = timeValuePairIterator.getRowsForWorkingTVListIterator(); int[] alignedTvListOffsets = timeValuePairIterator.getAlignedTVListOffsets(); - Statistics pageTimeStats = null; + // iterate to build column access info and split pages + int pointsInPage = 0; + long[] time = new long[MAX_NUMBER_OF_POINTS_IN_PAGE]; + PageColumnAccessInfo[] pageColumnAccessInfo = new PageColumnAccessInfo[dataTypes.size()]; + for (int i = 0; i < pageColumnAccessInfo.length; i++) { + pageColumnAccessInfo[i] = new PageColumnAccessInfo(); + } + + int[] timeDeleteCursor = new int[] {0}; + List valueColumnDeleteCursor = new ArrayList<>(); + if (valueColumnsDeletionList != null) { + valueColumnsDeletionList.forEach(x -> valueColumnDeleteCursor.add(new int[] {0})); + } + while (timeValuePairIterator.hasNextTimeValuePair()) { + long timestamp = timeValuePairIterator.getTime(); + if (timeColumnDeletion != null + && isPointDeleted(timestamp, timeColumnDeletion, timeDeleteCursor)) { + timeValuePairIterator.step(); + continue; + } + + BitMap bitMap = timeValuePairIterator.getBitmap(); + if (valueColumnsDeletionList != null) { + for (int columnIndex = 0; columnIndex < dataTypes.size(); columnIndex++) { + if (isPointDeleted( + timestamp, + valueColumnsDeletionList.get(columnIndex), + valueColumnDeleteCursor.get(columnIndex))) { + bitMap.mark(columnIndex); + } + } + } + if (context.isIgnoreAllNullRows() && bitMap.isAllMarked()) { + timeValuePairIterator.step(); + continue; + } + // Split pages - if (cnt % MAX_NUMBER_OF_POINTS_IN_PAGE == 0) { + if (pointsInPage == 0) { + // Add new pageTimeStatistics array and pageValueStatistics into list Statistics pageTimeStatistics = Statistics.getStatsByType(TSDataType.VECTOR); - pageTimeStatistics.setEmpty(false); timeStatisticsList.add(pageTimeStatistics); - pageTimeStats = timeStatisticsList.get(timeStatisticsList.size() - 1); - Statistics[] pageValueStatistics = new Statistics[valueChunkNames.size()]; - Arrays.fill(pageValueStatistics, null); valueStatisticsList.add(pageValueStatistics); + pageOffsetsList.add(Arrays.copyOf(alignedTvListOffsets, alignedTvListOffsets.length)); } - // Update Page & Chunk Statistics - TimeValuePair tvPair = timeValuePairIterator.nextTimeValuePair(); - pageTimeStats.update(tvPair.getTimestamp()); - chunkTimeStatistics.update(tvPair.getTimestamp()); + // prepare column access info for current page + int[][] accessInfo = timeValuePairIterator.getColumnAccessInfo(); + for (int i = 0; i < dataTypes.size(); i++) { + time[pointsInPage] = timeValuePairIterator.getTime(); + pageColumnAccessInfo[i].add(accessInfo[i]); + } + timeValuePairIterator.step(); + pointsInPage++; - Statistics[] pageValuesStats = - valueStatisticsList.get(valueStatisticsList.size() - 1); - TsPrimitiveType[] primitiveValues = tvPair.getValue().getVector(); - for (int column = 0; column < primitiveValues.length; column++) { - if (primitiveValues[column] == null) { - continue; + if (pointsInPage == MAX_NUMBER_OF_POINTS_IN_PAGE) { + Statistics pageTimeStatistics = + timeStatisticsList.get(timeStatisticsList.size() - 1); + Statistics[] pageValueStatistics = + valueStatisticsList.get(valueStatisticsList.size() - 1); + + // update page time & value statistics + updateTimeStatistics( + time, MAX_NUMBER_OF_POINTS_IN_PAGE, chunkTimeStatistics, pageTimeStatistics); + updateValueStatistics( + time, + pageColumnAccessInfo, + chunkValueStatistics, + pageValueStatistics, + timeValuePairIterator); + + // reset + for (PageColumnAccessInfo columnAccessInfo : pageColumnAccessInfo) { + columnAccessInfo.reset(); } + chunkTimeStatistics.setEmpty(false); + pointsInPage = 0; + } + } - if (pageValuesStats[column] == null) { - Statistics valueStatistics = - Statistics.getStatsByType(dataTypes.get(column)); - pageValuesStats[column] = valueStatistics; - } - if (chunkValueStatistics[column] == null) { - Statistics chunkValueStats = - Statistics.getStatsByType(dataTypes.get(column)); - chunkValueStatistics[column] = chunkValueStats; - } + if (pointsInPage > 0) { + Statistics pageTimeStatistics = + timeStatisticsList.get(timeStatisticsList.size() - 1); + Statistics[] pageValueStatistics = + valueStatisticsList.get(valueStatisticsList.size() - 1); - switch (dataTypes.get(column)) { - case BOOLEAN: - pageValuesStats[column].update( - tvPair.getTimestamp(), primitiveValues[column].getBoolean()); - chunkValueStatistics[column].update( - tvPair.getTimestamp(), primitiveValues[column].getBoolean()); - break; - case INT32: - case DATE: - pageValuesStats[column].update(tvPair.getTimestamp(), primitiveValues[column].getInt()); - chunkValueStatistics[column].update( - tvPair.getTimestamp(), primitiveValues[column].getInt()); - break; - case INT64: - case TIMESTAMP: - pageValuesStats[column].update( - tvPair.getTimestamp(), primitiveValues[column].getLong()); - chunkValueStatistics[column].update( - tvPair.getTimestamp(), primitiveValues[column].getLong()); - break; - case FLOAT: - pageValuesStats[column].update( - tvPair.getTimestamp(), primitiveValues[column].getFloat()); - chunkValueStatistics[column].update( - tvPair.getTimestamp(), primitiveValues[column].getFloat()); - break; - case DOUBLE: - pageValuesStats[column].update( - tvPair.getTimestamp(), primitiveValues[column].getDouble()); - chunkValueStatistics[column].update( - tvPair.getTimestamp(), primitiveValues[column].getDouble()); - break; - case TEXT: - case BLOB: - case STRING: - pageValuesStats[column].update( - tvPair.getTimestamp(), primitiveValues[column].getBinary()); - chunkValueStatistics[column].update( - tvPair.getTimestamp(), primitiveValues[column].getBinary()); - break; - default: - throw new UnSupportedDataTypeException( - String.format("Data type %s is not supported.", dataTypes.get(column))); - } - } - cnt++; + updateTimeStatistics(time, pointsInPage, chunkTimeStatistics, pageTimeStatistics); + updateValueStatistics( + time, + pageColumnAccessInfo, + chunkValueStatistics, + pageValueStatistics, + timeValuePairIterator); + chunkTimeStatistics.setEmpty(false); } pageOffsetsList.add(Arrays.copyOf(alignedTvListOffsets, alignedTvListOffsets.length)); - chunkTimeStatistics.setEmpty(cnt == 0); // aligned chunk meta List chunkValueMetadataList = new ArrayList<>(); @@ -296,21 +437,58 @@ private TsBlock buildTsBlock() { } } + private boolean isAllColumnNull(TsPrimitiveType[] primitiveValues) { + for (TsPrimitiveType primitiveValue : primitiveValues) { + if (primitiveValue != null) { + return false; + } + } + return true; + } + private void writeValidValuesIntoTsBlock(TsBlockBuilder builder) throws IOException { List alignedTvLists = new ArrayList<>(alignedTvListQueryMap.keySet()); - IPointReader timeValuePairIterator = + MergeSortAlignedTVListIterator timeValuePairIterator = new MergeSortAlignedTVListIterator( alignedTvLists, + dataTypes, columnIndexList, floatPrecision, encodingList, - timeColumnDeletion, - valueColumnsDeletionList, context.isIgnoreAllNullRows()); + + int[] timeDeleteCursor = new int[] {0}; + List valueColumnDeleteCursor = new ArrayList<>(); + if (valueColumnsDeletionList != null) { + valueColumnsDeletionList.forEach(x -> valueColumnDeleteCursor.add(new int[] {0})); + } + while (timeValuePairIterator.hasNextTimeValuePair()) { TimeValuePair tvPair = timeValuePairIterator.nextTimeValuePair(); - builder.getTimeColumnBuilder().writeLong(tvPair.getTimestamp()); + // skip deleted rows + if (timeColumnDeletion != null + && isPointDeleted(tvPair.getTimestamp(), timeColumnDeletion, timeDeleteCursor)) { + timeValuePairIterator.step(); + continue; + } + + TsPrimitiveType[] primitiveValues = tvPair.getValue().getVector(); + if (valueColumnsDeletionList != null) { + for (int columnIndex = 0; columnIndex < primitiveValues.length; columnIndex++) { + if (isPointDeleted( + tvPair.getTimestamp(), + valueColumnsDeletionList.get(columnIndex), + valueColumnDeleteCursor.get(columnIndex))) { + primitiveValues[columnIndex] = null; + } + } + } + if (context.isIgnoreAllNullRows() && isAllColumnNull(primitiveValues)) { + timeValuePairIterator.step(); + continue; + } + builder.getTimeColumnBuilder().writeLong(tvPair.getTimestamp()); // value columns TsPrimitiveType[] values = tvPair.getValue().getVector(); for (int columnIndex = 0; columnIndex < values.length; columnIndex++) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedWritableMemChunk.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedWritableMemChunk.java index 7f1e85cd883a9..87388a876a668 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedWritableMemChunk.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedWritableMemChunk.java @@ -27,11 +27,10 @@ import org.apache.iotdb.db.storageengine.dataregion.wal.utils.WALWriteUtils; import org.apache.iotdb.db.utils.datastructure.AlignedTVList; import org.apache.iotdb.db.utils.datastructure.MergeSortAlignedTVListIterator; +import org.apache.iotdb.db.utils.datastructure.PageColumnAccessInfo; import org.apache.iotdb.db.utils.datastructure.TVList; -import org.apache.tsfile.common.conf.TSFileDescriptor; import org.apache.tsfile.enums.TSDataType; -import org.apache.tsfile.read.TimeValuePair; import org.apache.tsfile.utils.Binary; import org.apache.tsfile.utils.BitMap; import org.apache.tsfile.utils.Pair; @@ -57,25 +56,23 @@ public class AlignedWritableMemChunk implements IWritableMemChunk { private final Map measurementIndexMap; + private final List dataTypes; private final List schemaList; private AlignedTVList list; private List sortedList; private boolean ignoreAllNullRows; - private static final int MAX_NUMBER_OF_POINTS_IN_PAGE = - TSFileDescriptor.getInstance().getConfig().getMaxNumberOfPointsInPage(); - private static final String UNSUPPORTED_TYPE = "Unsupported data type:"; public AlignedWritableMemChunk(List schemaList, boolean isTableModel) { this.measurementIndexMap = new LinkedHashMap<>(); - List dataTypeList = new ArrayList<>(); + this.dataTypes = new ArrayList<>(); this.schemaList = schemaList; for (int i = 0; i < schemaList.size(); i++) { measurementIndexMap.put(schemaList.get(i).getMeasurementName(), i); - dataTypeList.add(schemaList.get(i).getType()); + dataTypes.add(schemaList.get(i).getType()); } - this.list = AlignedTVList.newAlignedList(dataTypeList); + this.list = AlignedTVList.newAlignedList(dataTypes); this.sortedList = new ArrayList<>(); this.ignoreAllNullRows = !isTableModel; } @@ -88,6 +85,7 @@ private AlignedWritableMemChunk( measurementIndexMap.put(schemaList.get(i).getMeasurementName(), i); } this.list = list; + this.dataTypes = list.getTsDataTypes(); this.sortedList = new ArrayList<>(); this.ignoreAllNullRows = !isTableModel; } @@ -216,11 +214,7 @@ protected void handoverAlignedTvList() { } finally { list.unlockQueryList(); } - List dataTypeList = new ArrayList<>(); - for (IMeasurementSchema schema : schemaList) { - dataTypeList.add(schema.getType()); - } - this.list = AlignedTVList.newAlignedList(dataTypeList); + this.list = AlignedTVList.newAlignedList(dataTypes); } @Override @@ -588,6 +582,78 @@ private void encodeWorkingAlignedTVList(IChunkWriter chunkWriter) { } } + private void writePageValuesIntoWriter( + IChunkWriter chunkWriter, + long[] times, + PageColumnAccessInfo[] pageColumnAccessInfo, + MergeSortAlignedTVListIterator timeValuePairIterator) { + AlignedChunkWriterImpl alignedChunkWriter = (AlignedChunkWriterImpl) chunkWriter; + + // update value statistics + for (int columnIndex = 0; columnIndex < dataTypes.size(); columnIndex++) { + ValueChunkWriter valueChunkWriter = + alignedChunkWriter.getValueChunkWriterByIndex(columnIndex); + PageColumnAccessInfo pageAccessInfo = pageColumnAccessInfo[columnIndex]; + switch (dataTypes.get(columnIndex)) { + case BOOLEAN: + for (int index = 0; index < pageAccessInfo.count(); index++) { + int[] accessInfo = pageAccessInfo.get(index); + TsPrimitiveType value = + timeValuePairIterator.getPrimitiveObject(accessInfo, columnIndex); + valueChunkWriter.write(times[index], value.getBoolean(), value.getValue() == null); + } + break; + case INT32: + case DATE: + for (int index = 0; index < pageAccessInfo.count(); index++) { + int[] accessInfo = pageAccessInfo.get(index); + TsPrimitiveType value = + timeValuePairIterator.getPrimitiveObject(accessInfo, columnIndex); + valueChunkWriter.write(times[index], value.getInt(), value.getValue() == null); + } + break; + case INT64: + case TIMESTAMP: + for (int index = 0; index < pageAccessInfo.count(); index++) { + int[] accessInfo = pageAccessInfo.get(index); + TsPrimitiveType value = + timeValuePairIterator.getPrimitiveObject(accessInfo, columnIndex); + valueChunkWriter.write(times[index], value.getLong(), value.getValue() == null); + } + break; + case FLOAT: + for (int index = 0; index < pageAccessInfo.count(); index++) { + int[] accessInfo = pageAccessInfo.get(index); + TsPrimitiveType value = + timeValuePairIterator.getPrimitiveObject(accessInfo, columnIndex); + valueChunkWriter.write(times[index], value.getFloat(), value.getValue() == null); + } + break; + case DOUBLE: + for (int index = 0; index < pageAccessInfo.count(); index++) { + int[] accessInfo = pageAccessInfo.get(index); + TsPrimitiveType value = + timeValuePairIterator.getPrimitiveObject(accessInfo, columnIndex); + valueChunkWriter.write(times[index], value.getDouble(), value.getValue() == null); + } + break; + case TEXT: + case BLOB: + case STRING: + for (int index = 0; index < pageAccessInfo.count(); index++) { + int[] accessInfo = pageAccessInfo.get(index); + TsPrimitiveType value = + timeValuePairIterator.getPrimitiveObject(accessInfo, columnIndex); + valueChunkWriter.write(times[index], value.getBinary(), value.getValue() == null); + } + break; + default: + throw new UnSupportedDataTypeException( + String.format("Data type %s is not supported.", dataTypes.get(columnIndex))); + } + } + } + @SuppressWarnings({"squid:S6541", "squid:S3776"}) @Override public void encode(IChunkWriter chunkWriter) { @@ -602,51 +668,41 @@ public void encode(IChunkWriter chunkWriter) { alignedTvLists.add(list); MergeSortAlignedTVListIterator timeValuePairIterator = new MergeSortAlignedTVListIterator( - alignedTvLists, null, null, null, null, null, ignoreAllNullRows); + alignedTvLists, dataTypes, null, null, null, ignoreAllNullRows); int pointsInPage = 0; long[] times = new long[MAX_NUMBER_OF_POINTS_IN_PAGE]; + + PageColumnAccessInfo[] pageColumnAccessInfo = new PageColumnAccessInfo[dataTypes.size()]; + for (int i = 0; i < pageColumnAccessInfo.length; i++) { + pageColumnAccessInfo[i] = new PageColumnAccessInfo(); + } + while (timeValuePairIterator.hasNextTimeValuePair()) { - TimeValuePair tvPair = timeValuePairIterator.nextTimeValuePair(); - times[pointsInPage] = tvPair.getTimestamp(); - TsPrimitiveType[] values = tvPair.getValue().getVector(); - for (int columnIndex = 0; columnIndex < values.length; columnIndex++) { - ValueChunkWriter valueChunkWriter = - alignedChunkWriter.getValueChunkWriterByIndex(columnIndex); - boolean isNull = values[columnIndex].getValue() == null; - switch (schemaList.get(columnIndex).getType()) { - case BOOLEAN: - valueChunkWriter.write(tvPair.getTimestamp(), values[columnIndex].getBoolean(), isNull); - break; - case INT32: - case DATE: - valueChunkWriter.write(tvPair.getTimestamp(), values[columnIndex].getInt(), isNull); - break; - case INT64: - case TIMESTAMP: - valueChunkWriter.write(tvPair.getTimestamp(), values[columnIndex].getLong(), isNull); - break; - case FLOAT: - valueChunkWriter.write(tvPair.getTimestamp(), values[columnIndex].getFloat(), isNull); - break; - case DOUBLE: - valueChunkWriter.write(tvPair.getTimestamp(), values[columnIndex].getDouble(), isNull); - break; - case TEXT: - case BLOB: - case STRING: - valueChunkWriter.write(tvPair.getTimestamp(), values[columnIndex].getBinary(), isNull); - break; - default: - break; - } + // prepare column access info for current page + int[][] accessInfo = timeValuePairIterator.getColumnAccessInfo(); + for (int i = 0; i < dataTypes.size(); i++) { + times[pointsInPage] = timeValuePairIterator.getTime(); + pageColumnAccessInfo[i].add(accessInfo[i]); } + timeValuePairIterator.step(); pointsInPage++; - if (pointsInPage % MAX_NUMBER_OF_POINTS_IN_PAGE == 0) { + + if (pointsInPage == MAX_NUMBER_OF_POINTS_IN_PAGE) { + writePageValuesIntoWriter(chunkWriter, times, pageColumnAccessInfo, timeValuePairIterator); alignedChunkWriter.write(times, pointsInPage, 0); + + for (PageColumnAccessInfo columnAccessInfo : pageColumnAccessInfo) { + columnAccessInfo.reset(); + } pointsInPage = 0; } } + + if (pointsInPage > 0) { + writePageValuesIntoWriter(chunkWriter, times, pageColumnAccessInfo, timeValuePairIterator); + alignedChunkWriter.write(times, pointsInPage, 0); + } } private void maybeReleaseTvList(AlignedTVList alignedTvList) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemAlignedChunkReader.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemAlignedChunkReader.java index a666d2e2ea089..75cb1c51e1abf 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemAlignedChunkReader.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemAlignedChunkReader.java @@ -22,17 +22,19 @@ import org.apache.iotdb.db.storageengine.dataregion.memtable.AlignedReadOnlyMemChunk; import org.apache.iotdb.db.utils.datastructure.AlignedTVList; import org.apache.iotdb.db.utils.datastructure.MergeSortAlignedTVListIterator; +import org.apache.iotdb.db.utils.datastructure.PageColumnAccessInfo; import org.apache.tsfile.block.column.ColumnBuilder; import org.apache.tsfile.enums.TSDataType; import org.apache.tsfile.file.metadata.statistics.Statistics; -import org.apache.tsfile.read.TimeValuePair; import org.apache.tsfile.read.common.BatchData; +import org.apache.tsfile.read.common.TimeRange; import org.apache.tsfile.read.common.block.TsBlock; import org.apache.tsfile.read.common.block.TsBlockBuilder; import org.apache.tsfile.read.filter.basic.Filter; import org.apache.tsfile.read.reader.IChunkReader; import org.apache.tsfile.read.reader.IPageReader; +import org.apache.tsfile.utils.BitMap; import org.apache.tsfile.utils.TsPrimitiveType; import java.io.IOException; @@ -42,6 +44,7 @@ import java.util.function.Supplier; import static org.apache.iotdb.db.storageengine.dataregion.memtable.IWritableMemChunk.MAX_NUMBER_OF_POINTS_IN_PAGE; +import static org.apache.iotdb.db.utils.ModificationUtils.isPointDeleted; /** To read aligned chunk data in memory. */ public class MemAlignedChunkReader implements IChunkReader { @@ -57,11 +60,10 @@ public MemAlignedChunkReader(AlignedReadOnlyMemChunk readableChunk, Filter globa timeValuePairIterator = new MergeSortAlignedTVListIterator( alignedTVLists, + readableChunk.getDataTypes(), readableChunk.getColumnIndexList(), readableChunk.getFloatPrecision(), readableChunk.getEncodingList(), - readableChunk.getTimeColumnDeletion(), - readableChunk.getValueColumnsDeletionList(), readableChunk.getContext().isIgnoreAllNullRows()); timeValuePairIterator.setRowsForWorkingTVListIterator(readableChunk.workingTVListRows()); this.globalTimeFilter = globalTimeFilter; @@ -152,59 +154,165 @@ private boolean isOutOfMemPageBounds() { return true; } + private void writePageTimeIntoBuilder(long[] time, int count, TsBlockBuilder builder) { + for (int index = 0; index < count; index++) { + builder.getTimeColumnBuilder().writeLong(time[index]); + } + } + + private void writePageValuesIntoBuilder( + PageColumnAccessInfo[] columnAccessInfo, + List tsDataTypes, + TsBlockBuilder builder) { + for (int columnIndex = 0; columnIndex < tsDataTypes.size(); columnIndex++) { + PageColumnAccessInfo pageAccessInfo = columnAccessInfo[columnIndex]; + ColumnBuilder valueBuilder = builder.getColumnBuilder(columnIndex); + switch (tsDataTypes.get(columnIndex)) { + case BOOLEAN: + for (int index = 0; index < pageAccessInfo.count(); index++) { + int[] accessInfo = pageAccessInfo.get(index); + TsPrimitiveType value = + timeValuePairIterator.getPrimitiveObject(accessInfo, columnIndex); + if (value == null) { + valueBuilder.appendNull(); + } else { + valueBuilder.writeBoolean(value.getBoolean()); + } + } + break; + case INT32: + case DATE: + for (int index = 0; index < pageAccessInfo.count(); index++) { + int[] accessInfo = pageAccessInfo.get(index); + TsPrimitiveType value = + timeValuePairIterator.getPrimitiveObject(accessInfo, columnIndex); + if (value == null) { + valueBuilder.appendNull(); + } else { + valueBuilder.writeInt(value.getInt()); + } + } + break; + case INT64: + case TIMESTAMP: + for (int index = 0; index < pageAccessInfo.count(); index++) { + int[] accessInfo = pageAccessInfo.get(index); + TsPrimitiveType value = + timeValuePairIterator.getPrimitiveObject(accessInfo, columnIndex); + if (value == null) { + valueBuilder.appendNull(); + } else { + valueBuilder.writeLong(value.getLong()); + } + } + break; + case FLOAT: + for (int index = 0; index < pageAccessInfo.count(); index++) { + int[] accessInfo = pageAccessInfo.get(index); + TsPrimitiveType value = + timeValuePairIterator.getPrimitiveObject(accessInfo, columnIndex); + if (value == null) { + valueBuilder.appendNull(); + } else { + valueBuilder.writeFloat(value.getFloat()); + } + } + break; + case DOUBLE: + for (int index = 0; index < pageAccessInfo.count(); index++) { + int[] accessInfo = pageAccessInfo.get(index); + TsPrimitiveType value = + timeValuePairIterator.getPrimitiveObject(accessInfo, columnIndex); + if (value == null) { + valueBuilder.appendNull(); + } else { + valueBuilder.writeDouble(value.getDouble()); + } + } + break; + case TEXT: + case BLOB: + case STRING: + for (int index = 0; index < pageAccessInfo.count(); index++) { + int[] accessInfo = pageAccessInfo.get(index); + TsPrimitiveType value = + timeValuePairIterator.getPrimitiveObject(accessInfo, columnIndex); + if (value == null) { + valueBuilder.appendNull(); + } else { + valueBuilder.writeBinary(value.getBinary()); + } + } + break; + default: + break; + } + } + } + // read one page and write to tsblock private synchronized void writeValidValuesIntoTsBlock(TsBlockBuilder builder) { + boolean ignoreAllNullRows = readableChunk.getContext().isIgnoreAllNullRows(); List tsDataTypes = readableChunk.getDataTypes(); + List timeColumnDeletion = readableChunk.getTimeColumnDeletion(); + List> valueColumnsDeletionList = readableChunk.getValueColumnsDeletionList(); + + int pointsInPage = 0; + long[] time = new long[MAX_NUMBER_OF_POINTS_IN_PAGE]; + PageColumnAccessInfo[] pageColumnAccessInfo = new PageColumnAccessInfo[tsDataTypes.size()]; + for (int i = 0; i < pageColumnAccessInfo.length; i++) { + pageColumnAccessInfo[i] = new PageColumnAccessInfo(); + } + + int[] timeDeleteCursor = new int[] {0}; + List valueColumnDeleteCursor = new ArrayList<>(); + if (valueColumnsDeletionList != null) { + valueColumnsDeletionList.forEach(x -> valueColumnDeleteCursor.add(new int[] {0})); + } + while (timeValuePairIterator.hasNextTimeValuePair()) { if (isOutOfMemPageBounds()) { break; } - TimeValuePair tvPair = timeValuePairIterator.nextTimeValuePair(); - builder.getTimeColumnBuilder().writeLong(tvPair.getTimestamp()); - - // value columns - TsPrimitiveType[] values = tvPair.getValue().getVector(); - for (int columnIndex = 0; columnIndex < values.length; columnIndex++) { - if (values[columnIndex] == null) { - builder.getColumnBuilder(columnIndex).appendNull(); - continue; - } - ColumnBuilder valueBuilder = builder.getColumnBuilder(columnIndex); - switch (tsDataTypes.get(columnIndex)) { - case BOOLEAN: - valueBuilder.writeBoolean(values[columnIndex].getBoolean()); - break; - case INT32: - case DATE: - valueBuilder.writeInt(values[columnIndex].getInt()); - break; - case INT64: - case TIMESTAMP: - valueBuilder.writeLong(values[columnIndex].getLong()); - break; - case FLOAT: - valueBuilder.writeFloat(values[columnIndex].getFloat()); - break; - case DOUBLE: - valueBuilder.writeDouble(values[columnIndex].getDouble()); - break; - case TEXT: - case BLOB: - case STRING: - valueBuilder.writeBinary(values[columnIndex].getBinary()); - break; - default: - break; + + // skip deleted rows + long timestamp = timeValuePairIterator.getTime(); + if (timeColumnDeletion != null + && isPointDeleted(timestamp, timeColumnDeletion, timeDeleteCursor)) { + timeValuePairIterator.step(); + continue; + } + + BitMap bitMap = timeValuePairIterator.getBitmap(); + if (valueColumnsDeletionList != null) { + for (int columnIndex = 0; columnIndex < tsDataTypes.size(); columnIndex++) { + if (isPointDeleted( + timestamp, + valueColumnsDeletionList.get(columnIndex), + valueColumnDeleteCursor.get(columnIndex))) { + bitMap.mark(columnIndex); + } } } - builder.declarePosition(); - } - if (builder.getPositionCount() > MAX_NUMBER_OF_POINTS_IN_PAGE) { - throw new RuntimeException( - String.format( - "Points in current page %d is larger than %d", - builder.getPositionCount(), MAX_NUMBER_OF_POINTS_IN_PAGE)); + if (ignoreAllNullRows && bitMap.isAllMarked()) { + timeValuePairIterator.step(); + continue; + } + + // prepare column access info for current page + int[][] accessInfo = timeValuePairIterator.getColumnAccessInfo(); + for (int i = 0; i < tsDataTypes.size(); i++) { + time[pointsInPage] = timeValuePairIterator.getTime(); + pageColumnAccessInfo[i].add(accessInfo[i]); + } + timeValuePairIterator.step(); + pointsInPage++; } + + // write time and values into builders + writePageTimeIntoBuilder(time, pointsInPage, builder); + writePageValuesIntoBuilder(pageColumnAccessInfo, tsDataTypes, builder); + builder.declarePositions(pointsInPage); } } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemAlignedPageReader.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemAlignedPageReader.java index c0d388bed9f26..8206703262d59 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemAlignedPageReader.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemAlignedPageReader.java @@ -262,7 +262,7 @@ private void updatePageStatisticsFromTsBlock( for (int i = 0; i < tsBlock.getPositionCount(); i++) { timeStatistics.update(tsBlock.getTimeByIndex(i)); } - timeStatistics.setEmpty(false); + // timeStatistics.setEmpty(false); for (int column = 0; column < tsDataTypes.size(); column++) { switch (tsDataTypes.get(column)) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/AlignedTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/AlignedTVList.java index e506c62d1f9c7..56a349c2ea585 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/AlignedTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/AlignedTVList.java @@ -42,10 +42,12 @@ import org.apache.tsfile.utils.ReadWriteForEncodingUtils; import org.apache.tsfile.utils.ReadWriteIOUtils; import org.apache.tsfile.utils.TsPrimitiveType; +import org.apache.tsfile.write.UnSupportedDataTypeException; import java.io.DataInputStream; import java.io.IOException; import java.util.ArrayList; +import java.util.Arrays; import java.util.List; import java.util.Objects; @@ -1433,7 +1435,11 @@ public BitMap getTimeColDeletedMap() { * @param rowIndex should be the sorted index. */ public boolean isTimeDeleted(int rowIndex) { - int bitmapIndex = getValueIndex(rowIndex); + return isTimeDeleted(rowIndex, true); + } + + public boolean isTimeDeleted(int index, boolean needConvertIndex) { + int bitmapIndex = needConvertIndex ? getValueIndex(index) : index; if (timeColDeletedMap == null || timeColDeletedMap.getSize() <= bitmapIndex) { return false; } @@ -1487,80 +1493,49 @@ public List> getBitMaps() { } public AlignedTVListIterator iterator( + List dataTypeList, List columnIndexList, boolean ignoreAllNullRows, Integer floatPrecision, - List encodingList, - List timeColumnDeletion, - List> valueColumnsDeletionList) { + List encodingList) { return new AlignedTVListIterator( - columnIndexList, - ignoreAllNullRows, - floatPrecision, - encodingList, - timeColumnDeletion, - valueColumnsDeletionList); + dataTypeList, columnIndexList, ignoreAllNullRows, floatPrecision, encodingList); } /* AlignedTVList Iterator */ public class AlignedTVListIterator extends TVListIterator { private final BitMap allValueColDeletedMap; - private TimeValuePair currTvPair; + private final List dataTypeList; private final Integer floatPrecision; private final List encodingList; List columnIndexList; - - private final boolean ignoreAllNullRows; - private final List timeColumnDeletion; - private int[] timeDeleteCursor; - private final List> valueColumnsDeletionList; - private List valueColumnDeleteCursor; + private final int[] validRowIndex; public AlignedTVListIterator( + List dataTypeList, List columnIndexList, boolean ignoreAllNullRows, Integer floatPrecision, - List encodingList, - List timeColumnDeletion, - List> valueColumnsDeletionList) { + List encodingList) { super(null, null); + this.dataTypeList = dataTypeList; this.columnIndexList = columnIndexList; - this.ignoreAllNullRows = ignoreAllNullRows; this.allValueColDeletedMap = ignoreAllNullRows ? getAllValueColDeletedMap() : null; this.floatPrecision = floatPrecision; this.encodingList = encodingList; - this.timeColumnDeletion = timeColumnDeletion; - this.valueColumnsDeletionList = valueColumnsDeletionList; - if (timeColumnDeletion != null) { - this.timeDeleteCursor = new int[] {0}; - } - if (valueColumnsDeletionList != null) { - this.valueColumnDeleteCursor = new ArrayList<>(); - valueColumnsDeletionList.forEach(x -> valueColumnDeleteCursor.add(new int[] {0})); - } - } - - private boolean isAllColumnNull(TimeValuePair tvPair) { - TsPrimitiveType[] primitiveValues = tvPair.getValue().getVector(); - for (TsPrimitiveType primitiveValue : primitiveValues) { - if (primitiveValue != null) { - return false; - } - } - return true; + this.validRowIndex = new int[dataTypeList.size()]; } private void prepareNext() { - currTvPair = null; // find the first row that is neither deleted nor empty (all NULL values) boolean findValidRow = false; while (index < rows && !findValidRow) { + int rowIndex = getValueIndex(index); // all columns values are deleted - if ((allValueColDeletedMap != null && allValueColDeletedMap.isMarked(getValueIndex(index))) - || (timeColumnDeletion != null - && isPointDeleted(currentTime, timeColumnDeletion, timeDeleteCursor))) { + if ((allValueColDeletedMap != null && allValueColDeletedMap.isMarked(rowIndex)) + || isTimeDeleted(rowIndex, false)) { index++; currentTime = index < rows ? getTime(index) : Long.MIN_VALUE; continue; @@ -1571,42 +1546,20 @@ && isPointDeleted(currentTime, timeColumnDeletion, timeDeleteCursor))) { probeNext = true; return; } - - // check whether null column exits - currTvPair = - getTimeValuePair(index, currentTime, floatPrecision, encodingList, columnIndexList); - TsPrimitiveType[] primitiveValues = currTvPair.getValue().getVector(); - for (int columnIndex = 0; columnIndex < primitiveValues.length; columnIndex++) { - if (valueColumnsDeletionList != null - && isPointDeleted( - currentTime, - valueColumnsDeletionList.get(columnIndex), - valueColumnDeleteCursor.get(columnIndex))) { - primitiveValues[columnIndex] = null; - } - } - if (ignoreAllNullRows && isAllColumnNull(currTvPair)) { - currTvPair = null; - index++; - currentTime = index < rows ? getTime(index) : Long.MIN_VALUE; - } else { - findValidRow = true; - } + Arrays.fill(validRowIndex, rowIndex); + findValidRow = true; } // handle duplicated timestamp while (index + 1 < rows && getTime(index + 1) == currentTime) { index++; // skip all-Null rows if allValueColDeletedMap exits - if (allValueColDeletedMap == null - || !allValueColDeletedMap.isMarked(getValueIndex(index))) { - TimeValuePair tvPair = - getTimeValuePair(index, currentTime, floatPrecision, encodingList, columnIndexList); - TsPrimitiveType[] primitiveValues = tvPair.getValue().getVector(); - for (int columnIndex = 0; columnIndex < primitiveValues.length; columnIndex++) { + int rowIndex = getValueIndex(index); + if (allValueColDeletedMap == null || !allValueColDeletedMap.isMarked(rowIndex)) { + for (int columnIndex = 0; columnIndex < dataTypeList.size(); columnIndex++) { // update currTvPair if the column is not null - if (primitiveValues[columnIndex] != null) { - currTvPair.getValue().getVector()[columnIndex] = primitiveValues[columnIndex]; + if (!isNull(rowIndex, columnIndex)) { + validRowIndex[columnIndex] = rowIndex; } } } @@ -1632,12 +1585,17 @@ public TimeValuePair next() { if (!hasNext()) { return null; } - TimeValuePair ret = currTvPair; - index++; - currentTime = index < rows ? getTime(index) : Long.MIN_VALUE; - currTvPair = null; - probeNext = false; - return ret; + + TsPrimitiveType[] vector = new TsPrimitiveType[dataTypeList.size()]; + for (int columnIndex = 0; columnIndex < dataTypeList.size(); columnIndex++) { + // update currTvPair if the column is not null + vector[columnIndex] = getPrimitiveObject(validRowIndex[columnIndex], columnIndex); + } + TimeValuePair tvPair = + new TimeValuePair(currentTime, TsPrimitiveType.getByType(TSDataType.VECTOR, vector)); + + step(); + return tvPair; } @Override @@ -1645,7 +1603,80 @@ public TimeValuePair current() { if (!hasCurrent()) { return null; } - return currTvPair; + TsPrimitiveType[] vector = new TsPrimitiveType[dataTypeList.size()]; + for (int columnIndex = 0; columnIndex < dataTypeList.size(); columnIndex++) { + // update currTvPair if the column is not null + vector[columnIndex] = getPrimitiveObject(validRowIndex[columnIndex], columnIndex); + } + return new TimeValuePair(currentTime, TsPrimitiveType.getByType(TSDataType.VECTOR, vector)); + } + + public boolean isNull(int rowIndex, int columnIndex) { + int validColumnIndex = + (columnIndexList == null) ? columnIndex : columnIndexList.get(columnIndex); + if (validColumnIndex < 0 || validColumnIndex >= dataTypes.size()) { + return true; + } + return isNullValue(rowIndex, validColumnIndex); + } + + public TsPrimitiveType getPrimitiveObject(int rowIndex, int columnIndex) { + if (rowIndex < 0 || rowIndex >= rows) { + return null; + } + int validColumnIndex = + (columnIndexList == null) ? columnIndex : columnIndexList.get(columnIndex); + if (validColumnIndex < 0 || validColumnIndex >= dataTypes.size()) { + return null; + } + if (isNullValue(rowIndex, validColumnIndex)) { + return null; + } + switch (dataTypeList.get(columnIndex)) { + case BOOLEAN: + return TsPrimitiveType.getByType( + TSDataType.BOOLEAN, getBooleanByValueIndex(rowIndex, validColumnIndex)); + case INT32: + case DATE: + return TsPrimitiveType.getByType( + TSDataType.INT32, getIntByValueIndex(rowIndex, validColumnIndex)); + case INT64: + case TIMESTAMP: + return TsPrimitiveType.getByType( + TSDataType.INT64, getLongByValueIndex(rowIndex, validColumnIndex)); + case FLOAT: + float valueF = getFloatByValueIndex(rowIndex, validColumnIndex); + if (floatPrecision != null + && encodingList != null + && !Float.isNaN(valueF) + && (encodingList.get(columnIndex) == TSEncoding.RLE + || encodingList.get(columnIndex) == TSEncoding.TS_2DIFF)) { + valueF = MathUtils.roundWithGivenPrecision(valueF, floatPrecision); + } + return TsPrimitiveType.getByType(TSDataType.FLOAT, valueF); + case DOUBLE: + double valueD = getDoubleByValueIndex(rowIndex, validColumnIndex); + if (floatPrecision != null + && encodingList != null + && !Double.isNaN(valueD) + && (encodingList.get(columnIndex) == TSEncoding.RLE + || encodingList.get(columnIndex) == TSEncoding.TS_2DIFF)) { + valueD = MathUtils.roundWithGivenPrecision(valueD, floatPrecision); + } + return TsPrimitiveType.getByType(TSDataType.DOUBLE, valueD); + case TEXT: + case BLOB: + case STRING: + return TsPrimitiveType.getByType( + TSDataType.TEXT, getBinaryByValueIndex(rowIndex, validColumnIndex)); + default: + throw new UnSupportedDataTypeException( + String.format("Data type %s is not supported.", dataTypeList.get(columnIndex))); + } + } + + public int getValidRowIndex(int columnIndex) { + return validRowIndex[columnIndex]; } } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MergeSortAlignedTVListIterator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MergeSortAlignedTVListIterator.java index 56a7705644646..686c8d26a476c 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MergeSortAlignedTVListIterator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MergeSortAlignedTVListIterator.java @@ -19,10 +19,11 @@ package org.apache.iotdb.db.utils.datastructure; +import org.apache.tsfile.enums.TSDataType; import org.apache.tsfile.file.metadata.enums.TSEncoding; import org.apache.tsfile.read.TimeValuePair; -import org.apache.tsfile.read.common.TimeRange; import org.apache.tsfile.read.reader.IPointReader; +import org.apache.tsfile.utils.BitMap; import org.apache.tsfile.utils.TsPrimitiveType; import java.io.IOException; @@ -30,18 +31,23 @@ public class MergeSortAlignedTVListIterator implements IPointReader { private final AlignedTVList.AlignedTVListIterator[] alignedTvListIterators; + private final List tsDataTypes; + private boolean probeNext = false; - private TimeValuePair currentTvPair; + private boolean hasNext = false; private final int[] alignedTvListOffsets; + private final int[][] columnAccessInfo; + private long time; + private final BitMap bitMap; + public MergeSortAlignedTVListIterator( List alignedTvLists, + List tsDataTypes, List columnIndexList, Integer floatPrecision, List encodingList, - List timeColumnDeletion, - List> valueColumnsDeletionList, boolean ignoreAllNullRows) { this.alignedTvListIterators = new AlignedTVList.AlignedTVListIterator[alignedTvLists.size()]; for (int i = 0; i < alignedTvLists.size(); i++) { @@ -49,36 +55,43 @@ public MergeSortAlignedTVListIterator( alignedTvLists .get(i) .iterator( - columnIndexList, - ignoreAllNullRows, - floatPrecision, - encodingList, - timeColumnDeletion, - valueColumnsDeletionList); + tsDataTypes, columnIndexList, ignoreAllNullRows, floatPrecision, encodingList); } this.alignedTvListOffsets = new int[alignedTvLists.size()]; + this.tsDataTypes = tsDataTypes; + this.columnAccessInfo = new int[tsDataTypes.size()][]; + for (int i = 0; i < columnAccessInfo.length; i++) { + columnAccessInfo[i] = new int[2]; + } + this.bitMap = new BitMap(tsDataTypes.size()); } private void prepareNextRow() { - currentTvPair = null; - long time = Long.MAX_VALUE; + time = Long.MAX_VALUE; for (int i = 0; i < alignedTvListIterators.length; i++) { AlignedTVList.AlignedTVListIterator iterator = alignedTvListIterators[i]; if (iterator.hasNext() && iterator.currentTime() <= time) { - TimeValuePair tvPair = iterator.current(); - // check valueColumnsDeletionList - if (currentTvPair == null || iterator.currentTime() < time) { - currentTvPair = tvPair; + if (i == 0 || iterator.currentTime() < time) { + for (int columnIndex = 0; columnIndex < tsDataTypes.size(); columnIndex++) { + int rowIndex = iterator.getValidRowIndex(columnIndex); + columnAccessInfo[columnIndex] = new int[] {i, rowIndex}; + if (iterator.isNull(rowIndex, columnIndex)) { + bitMap.mark(columnIndex); + } + } + time = iterator.currentTime(); } else { - TsPrimitiveType[] primitiveValues = tvPair.getValue().getVector(); - for (int columnIndex = 0; columnIndex < primitiveValues.length; columnIndex++) { - // update currentTvPair if the column is not null - if (primitiveValues[columnIndex] != null) { - currentTvPair.getValue().getVector()[columnIndex] = primitiveValues[columnIndex]; + for (int columnIndex = 0; columnIndex < tsDataTypes.size(); columnIndex++) { + int rowIndex = iterator.getValidRowIndex(columnIndex); + // update if the column is not null + if (!iterator.isNull(rowIndex, columnIndex)) { + columnAccessInfo[columnIndex][0] = i; + columnAccessInfo[columnIndex][1] = rowIndex; + bitMap.unmark(columnIndex); } } } - time = iterator.currentTime(); + hasNext = true; } } probeNext = true; @@ -89,7 +102,7 @@ public boolean hasNextTimeValuePair() { if (!probeNext) { prepareNextRow(); } - return currentTvPair != null; + return hasNext; } @Override @@ -97,26 +110,40 @@ public TimeValuePair nextTimeValuePair() { if (!hasNextTimeValuePair()) { return null; } + TimeValuePair tvPair = buildTimeValuePair(); + step(); + return tvPair; + } + @Override + public TimeValuePair currentTimeValuePair() { + if (!hasNextTimeValuePair()) { + return null; + } + return buildTimeValuePair(); + } + + private TimeValuePair buildTimeValuePair() { + TsPrimitiveType[] vector = new TsPrimitiveType[tsDataTypes.size()]; + for (int columnIndex = 0; columnIndex < vector.length; columnIndex++) { + int[] accessInfo = columnAccessInfo[columnIndex]; + AlignedTVList.AlignedTVListIterator iterator = alignedTvListIterators[accessInfo[0]]; + vector[columnIndex] = iterator.getPrimitiveObject(accessInfo[1], columnIndex); + } + return new TimeValuePair(time, TsPrimitiveType.getByType(TSDataType.VECTOR, vector)); + } + + public void step() { for (int i = 0; i < alignedTvListIterators.length; i++) { AlignedTVList.AlignedTVListIterator iterator = alignedTvListIterators[i]; - if (iterator.hasCurrent() && iterator.currentTime() == currentTvPair.getTimestamp()) { + if (iterator.hasCurrent() && iterator.currentTime() == time) { alignedTvListIterators[i].step(); alignedTvListOffsets[i] = alignedTvListIterators[i].getIndex(); } } - - TimeValuePair ret = currentTvPair; probeNext = false; - return ret; - } - - @Override - public TimeValuePair currentTimeValuePair() { - if (!hasNextTimeValuePair()) { - return null; - } - return currentTvPair; + hasNext = false; + bitMap.reset(); } @Override @@ -138,6 +165,28 @@ public void setAlignedTVListOffsets(int[] alignedTvListOffsets) { this.alignedTvListOffsets[i] = alignedTvListOffsets[i]; } probeNext = false; + hasNext = false; + bitMap.reset(); + } + + public int[][] getColumnAccessInfo() { + return columnAccessInfo; + } + + public long getTime() { + return time; + } + + public TsPrimitiveType getPrimitiveObject(int[] accessInfo, int columnIndex) { + if (columnIndex >= columnAccessInfo.length) { + return null; + } + AlignedTVList.AlignedTVListIterator iterator = alignedTvListIterators[accessInfo[0]]; + return iterator.getPrimitiveObject(accessInfo[1], columnIndex); + } + + public BitMap getBitmap() { + return bitMap; } public int getRowsForWorkingTVListIterator() { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/PageColumnAccessInfo.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/PageColumnAccessInfo.java new file mode 100644 index 0000000000000..8a0e4c91c6cbc --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/PageColumnAccessInfo.java @@ -0,0 +1,49 @@ +/* + * 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 static org.apache.iotdb.db.storageengine.dataregion.memtable.IWritableMemChunk.MAX_NUMBER_OF_POINTS_IN_PAGE; + +public class PageColumnAccessInfo { + // time -> (selectedTVList, rowIndex) + private final int[][] accessIndices; + private int count; + + public PageColumnAccessInfo() { + this.accessIndices = new int[MAX_NUMBER_OF_POINTS_IN_PAGE][]; + this.count = 0; + } + + public int[] get(int index) { + return accessIndices[index]; + } + + public void add(int[] columnAccess) { + accessIndices[count++] = columnAccess; + } + + public int count() { + return count; + } + + public void reset() { + count = 0; + } +} From 298e16c44cc8cd08327769c2bdc0537beda66d83 Mon Sep 17 00:00:00 2001 From: shizy Date: Tue, 24 Dec 2024 21:40:38 +0800 Subject: [PATCH 23/54] revert: getAlignedValueForQuery --- .../db/utils/datastructure/AlignedTVList.java | 55 +++++++------------ 1 file changed, 19 insertions(+), 36 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/AlignedTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/AlignedTVList.java index 56a349c2ea585..b7cc51ad34331 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/AlignedTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/AlignedTVList.java @@ -245,7 +245,7 @@ public void putAlignedValue(long timestamp, Object[] value) { @Override public Object getAlignedValue(int index) { - return getAlignedValueForQuery(index, null, null, null); + return getAlignedValueForQuery(index, null, null); } @Override @@ -257,41 +257,30 @@ protected TimeValuePair getTimeValuePair( @Override public TimeValuePair getTimeValuePair(int index) { return new TimeValuePair( - getTime(index), (TsPrimitiveType) getAlignedValueForQuery(index, null, null, null)); + getTime(index), (TsPrimitiveType) getAlignedValueForQuery(index, null, null)); } private Object getAlignedValueForQuery( - int index, - Integer floatPrecision, - List encodingList, - List columnIndexList) { + int index, Integer floatPrecision, List encodingList) { if (index >= rowCount) { throw new ArrayIndexOutOfBoundsException(index); } int arrayIndex = index / ARRAY_SIZE; int elementIndex = index % ARRAY_SIZE; int valueIndex = indices.get(arrayIndex)[elementIndex]; - return getAlignedValueByValueIndex( - valueIndex, null, floatPrecision, encodingList, columnIndexList); + return getAlignedValueByValueIndex(valueIndex, null, floatPrecision, encodingList); } private TsPrimitiveType getAlignedValueByValueIndex( int valueIndex, int[] validIndexesForTimeDuplicatedRows, Integer floatPrecision, - List encodingList, - List columnIndexList) { + List encodingList) { if (valueIndex >= rowCount) { throw new ArrayIndexOutOfBoundsException(valueIndex); } - - int columns = columnIndexList == null ? values.size() : columnIndexList.size(); - TsPrimitiveType[] vector = new TsPrimitiveType[columns]; - for (int i = 0; i < columns; i++) { - int columnIndex = columnIndexList == null ? i : columnIndexList.get(i); - if (columnIndex < 0 || columnIndex >= values.size()) { - continue; - } + TsPrimitiveType[] vector = new TsPrimitiveType[values.size()]; + for (int columnIndex = 0; columnIndex < values.size(); columnIndex++) { List columnValues = values.get(columnIndex); int validValueIndex; if (validIndexesForTimeDuplicatedRows != null) { @@ -309,43 +298,43 @@ private TsPrimitiveType getAlignedValueByValueIndex( case BLOB: case STRING: Binary valueT = ((Binary[]) columnValues.get(arrayIndex))[elementIndex]; - vector[i] = TsPrimitiveType.getByType(TSDataType.TEXT, valueT); + vector[columnIndex] = TsPrimitiveType.getByType(TSDataType.TEXT, valueT); break; case FLOAT: float valueF = ((float[]) columnValues.get(arrayIndex))[elementIndex]; if (floatPrecision != null && encodingList != null && !Float.isNaN(valueF) - && (encodingList.get(i) == TSEncoding.RLE - || encodingList.get(i) == TSEncoding.TS_2DIFF)) { + && (encodingList.get(columnIndex) == TSEncoding.RLE + || encodingList.get(columnIndex) == TSEncoding.TS_2DIFF)) { valueF = MathUtils.roundWithGivenPrecision(valueF, floatPrecision); } - vector[i] = TsPrimitiveType.getByType(TSDataType.FLOAT, valueF); + vector[columnIndex] = TsPrimitiveType.getByType(TSDataType.FLOAT, valueF); break; case INT32: case DATE: int valueI = ((int[]) columnValues.get(arrayIndex))[elementIndex]; - vector[i] = TsPrimitiveType.getByType(TSDataType.INT32, valueI); + vector[columnIndex] = TsPrimitiveType.getByType(TSDataType.INT32, valueI); break; case INT64: case TIMESTAMP: long valueL = ((long[]) columnValues.get(arrayIndex))[elementIndex]; - vector[i] = TsPrimitiveType.getByType(TSDataType.INT64, valueL); + vector[columnIndex] = TsPrimitiveType.getByType(TSDataType.INT64, valueL); break; case DOUBLE: double valueD = ((double[]) columnValues.get(arrayIndex))[elementIndex]; if (floatPrecision != null && encodingList != null && !Double.isNaN(valueD) - && (encodingList.get(i) == TSEncoding.RLE - || encodingList.get(i) == TSEncoding.TS_2DIFF)) { + && (encodingList.get(columnIndex) == TSEncoding.RLE + || encodingList.get(columnIndex) == TSEncoding.TS_2DIFF)) { valueD = MathUtils.roundWithGivenPrecision(valueD, floatPrecision); } - vector[i] = TsPrimitiveType.getByType(TSDataType.DOUBLE, valueD); + vector[columnIndex] = TsPrimitiveType.getByType(TSDataType.DOUBLE, valueD); break; case BOOLEAN: boolean valueB = ((boolean[]) columnValues.get(arrayIndex))[elementIndex]; - vector[i] = TsPrimitiveType.getByType(TSDataType.BOOLEAN, valueB); + vector[columnIndex] = TsPrimitiveType.getByType(TSDataType.BOOLEAN, valueB); break; default: throw new UnsupportedOperationException(ERR_DATATYPE_NOT_CONSISTENT); @@ -755,15 +744,9 @@ public int getValidRowIndexForTimeDuplicatedRows( } protected TimeValuePair getTimeValuePair( - int index, - long time, - Integer floatPrecision, - List encodingList, - List columnIndexList) { + int index, long time, Integer floatPrecision, List encodingList) { return new TimeValuePair( - time, - (TsPrimitiveType) - getAlignedValueForQuery(index, floatPrecision, encodingList, columnIndexList)); + time, (TsPrimitiveType) getAlignedValueForQuery(index, floatPrecision, encodingList)); } @Override From 359929ddd79c059d8f4887aa6f318edc24796fcd Mon Sep 17 00:00:00 2001 From: shizy Date: Wed, 25 Dec 2024 08:57:41 +0800 Subject: [PATCH 24/54] fix: * CopyOnWriteArrayList for AlignedTVList bitmaps * memory control of column access --- .../iotdb/db/utils/datastructure/AlignedTVList.java | 12 ++++++------ .../MergeSortAlignedTVListIterator.java | 3 ++- .../utils/datastructure/PageColumnAccessInfo.java | 13 +++++++++---- 3 files changed, 17 insertions(+), 11 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/AlignedTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/AlignedTVList.java index b7cc51ad34331..fa10f9b603337 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/AlignedTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/AlignedTVList.java @@ -50,6 +50,7 @@ import java.util.Arrays; import java.util.List; import java.util.Objects; +import java.util.concurrent.CopyOnWriteArrayList; import static org.apache.iotdb.db.storageengine.rescon.memory.PrimitiveArrayManager.ARRAY_SIZE; import static org.apache.iotdb.db.storageengine.rescon.memory.PrimitiveArrayManager.TVLIST_SORT_ALGORITHM; @@ -85,14 +86,13 @@ public abstract class AlignedTVList extends TVList { AlignedTVList(List types) { super(); - indices = new ArrayList<>(types.size()); dataTypes = types; memoryBinaryChunkSize = new long[dataTypes.size()]; reachMaxChunkSizeFlag = false; values = new ArrayList<>(types.size()); for (int i = 0; i < types.size(); i++) { - values.add(new ArrayList<>()); + values.add(new CopyOnWriteArrayList<>()); } } @@ -166,7 +166,7 @@ public AlignedTVList clone() { } } if (cloneList.bitMaps.get(i) == null) { - List cloneColumnBitMaps = new ArrayList<>(); + List cloneColumnBitMaps = new CopyOnWriteArrayList<>(); for (BitMap bitMap : columnBitMaps) { cloneColumnBitMaps.add(bitMap == null ? null : bitMap.clone()); } @@ -350,8 +350,8 @@ public void extendColumn(TSDataType dataType) { bitMaps.add(null); } } - List columnValue = new ArrayList<>(); - List columnBitMaps = new ArrayList<>(); + List columnValue = new CopyOnWriteArrayList<>(); + List columnBitMaps = new CopyOnWriteArrayList<>(); for (int i = 0; i < timestamps.size(); i++) { switch (dataType) { case TEXT: @@ -879,7 +879,7 @@ private void markNullValue(int columnIndex, int arrayIndex, int elementIndex) { // if the bitmap in columnIndex is null, init the bitmap of this column from the beginning if (bitMaps.get(columnIndex) == null) { - List columnBitMaps = new ArrayList<>(); + List columnBitMaps = new CopyOnWriteArrayList<>(); for (int i = 0; i < values.get(columnIndex).size(); i++) { columnBitMaps.add(new BitMap(ARRAY_SIZE)); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MergeSortAlignedTVListIterator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MergeSortAlignedTVListIterator.java index 686c8d26a476c..9988874086100 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MergeSortAlignedTVListIterator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MergeSortAlignedTVListIterator.java @@ -74,7 +74,8 @@ private void prepareNextRow() { if (i == 0 || iterator.currentTime() < time) { for (int columnIndex = 0; columnIndex < tsDataTypes.size(); columnIndex++) { int rowIndex = iterator.getValidRowIndex(columnIndex); - columnAccessInfo[columnIndex] = new int[] {i, rowIndex}; + columnAccessInfo[columnIndex][0] = i; + columnAccessInfo[columnIndex][1] = rowIndex; if (iterator.isNull(rowIndex, columnIndex)) { bitMap.mark(columnIndex); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/PageColumnAccessInfo.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/PageColumnAccessInfo.java index 8a0e4c91c6cbc..1e84934741c74 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/PageColumnAccessInfo.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/PageColumnAccessInfo.java @@ -23,20 +23,25 @@ public class PageColumnAccessInfo { // time -> (selectedTVList, rowIndex) - private final int[][] accessIndices; + private final int[][] indices; private int count; public PageColumnAccessInfo() { - this.accessIndices = new int[MAX_NUMBER_OF_POINTS_IN_PAGE][]; + this.indices = new int[MAX_NUMBER_OF_POINTS_IN_PAGE][]; + for (int i = 0; i < MAX_NUMBER_OF_POINTS_IN_PAGE; i++) { + indices[i] = new int[2]; + } this.count = 0; } public int[] get(int index) { - return accessIndices[index]; + return indices[index]; } public void add(int[] columnAccess) { - accessIndices[count++] = columnAccess; + indices[count][0] = columnAccess[0]; + indices[count][1] = columnAccess[1]; + count++; } public int count() { From 6817370aea3a5ab81ef017d6879c0b695c12cb11 Mon Sep 17 00:00:00 2001 From: shizy Date: Wed, 25 Dec 2024 15:31:16 +0800 Subject: [PATCH 25/54] refactor: Tim/Quick/Backward TVList --- .../datastructure/BackAlignedTVList.java | 63 +--------- .../utils/datastructure/BackBinaryTVList.java | 68 ++--------- .../datastructure/BackBooleanTVList.java | 68 ++--------- .../utils/datastructure/BackDoubleTVList.java | 68 ++--------- .../utils/datastructure/BackFloatTVList.java | 68 ++--------- .../db/utils/datastructure/BackIntTVList.java | 69 ++--------- .../utils/datastructure/BackLongTVList.java | 68 ++--------- .../db/utils/datastructure/BackwardSort.java | 65 +++++++++-- .../datastructure/QuickAlignedTVList.java | 31 +---- .../datastructure/QuickBinaryTVList.java | 29 +---- .../datastructure/QuickBooleanTVList.java | 28 +---- .../datastructure/QuickDoubleTVList.java | 28 +---- .../utils/datastructure/QuickFloatTVList.java | 28 +---- .../utils/datastructure/QuickIntTVList.java | 34 ++---- .../utils/datastructure/QuickLongTVList.java | 28 +---- .../db/utils/datastructure/QuickSort.java | 28 ++++- .../iotdb/db/utils/datastructure/TVList.java | 12 +- .../utils/datastructure/TimAlignedTVList.java | 106 ++--------------- .../utils/datastructure/TimBinaryTVList.java | 108 ++---------------- .../utils/datastructure/TimBooleanTVList.java | 108 ++---------------- .../utils/datastructure/TimDoubleTVList.java | 108 ++---------------- .../utils/datastructure/TimFloatTVList.java | 108 ++---------------- .../db/utils/datastructure/TimIntTVList.java | 108 ++---------------- .../db/utils/datastructure/TimLongTVList.java | 108 ++---------------- .../iotdb/db/utils/datastructure/TimSort.java | 105 ++++++++++++++--- 25 files changed, 331 insertions(+), 1311 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BackAlignedTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BackAlignedTVList.java index e0616fda26e71..26c1b7b8a8ded 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BackAlignedTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BackAlignedTVList.java @@ -19,77 +19,24 @@ package org.apache.iotdb.db.utils.datastructure; -import org.apache.iotdb.db.storageengine.rescon.memory.PrimitiveArrayManager; - import org.apache.tsfile.enums.TSDataType; -import java.util.ArrayList; import java.util.List; -import static org.apache.iotdb.db.storageengine.rescon.memory.PrimitiveArrayManager.ARRAY_SIZE; - -public class BackAlignedTVList extends QuickAlignedTVList implements BackwardSort { - private final List tmpTimestamps = new ArrayList<>(); - private final List tmpIndices = new ArrayList<>(); - private int tmpLength = 0; +public class BackAlignedTVList extends QuickAlignedTVList { + private final BackwardSort policy; BackAlignedTVList(List types) { super(types); + policy = new BackwardSort(this); } @Override public void sort() { if (!sorted) { - backwardSort(timestamps, rowCount); - clearTmp(); + policy.backwardSort(timestamps, rowCount); + policy.clearTmp(); } sorted = true; } - - @Override - public void setFromTmp(int src, int dest) { - set( - dest, - tmpTimestamps.get(src / ARRAY_SIZE)[src % ARRAY_SIZE], - tmpIndices.get(src / ARRAY_SIZE)[src % ARRAY_SIZE]); - } - - @Override - public void setToTmp(int src, int dest) { - tmpTimestamps.get(dest / ARRAY_SIZE)[dest % ARRAY_SIZE] = getTime(src); - tmpIndices.get(dest / ARRAY_SIZE)[dest % ARRAY_SIZE] = getValueIndex(src); - } - - @Override - public void backward_set(int src, int dest) { - set(src, dest); - } - - @Override - public int compareTmp(int idx, int tmpIdx) { - long t1 = getTime(idx); - long t2 = tmpTimestamps.get(tmpIdx / ARRAY_SIZE)[tmpIdx % ARRAY_SIZE]; - return Long.compare(t1, t2); - } - - @Override - public void checkTmpLength(int len) { - while (len > tmpLength) { - tmpTimestamps.add((long[]) getPrimitiveArraysByType(TSDataType.INT64)); - tmpIndices.add((int[]) getPrimitiveArraysByType(TSDataType.INT32)); - tmpLength += ARRAY_SIZE; - } - } - - @Override - public void clearTmp() { - for (long[] dataArray : tmpTimestamps) { - PrimitiveArrayManager.release(dataArray); - } - tmpTimestamps.clear(); - for (int[] dataArray : tmpIndices) { - PrimitiveArrayManager.release(dataArray); - } - tmpIndices.clear(); - } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BackBinaryTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BackBinaryTVList.java index 9a72499376b4c..48b0ecd4a199d 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BackBinaryTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BackBinaryTVList.java @@ -18,73 +18,19 @@ */ package org.apache.iotdb.db.utils.datastructure; -import org.apache.iotdb.db.storageengine.rescon.memory.PrimitiveArrayManager; +public class BackBinaryTVList extends QuickBinaryTVList { + private final BackwardSort policy; -import org.apache.tsfile.enums.TSDataType; - -import java.util.ArrayList; -import java.util.List; - -import static org.apache.iotdb.db.storageengine.rescon.memory.PrimitiveArrayManager.ARRAY_SIZE; - -public class BackBinaryTVList extends QuickBinaryTVList implements BackwardSort { - private final List tmpTimestamps = new ArrayList<>(); - private final List tmpIndices = new ArrayList<>(); - private int tmpLength = 0; + BackBinaryTVList() { + policy = new BackwardSort(this); + } @Override public void sort() { if (!sorted) { - backwardSort(timestamps, rowCount); - clearTmp(); + policy.backwardSort(timestamps, rowCount); + policy.clearTmp(); } sorted = true; } - - @Override - public void setFromTmp(int src, int dest) { - set( - dest, - tmpTimestamps.get(src / ARRAY_SIZE)[src % ARRAY_SIZE], - tmpIndices.get(src / ARRAY_SIZE)[src % ARRAY_SIZE]); - } - - @Override - public void setToTmp(int src, int dest) { - tmpTimestamps.get(dest / ARRAY_SIZE)[dest % ARRAY_SIZE] = getTime(src); - tmpIndices.get(dest / ARRAY_SIZE)[dest % ARRAY_SIZE] = getValueIndex(src); - } - - @Override - public void backward_set(int src, int dest) { - set(src, dest); - } - - @Override - public int compareTmp(int idx, int tmpIdx) { - long t1 = getTime(idx); - long t2 = tmpTimestamps.get(tmpIdx / ARRAY_SIZE)[tmpIdx % ARRAY_SIZE]; - return Long.compare(t1, t2); - } - - @Override - public void checkTmpLength(int len) { - while (len > tmpLength) { - tmpTimestamps.add((long[]) getPrimitiveArraysByType(TSDataType.INT64)); - tmpIndices.add((int[]) getPrimitiveArraysByType(TSDataType.INT32)); - tmpLength += ARRAY_SIZE; - } - } - - @Override - public void clearTmp() { - for (long[] dataArray : tmpTimestamps) { - PrimitiveArrayManager.release(dataArray); - } - tmpTimestamps.clear(); - for (int[] dataArray : tmpIndices) { - PrimitiveArrayManager.release(dataArray); - } - tmpIndices.clear(); - } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BackBooleanTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BackBooleanTVList.java index c7b39a388ee91..e6e2153e3df5c 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BackBooleanTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BackBooleanTVList.java @@ -19,73 +19,19 @@ package org.apache.iotdb.db.utils.datastructure; -import org.apache.iotdb.db.storageengine.rescon.memory.PrimitiveArrayManager; +public class BackBooleanTVList extends QuickBooleanTVList { + private final BackwardSort policy; -import org.apache.tsfile.enums.TSDataType; - -import java.util.ArrayList; -import java.util.List; - -import static org.apache.iotdb.db.storageengine.rescon.memory.PrimitiveArrayManager.ARRAY_SIZE; - -public class BackBooleanTVList extends QuickBooleanTVList implements BackwardSort { - private final List tmpTimestamps = new ArrayList<>(); - private final List tmpIndices = new ArrayList<>(); - private int tmpLength = 0; + BackBooleanTVList() { + policy = new BackwardSort(this); + } @Override public void sort() { if (!sorted) { - backwardSort(timestamps, rowCount); - clearTmp(); + policy.backwardSort(timestamps, rowCount); + policy.clearTmp(); } sorted = true; } - - @Override - public void setFromTmp(int src, int dest) { - set( - dest, - tmpTimestamps.get(src / ARRAY_SIZE)[src % ARRAY_SIZE], - tmpIndices.get(src / ARRAY_SIZE)[src % ARRAY_SIZE]); - } - - @Override - public void setToTmp(int src, int dest) { - tmpTimestamps.get(dest / ARRAY_SIZE)[dest % ARRAY_SIZE] = getTime(src); - tmpIndices.get(dest / ARRAY_SIZE)[dest % ARRAY_SIZE] = getValueIndex(src); - } - - @Override - public void backward_set(int src, int dest) { - set(src, dest); - } - - @Override - public int compareTmp(int idx, int tmpIdx) { - long t1 = getTime(idx); - long t2 = tmpTimestamps.get(tmpIdx / ARRAY_SIZE)[tmpIdx % ARRAY_SIZE]; - return Long.compare(t1, t2); - } - - @Override - public void checkTmpLength(int len) { - while (len > tmpLength) { - tmpTimestamps.add((long[]) getPrimitiveArraysByType(TSDataType.INT64)); - tmpIndices.add((int[]) getPrimitiveArraysByType(TSDataType.INT32)); - tmpLength += ARRAY_SIZE; - } - } - - @Override - public void clearTmp() { - for (long[] dataArray : tmpTimestamps) { - PrimitiveArrayManager.release(dataArray); - } - tmpTimestamps.clear(); - for (int[] dataArray : tmpIndices) { - PrimitiveArrayManager.release(dataArray); - } - tmpIndices.clear(); - } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BackDoubleTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BackDoubleTVList.java index 48c80d1a3ab40..2915906e0e61d 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BackDoubleTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BackDoubleTVList.java @@ -19,73 +19,19 @@ package org.apache.iotdb.db.utils.datastructure; -import org.apache.iotdb.db.storageengine.rescon.memory.PrimitiveArrayManager; +public class BackDoubleTVList extends QuickDoubleTVList { + private final BackwardSort policy; -import org.apache.tsfile.enums.TSDataType; - -import java.util.ArrayList; -import java.util.List; - -import static org.apache.iotdb.db.storageengine.rescon.memory.PrimitiveArrayManager.ARRAY_SIZE; - -public class BackDoubleTVList extends QuickDoubleTVList implements BackwardSort { - private final List tmpTimestamps = new ArrayList<>(); - private final List tmpIndices = new ArrayList<>(); - private int tmpLength = 0; + BackDoubleTVList() { + policy = new BackwardSort(this); + } @Override public void sort() { if (!sorted) { - backwardSort(timestamps, rowCount); - clearTmp(); + policy.backwardSort(timestamps, rowCount); + policy.clearTmp(); } sorted = true; } - - @Override - public void setFromTmp(int src, int dest) { - set( - dest, - tmpTimestamps.get(src / ARRAY_SIZE)[src % ARRAY_SIZE], - tmpIndices.get(src / ARRAY_SIZE)[src % ARRAY_SIZE]); - } - - @Override - public void setToTmp(int src, int dest) { - tmpTimestamps.get(dest / ARRAY_SIZE)[dest % ARRAY_SIZE] = getTime(src); - tmpIndices.get(dest / ARRAY_SIZE)[dest % ARRAY_SIZE] = getValueIndex(src); - } - - @Override - public void backward_set(int src, int dest) { - set(src, dest); - } - - @Override - public int compareTmp(int idx, int tmpIdx) { - long t1 = getTime(idx); - long t2 = tmpTimestamps.get(tmpIdx / ARRAY_SIZE)[tmpIdx % ARRAY_SIZE]; - return Long.compare(t1, t2); - } - - @Override - public void checkTmpLength(int len) { - while (len > tmpLength) { - tmpTimestamps.add((long[]) getPrimitiveArraysByType(TSDataType.INT64)); - tmpIndices.add((int[]) getPrimitiveArraysByType(TSDataType.INT32)); - tmpLength += ARRAY_SIZE; - } - } - - @Override - public void clearTmp() { - for (long[] dataArray : tmpTimestamps) { - PrimitiveArrayManager.release(dataArray); - } - tmpTimestamps.clear(); - for (int[] dataArray : tmpIndices) { - PrimitiveArrayManager.release(dataArray); - } - tmpIndices.clear(); - } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BackFloatTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BackFloatTVList.java index 79cf6d2bc06c3..41a1acfa1bc3f 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BackFloatTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BackFloatTVList.java @@ -19,73 +19,19 @@ package org.apache.iotdb.db.utils.datastructure; -import org.apache.iotdb.db.storageengine.rescon.memory.PrimitiveArrayManager; +public class BackFloatTVList extends QuickFloatTVList { + private final BackwardSort policy; -import org.apache.tsfile.enums.TSDataType; - -import java.util.ArrayList; -import java.util.List; - -import static org.apache.iotdb.db.storageengine.rescon.memory.PrimitiveArrayManager.ARRAY_SIZE; - -public class BackFloatTVList extends QuickFloatTVList implements BackwardSort { - private final List tmpTimestamps = new ArrayList<>(); - private final List tmpIndices = new ArrayList<>(); - private int tmpLength = 0; + BackFloatTVList() { + policy = new BackwardSort(this); + } @Override public void sort() { if (!sorted) { - backwardSort(timestamps, rowCount); - clearTmp(); + policy.backwardSort(timestamps, rowCount); + policy.clearTmp(); } sorted = true; } - - @Override - public void setFromTmp(int src, int dest) { - set( - dest, - tmpTimestamps.get(src / ARRAY_SIZE)[src % ARRAY_SIZE], - tmpIndices.get(src / ARRAY_SIZE)[src % ARRAY_SIZE]); - } - - @Override - public void setToTmp(int src, int dest) { - tmpTimestamps.get(dest / ARRAY_SIZE)[dest % ARRAY_SIZE] = getTime(src); - tmpIndices.get(dest / ARRAY_SIZE)[dest % ARRAY_SIZE] = getValueIndex(src); - } - - @Override - public void backward_set(int src, int dest) { - set(src, dest); - } - - @Override - public int compareTmp(int idx, int tmpIdx) { - long t1 = getTime(idx); - long t2 = tmpTimestamps.get(tmpIdx / ARRAY_SIZE)[tmpIdx % ARRAY_SIZE]; - return Long.compare(t1, t2); - } - - @Override - public void checkTmpLength(int len) { - while (len > tmpLength) { - tmpTimestamps.add((long[]) getPrimitiveArraysByType(TSDataType.INT64)); - tmpIndices.add((int[]) getPrimitiveArraysByType(TSDataType.INT32)); - tmpLength += ARRAY_SIZE; - } - } - - @Override - public void clearTmp() { - for (long[] dataArray : tmpTimestamps) { - PrimitiveArrayManager.release(dataArray); - } - tmpTimestamps.clear(); - for (int[] dataArray : tmpIndices) { - PrimitiveArrayManager.release(dataArray); - } - tmpIndices.clear(); - } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BackIntTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BackIntTVList.java index 093af0d175b51..776e065a4ac98 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BackIntTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BackIntTVList.java @@ -19,74 +19,19 @@ package org.apache.iotdb.db.utils.datastructure; -import org.apache.iotdb.db.storageengine.rescon.memory.PrimitiveArrayManager; +public class BackIntTVList extends QuickIntTVList { + private final BackwardSort policy; -import org.apache.tsfile.enums.TSDataType; - -import java.util.ArrayList; -import java.util.List; - -import static org.apache.iotdb.db.storageengine.rescon.memory.PrimitiveArrayManager.ARRAY_SIZE; - -public class BackIntTVList extends QuickIntTVList implements BackwardSort { - - private final List tmpTimestamps = new ArrayList<>(); - private final List tmpIndices = new ArrayList<>(); - private int tmpLength = 0; + BackIntTVList() { + policy = new BackwardSort(this); + } @Override public void sort() { if (!sorted) { - backwardSort(timestamps, rowCount); - clearTmp(); + policy.backwardSort(timestamps, rowCount); + policy.clearTmp(); } sorted = true; } - - @Override - public void setFromTmp(int src, int dest) { - set( - dest, - tmpTimestamps.get(src / ARRAY_SIZE)[src % ARRAY_SIZE], - tmpIndices.get(src / ARRAY_SIZE)[src % ARRAY_SIZE]); - } - - @Override - public void setToTmp(int src, int dest) { - tmpTimestamps.get(dest / ARRAY_SIZE)[dest % ARRAY_SIZE] = getTime(src); - tmpIndices.get(dest / ARRAY_SIZE)[dest % ARRAY_SIZE] = getValueIndex(src); - } - - @Override - public void backward_set(int src, int dest) { - set(src, dest); - } - - @Override - public int compareTmp(int idx, int tmpIdx) { - long t1 = getTime(idx); - long t2 = tmpTimestamps.get(tmpIdx / ARRAY_SIZE)[tmpIdx % ARRAY_SIZE]; - return Long.compare(t1, t2); - } - - @Override - public void checkTmpLength(int len) { - while (len > tmpLength) { - tmpTimestamps.add((long[]) getPrimitiveArraysByType(TSDataType.INT64)); - tmpIndices.add((int[]) getPrimitiveArraysByType(TSDataType.INT32)); - tmpLength += ARRAY_SIZE; - } - } - - @Override - public void clearTmp() { - for (long[] dataArray : tmpTimestamps) { - PrimitiveArrayManager.release(dataArray); - } - tmpTimestamps.clear(); - for (int[] dataArray : tmpIndices) { - PrimitiveArrayManager.release(dataArray); - } - tmpIndices.clear(); - } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BackLongTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BackLongTVList.java index cc28e8f09e641..b55e7c15df841 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BackLongTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BackLongTVList.java @@ -18,73 +18,19 @@ */ package org.apache.iotdb.db.utils.datastructure; -import org.apache.iotdb.db.storageengine.rescon.memory.PrimitiveArrayManager; +public class BackLongTVList extends QuickLongTVList { + private final BackwardSort policy; -import org.apache.tsfile.enums.TSDataType; - -import java.util.ArrayList; -import java.util.List; - -import static org.apache.iotdb.db.storageengine.rescon.memory.PrimitiveArrayManager.ARRAY_SIZE; - -public class BackLongTVList extends QuickLongTVList implements BackwardSort { - private final List tmpTimestamps = new ArrayList<>(); - private final List tmpIndices = new ArrayList<>(); - private int tmpLength = 0; + BackLongTVList() { + policy = new BackwardSort(this); + } @Override public void sort() { if (!sorted) { - backwardSort(timestamps, rowCount); - clearTmp(); + policy.backwardSort(timestamps, rowCount); + policy.clearTmp(); } sorted = true; } - - @Override - public void setFromTmp(int src, int dest) { - set( - dest, - tmpTimestamps.get(src / ARRAY_SIZE)[src % ARRAY_SIZE], - tmpIndices.get(src / ARRAY_SIZE)[src % ARRAY_SIZE]); - } - - @Override - public void setToTmp(int src, int dest) { - tmpTimestamps.get(dest / ARRAY_SIZE)[dest % ARRAY_SIZE] = getTime(src); - tmpIndices.get(dest / ARRAY_SIZE)[dest % ARRAY_SIZE] = getValueIndex(src); - } - - @Override - public void backward_set(int src, int dest) { - set(src, dest); - } - - @Override - public int compareTmp(int idx, int tmpIdx) { - long t1 = getTime(idx); - long t2 = tmpTimestamps.get(tmpIdx / ARRAY_SIZE)[tmpIdx % ARRAY_SIZE]; - return Long.compare(t1, t2); - } - - @Override - public void checkTmpLength(int len) { - while (len > tmpLength) { - tmpTimestamps.add((long[]) getPrimitiveArraysByType(TSDataType.INT64)); - tmpIndices.add((int[]) getPrimitiveArraysByType(TSDataType.INT32)); - tmpLength += ARRAY_SIZE; - } - } - - @Override - public void clearTmp() { - for (long[] dataArray : tmpTimestamps) { - PrimitiveArrayManager.release(dataArray); - } - tmpTimestamps.clear(); - for (int[] dataArray : tmpIndices) { - PrimitiveArrayManager.release(dataArray); - } - tmpIndices.clear(); - } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BackwardSort.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BackwardSort.java index 6271a31734e67..4800032c7a06a 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BackwardSort.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BackwardSort.java @@ -19,27 +19,68 @@ package org.apache.iotdb.db.utils.datastructure; +import org.apache.iotdb.db.storageengine.rescon.memory.PrimitiveArrayManager; + +import org.apache.tsfile.enums.TSDataType; + +import java.util.ArrayList; import java.util.List; import static org.apache.iotdb.db.storageengine.rescon.memory.PrimitiveArrayManager.ARRAY_SIZE; -public interface BackwardSort extends QuickSort { +public class BackwardSort extends QuickSort { + public static double INVERSION_RATIOS_THRESHOLD = 0.004; - double INVERSION_RATIOS_THRESHOLD = 0.004; + private final List tmpTimestamps = new ArrayList<>(); + private final List tmpIndices = new ArrayList<>(); + private int tmpLength = 0; - void setFromTmp(int src, int dest); + public BackwardSort(TVList tvList) { + super(tvList); + } - void setToTmp(int src, int dest); + public void setFromTmp(int src, int dest) { + tvList.set( + dest, + tmpTimestamps.get(src / ARRAY_SIZE)[src % ARRAY_SIZE], + tmpIndices.get(src / ARRAY_SIZE)[src % ARRAY_SIZE]); + } - void backward_set(int src, int dest); + public void setToTmp(int src, int dest) { + tmpTimestamps.get(dest / ARRAY_SIZE)[dest % ARRAY_SIZE] = tvList.getTime(src); + tmpIndices.get(dest / ARRAY_SIZE)[dest % ARRAY_SIZE] = tvList.getValueIndex(src); + } - int compareTmp(int idx, int tmpIdx); + public void backward_set(int src, int dest) { + tvList.set(src, dest); + } - void checkTmpLength(int len); + public int compareTmp(int idx, int tmpIdx) { + long t1 = tvList.getTime(idx); + long t2 = tmpTimestamps.get(tmpIdx / ARRAY_SIZE)[tmpIdx % ARRAY_SIZE]; + return Long.compare(t1, t2); + } - void clearTmp(); + public void checkTmpLength(int len) { + while (len > tmpLength) { + tmpTimestamps.add((long[]) tvList.getPrimitiveArraysByType(TSDataType.INT64)); + tmpIndices.add((int[]) tvList.getPrimitiveArraysByType(TSDataType.INT32)); + tmpLength += ARRAY_SIZE; + } + } + + public void clearTmp() { + for (long[] dataArray : tmpTimestamps) { + PrimitiveArrayManager.release(dataArray); + } + tmpTimestamps.clear(); + for (int[] dataArray : tmpIndices) { + PrimitiveArrayManager.release(dataArray); + } + tmpIndices.clear(); + } - default void backwardSort(List timestamps, int rowCount) { + public void backwardSort(List timestamps, int rowCount) { int block_size = setBlockLength(timestamps, 1); // System.out.printf("rowCount=%d, block_size=%d\n",rowCount, block_size); int B = rowCount / block_size + 1; @@ -60,7 +101,7 @@ default void backwardSort(List timestamps, int rowCount) { * @param step * @return */ - default int setBlockLength(List timestamps, int step) { + public int setBlockLength(List timestamps, int step) { double overlap = 0; long last_time = timestamps.get(0)[0]; int i = step, blocks = 0; @@ -90,7 +131,7 @@ default int setBlockLength(List timestamps, int step) { * @param hi * @param rowCount */ - default void backwardMergeBlocks(int lo, int hi, int rowCount) { + public void backwardMergeBlocks(int lo, int hi, int rowCount) { int overlapIdx = hi + 1; while (overlapIdx < rowCount && compare(hi, overlapIdx) == 1) { overlapIdx++; @@ -131,7 +172,7 @@ default void backwardMergeBlocks(int lo, int hi, int rowCount) { * @param lo * @param hi */ - default void sortBlock(int lo, int hi) { + public void sortBlock(int lo, int hi) { qsort(lo, hi); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/QuickAlignedTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/QuickAlignedTVList.java index 4833adc3c20ff..ff11bdcd4a123 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/QuickAlignedTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/QuickAlignedTVList.java @@ -22,40 +22,19 @@ import java.util.List; -public class QuickAlignedTVList extends AlignedTVList implements QuickSort { +public class QuickAlignedTVList extends AlignedTVList { + private final QuickSort policy; + QuickAlignedTVList(List types) { super(types); + policy = new QuickSort(this); } @Override public void sort() { if (!sorted) { - qsort(0, rowCount - 1); + policy.qsort(0, rowCount - 1); } sorted = true; } - - @Override - protected void set(int src, int dest) { - long srcT = getTime(src); - int srcV = getValueIndex(src); - set(dest, srcT, srcV); - } - - @Override - public int compare(int idx1, int idx2) { - long t1 = getTime(idx1); - long t2 = getTime(idx2); - return Long.compare(t1, t2); - } - - @Override - public void swap(int p, int q) { - int valueP = getValueIndex(p); - long timeP = getTime(p); - int valueQ = getValueIndex(q); - long timeQ = getTime(q); - set(p, timeQ, valueQ); - set(q, timeP, valueP); - } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/QuickBinaryTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/QuickBinaryTVList.java index 7a524a1489880..f5bb5e65e34b6 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/QuickBinaryTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/QuickBinaryTVList.java @@ -18,37 +18,18 @@ */ package org.apache.iotdb.db.utils.datastructure; -public class QuickBinaryTVList extends BinaryTVList implements QuickSort { +public class QuickBinaryTVList extends BinaryTVList { + private final QuickSort policy; - @Override - public int compare(int idx1, int idx2) { - long t1 = getTime(idx1); - long t2 = getTime(idx2); - return Long.compare(t1, t2); - } - - @Override - public void swap(int p, int q) { - int valueP = getValueIndex(p); - long timeP = getTime(p); - int valueQ = getValueIndex(q); - long timeQ = getTime(q); - set(p, timeQ, valueQ); - set(q, timeP, valueP); + QuickBinaryTVList() { + policy = new QuickSort(this); } @Override public void sort() { if (!sorted) { - qsort(0, rowCount - 1); + policy.qsort(0, rowCount - 1); } sorted = true; } - - @Override - protected void set(int src, int dest) { - long srcT = getTime(src); - int srcV = getValueIndex(src); - set(dest, srcT, srcV); - } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/QuickBooleanTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/QuickBooleanTVList.java index cbbf332fa996f..0278378ea8663 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/QuickBooleanTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/QuickBooleanTVList.java @@ -18,36 +18,18 @@ */ package org.apache.iotdb.db.utils.datastructure; -public class QuickBooleanTVList extends BooleanTVList implements QuickSort { - @Override - public int compare(int idx1, int idx2) { - long t1 = getTime(idx1); - long t2 = getTime(idx2); - return Long.compare(t1, t2); - } +public class QuickBooleanTVList extends BooleanTVList { + private final QuickSort policy; - @Override - public void swap(int p, int q) { - int valueP = getValueIndex(p); - long timeP = getTime(p); - int valueQ = getValueIndex(q); - long timeQ = getTime(q); - set(p, timeQ, valueQ); - set(q, timeP, valueP); + QuickBooleanTVList() { + policy = new QuickSort(this); } @Override public void sort() { if (!sorted) { - qsort(0, rowCount - 1); + policy.qsort(0, rowCount - 1); } sorted = true; } - - @Override - protected void set(int src, int dest) { - long srcT = getTime(src); - int srcV = getValueIndex(src); - set(dest, srcT, srcV); - } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/QuickDoubleTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/QuickDoubleTVList.java index 2356d68f8a360..7f0f43d0b2d97 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/QuickDoubleTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/QuickDoubleTVList.java @@ -18,36 +18,18 @@ */ package org.apache.iotdb.db.utils.datastructure; -public class QuickDoubleTVList extends DoubleTVList implements QuickSort { - @Override - public int compare(int idx1, int idx2) { - long t1 = getTime(idx1); - long t2 = getTime(idx2); - return Long.compare(t1, t2); - } +public class QuickDoubleTVList extends DoubleTVList { + private final QuickSort policy; - @Override - public void swap(int p, int q) { - int valueP = getValueIndex(p); - long timeP = getTime(p); - int valueQ = getValueIndex(q); - long timeQ = getTime(q); - set(p, timeQ, valueQ); - set(q, timeP, valueP); + QuickDoubleTVList() { + policy = new QuickSort(this); } @Override public void sort() { if (!sorted) { - qsort(0, rowCount - 1); + policy.qsort(0, rowCount - 1); } sorted = true; } - - @Override - protected void set(int src, int dest) { - long srcT = getTime(src); - int srcV = getValueIndex(src); - set(dest, srcT, srcV); - } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/QuickFloatTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/QuickFloatTVList.java index a67003664bb91..896135dae8c1b 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/QuickFloatTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/QuickFloatTVList.java @@ -18,36 +18,18 @@ */ package org.apache.iotdb.db.utils.datastructure; -public class QuickFloatTVList extends FloatTVList implements QuickSort { - @Override - public int compare(int idx1, int idx2) { - long t1 = getTime(idx1); - long t2 = getTime(idx2); - return Long.compare(t1, t2); - } +public class QuickFloatTVList extends FloatTVList { + private final QuickSort policy; - @Override - public void swap(int p, int q) { - int valueP = getValueIndex(p); - long timeP = getTime(p); - int valueQ = getValueIndex(q); - long timeQ = getTime(q); - set(p, timeQ, valueQ); - set(q, timeP, valueP); + QuickFloatTVList() { + policy = new QuickSort(this); } @Override public void sort() { if (!sorted) { - qsort(0, rowCount - 1); + policy.qsort(0, rowCount - 1); } sorted = true; } - - @Override - protected void set(int src, int dest) { - long srcT = getTime(src); - int srcV = getValueIndex(src); - set(dest, srcT, srcV); - } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/QuickIntTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/QuickIntTVList.java index c898fe99dfa15..afca18e18318a 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/QuickIntTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/QuickIntTVList.java @@ -18,36 +18,18 @@ */ package org.apache.iotdb.db.utils.datastructure; -public class QuickIntTVList extends IntTVList implements QuickSort { +public class QuickIntTVList extends IntTVList { + private final QuickSort policy; + + QuickIntTVList() { + policy = new QuickSort(this); + } + @Override public void sort() { if (!sorted) { - qsort(0, rowCount - 1); + policy.qsort(0, rowCount - 1); } sorted = true; } - - @Override - public void swap(int p, int q) { - int valueP = getValueIndex(p); - long timeP = getTime(p); - int valueQ = getValueIndex(q); - long timeQ = getTime(q); - set(p, timeQ, valueQ); - set(q, timeP, valueP); - } - - @Override - protected void set(int src, int dest) { - long srcT = getTime(src); - int srcV = getValueIndex(src); - set(dest, srcT, srcV); - } - - @Override - public int compare(int idx1, int idx2) { - long t1 = getTime(idx1); - long t2 = getTime(idx2); - return Long.compare(t1, t2); - } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/QuickLongTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/QuickLongTVList.java index 423ce2bf19dae..dfe5da1f7ad34 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/QuickLongTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/QuickLongTVList.java @@ -18,36 +18,18 @@ */ package org.apache.iotdb.db.utils.datastructure; -public class QuickLongTVList extends LongTVList implements QuickSort { - @Override - public int compare(int idx1, int idx2) { - long t1 = getTime(idx1); - long t2 = getTime(idx2); - return Long.compare(t1, t2); - } +public class QuickLongTVList extends LongTVList { + private final QuickSort policy; - @Override - public void swap(int p, int q) { - int valueP = getValueIndex(p); - long timeP = getTime(p); - int valueQ = getValueIndex(q); - long timeQ = getTime(q); - set(p, timeQ, valueQ); - set(q, timeP, valueP); + QuickLongTVList() { + policy = new QuickSort(this); } @Override public void sort() { if (!sorted) { - qsort(0, rowCount - 1); + policy.qsort(0, rowCount - 1); } sorted = true; } - - @Override - protected void set(int src, int dest) { - long srcT = getTime(src); - int srcV = getValueIndex(src); - set(dest, srcT, srcV); - } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/QuickSort.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/QuickSort.java index 8229781aa053c..655ff2bec99dd 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/QuickSort.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/QuickSort.java @@ -16,15 +16,33 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.iotdb.db.utils.datastructure; -public interface QuickSort { +public class QuickSort { + protected final TVList tvList; + + public QuickSort(TVList tvList) { + this.tvList = tvList; + } + /** compare the timestamps in idx1 and idx2 */ - int compare(int idx1, int idx2); + public int compare(int idx1, int idx2) { + long t1 = tvList.getTime(idx1); + long t2 = tvList.getTime(idx2); + return Long.compare(t1, t2); + } - void swap(int p, int q); + public void swap(int p, int q) { + int valueIndexP = tvList.getValueIndex(p); + long timeP = tvList.getTime(p); + int valueIndexQ = tvList.getValueIndex(q); + long timeQ = tvList.getTime(q); + tvList.set(p, timeQ, valueIndexQ); + tvList.set(q, timeP, valueIndexP); + } - default int partition(int lo, int hi) { + public int partition(int lo, int hi) { // Choose the middle of the array as pivot. // In time series, usually the middle element is of middle range int pIndex = (lo + hi) / 2; @@ -58,7 +76,7 @@ default int partition(int lo, int hi) { // } // } - default void qsort(int lo, int hi) { + public void qsort(int lo, int hi) { if (lo < hi) { // TODO: use insertion sort in smaller array // if(hi - lo <= 32) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java index eab94a7c7d1c3..f24f8d671708d 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java @@ -192,14 +192,20 @@ public long getTime(int index) { return timestamps.get(arrayIndex)[elementIndex]; } - protected void set(int index, long timestamp, int value) { + protected void set(int src, int dest) { + long srcT = getTime(src); + int srcV = getValueIndex(src); + set(dest, srcT, srcV); + } + + protected void set(int index, long timestamp, int valueIndex) { if (index >= rowCount) { throw new ArrayIndexOutOfBoundsException(index); } int arrayIndex = index / ARRAY_SIZE; int elementIndex = index % ARRAY_SIZE; timestamps.get(arrayIndex)[elementIndex] = timestamp; - indices.get(arrayIndex)[elementIndex] = value; + indices.get(arrayIndex)[elementIndex] = valueIndex; } protected int[] cloneIndex(int[] array) { @@ -394,8 +400,6 @@ public long getVersion() { return version; } - protected abstract void set(int src, int dest); - protected abstract void expandValues(); @Override diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TimAlignedTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TimAlignedTVList.java index 975eab5cbc806..558291dc02079 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TimAlignedTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TimAlignedTVList.java @@ -18,121 +18,33 @@ */ package org.apache.iotdb.db.utils.datastructure; -import org.apache.iotdb.db.storageengine.rescon.memory.PrimitiveArrayManager; - import org.apache.tsfile.enums.TSDataType; import java.util.List; -import static org.apache.iotdb.db.storageengine.rescon.memory.PrimitiveArrayManager.ARRAY_SIZE; - -public class TimAlignedTVList extends AlignedTVList implements TimSort { - - private long[][] sortedTimestamps; - private long pivotTime; - - private int[][] sortedIndices; - private int pivotIndex; +public class TimAlignedTVList extends AlignedTVList { + private final TimSort policy; TimAlignedTVList(List types) { super(types); + policy = new TimSort(this); } @Override public void sort() { - if (sortedTimestamps == null - || sortedTimestamps.length < PrimitiveArrayManager.getArrayRowCount(rowCount)) { - sortedTimestamps = - (long[][]) PrimitiveArrayManager.createDataListsByType(TSDataType.INT64, rowCount); - } - if (sortedIndices == null - || sortedIndices.length < PrimitiveArrayManager.getArrayRowCount(rowCount)) { - sortedIndices = - (int[][]) PrimitiveArrayManager.createDataListsByType(TSDataType.INT32, rowCount); - } + policy.checkSortedTimestampsAndIndices(); if (!sorted) { - sort(0, rowCount); + policy.sort(0, rowCount); } - clearSortedValue(); - clearSortedTime(); + policy.clearSortedValue(); + policy.clearSortedTime(); sorted = true; } - @Override - public void tim_set(int src, int dest) { - set(src, dest); - } - - @Override - protected void set(int src, int dest) { - long srcT = getTime(src); - int srcV = getValueIndex(src); - set(dest, srcT, srcV); - } - - @Override - public void setFromSorted(int src, int dest) { - set( - dest, - sortedTimestamps[src / ARRAY_SIZE][src % ARRAY_SIZE], - sortedIndices[src / ARRAY_SIZE][src % ARRAY_SIZE]); - } - - @Override - public void setToSorted(int src, int dest) { - sortedTimestamps[dest / ARRAY_SIZE][dest % ARRAY_SIZE] = getTime(src); - sortedIndices[dest / ARRAY_SIZE][dest % ARRAY_SIZE] = getValueIndex(src); - } - - @Override - public void setPivotTo(int pos) { - set(pos, pivotTime, pivotIndex); - } - - @Override - public void saveAsPivot(int pos) { - pivotTime = getTime(pos); - pivotIndex = getValueIndex(pos); - } - - @Override - public void clearSortedTime() { - if (sortedTimestamps != null) { - sortedTimestamps = null; - } - } - - @Override - public void clearSortedValue() { - if (sortedIndices != null) { - sortedIndices = 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 = getValueIndex(lo); - long hiT = getTime(hi); - int hiV = getValueIndex(hi); - set(lo++, hiT, hiV); - set(hi--, loT, loV); - } - } - @Override public void clear() { super.clear(); - clearSortedTime(); - clearSortedValue(); + policy.clearSortedTime(); + policy.clearSortedValue(); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TimBinaryTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TimBinaryTVList.java index f0a2b5c2c45d8..379b9248b1b75 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TimBinaryTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TimBinaryTVList.java @@ -18,112 +18,28 @@ */ package org.apache.iotdb.db.utils.datastructure; -import org.apache.iotdb.db.storageengine.rescon.memory.PrimitiveArrayManager; +public class TimBinaryTVList extends BinaryTVList { + private final TimSort policy; -import org.apache.tsfile.enums.TSDataType; - -import static org.apache.iotdb.db.storageengine.rescon.memory.PrimitiveArrayManager.ARRAY_SIZE; - -public class TimBinaryTVList extends BinaryTVList implements TimSort { - private long[][] sortedTimestamps; - private int[][] sortedIndices; - - private long pivotTime; - private int pivotIndex; + TimBinaryTVList() { + policy = new TimSort(this); + } @Override public void sort() { - if (sortedTimestamps == null - || sortedTimestamps.length < PrimitiveArrayManager.getArrayRowCount(rowCount)) { - sortedTimestamps = - (long[][]) PrimitiveArrayManager.createDataListsByType(TSDataType.INT64, rowCount); - } - if (sortedIndices == null - || sortedIndices.length < PrimitiveArrayManager.getArrayRowCount(rowCount)) { - sortedIndices = - (int[][]) PrimitiveArrayManager.createDataListsByType(TSDataType.INT32, rowCount); + policy.checkSortedTimestampsAndIndices(); + if (!sorted) { + policy.sort(0, rowCount); } - sort(0, rowCount); - clearSortedValue(); - clearSortedTime(); + policy.clearSortedValue(); + policy.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 = getValueIndex(src); - set(dest, srcT, srcV); - } - - @Override - public void setToSorted(int src, int dest) { - sortedTimestamps[dest / ARRAY_SIZE][dest % ARRAY_SIZE] = getTime(src); - sortedIndices[dest / ARRAY_SIZE][dest % ARRAY_SIZE] = getValueIndex(src); - } - - @Override - public void saveAsPivot(int pos) { - pivotTime = getTime(pos); - pivotIndex = getValueIndex(pos); - } - - @Override - public void setFromSorted(int src, int dest) { - set( - dest, - sortedTimestamps[src / ARRAY_SIZE][src % ARRAY_SIZE], - sortedIndices[src / ARRAY_SIZE][src % ARRAY_SIZE]); - } - - @Override - public void setPivotTo(int pos) { - set(pos, pivotTime, pivotIndex); - } - - @Override - public void clearSortedTime() { - if (sortedTimestamps != null) { - sortedTimestamps = null; - } - } - - @Override - public void clearSortedValue() { - if (sortedIndices != null) { - sortedIndices = 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 = getValueIndex(lo); - long hiT = getTime(hi); - int hiV = getValueIndex(hi); - set(lo++, hiT, hiV); - set(hi--, loT, loV); - } - } - @Override public void clear() { super.clear(); - clearSortedTime(); - clearSortedValue(); + policy.clearSortedTime(); + policy.clearSortedValue(); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TimBooleanTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TimBooleanTVList.java index 39de1435f04f7..1f45779fc3485 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TimBooleanTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TimBooleanTVList.java @@ -18,114 +18,28 @@ */ package org.apache.iotdb.db.utils.datastructure; -import org.apache.iotdb.db.storageengine.rescon.memory.PrimitiveArrayManager; +public class TimBooleanTVList extends BooleanTVList { + private final TimSort policy; -import org.apache.tsfile.enums.TSDataType; - -import static org.apache.iotdb.db.storageengine.rescon.memory.PrimitiveArrayManager.ARRAY_SIZE; - -public class TimBooleanTVList extends BooleanTVList implements TimSort { - private long[][] sortedTimestamps; - private int[][] sortedIndices; - - private long pivotTime; - private int pivotIndex; + TimBooleanTVList() { + policy = new TimSort(this); + } @Override public void sort() { - if (sortedTimestamps == null - || sortedTimestamps.length < PrimitiveArrayManager.getArrayRowCount(rowCount)) { - sortedTimestamps = - (long[][]) PrimitiveArrayManager.createDataListsByType(TSDataType.INT64, rowCount); - } - if (sortedIndices == null - || sortedIndices.length < PrimitiveArrayManager.getArrayRowCount(rowCount)) { - sortedIndices = - (int[][]) PrimitiveArrayManager.createDataListsByType(TSDataType.INT32, rowCount); - } + policy.checkSortedTimestampsAndIndices(); if (!sorted) { - sort(0, rowCount); + policy.sort(0, rowCount); } - clearSortedValue(); - clearSortedTime(); + policy.clearSortedValue(); + policy.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 = getValueIndex(src); - set(dest, srcT, srcV); - } - - @Override - public void setToSorted(int src, int dest) { - sortedTimestamps[dest / ARRAY_SIZE][dest % ARRAY_SIZE] = getTime(src); - sortedIndices[dest / ARRAY_SIZE][dest % ARRAY_SIZE] = getValueIndex(src); - } - - @Override - public void saveAsPivot(int pos) { - pivotTime = getTime(pos); - pivotIndex = getValueIndex(pos); - } - - @Override - public void setFromSorted(int src, int dest) { - set( - dest, - sortedTimestamps[src / ARRAY_SIZE][src % ARRAY_SIZE], - sortedIndices[src / ARRAY_SIZE][src % ARRAY_SIZE]); - } - - @Override - public void setPivotTo(int pos) { - set(pos, pivotTime, pivotIndex); - } - - @Override - public void clearSortedTime() { - if (sortedTimestamps != null) { - sortedTimestamps = null; - } - } - - @Override - public void clearSortedValue() { - if (sortedIndices != null) { - sortedIndices = 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 = getValueIndex(lo); - long hiT = getTime(hi); - int hiV = getValueIndex(hi); - set(lo++, hiT, hiV); - set(hi--, loT, loV); - } - } - @Override public void clear() { super.clear(); - clearSortedTime(); - clearSortedValue(); + policy.clearSortedTime(); + policy.clearSortedValue(); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TimDoubleTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TimDoubleTVList.java index f62bfa94cefd8..74a5db378facf 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TimDoubleTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TimDoubleTVList.java @@ -18,114 +18,28 @@ */ package org.apache.iotdb.db.utils.datastructure; -import org.apache.iotdb.db.storageengine.rescon.memory.PrimitiveArrayManager; +public class TimDoubleTVList extends DoubleTVList { + private final TimSort policy; -import org.apache.tsfile.enums.TSDataType; - -import static org.apache.iotdb.db.storageengine.rescon.memory.PrimitiveArrayManager.ARRAY_SIZE; - -public class TimDoubleTVList extends DoubleTVList implements TimSort { - private long[][] sortedTimestamps; - private int[][] sortedIndices; - - private long pivotTime; - private int pivotIndex; + TimDoubleTVList() { + policy = new TimSort(this); + } @Override public void sort() { - if (sortedTimestamps == null - || sortedTimestamps.length < PrimitiveArrayManager.getArrayRowCount(rowCount)) { - sortedTimestamps = - (long[][]) PrimitiveArrayManager.createDataListsByType(TSDataType.INT64, rowCount); - } - if (sortedIndices == null - || sortedIndices.length < PrimitiveArrayManager.getArrayRowCount(rowCount)) { - sortedIndices = - (int[][]) PrimitiveArrayManager.createDataListsByType(TSDataType.INT32, rowCount); - } + policy.checkSortedTimestampsAndIndices(); if (!sorted) { - sort(0, rowCount); + policy.sort(0, rowCount); } - clearSortedValue(); - clearSortedTime(); + policy.clearSortedValue(); + policy.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 = getValueIndex(src); - set(dest, srcT, srcV); - } - - @Override - public void setToSorted(int src, int dest) { - sortedTimestamps[dest / ARRAY_SIZE][dest % ARRAY_SIZE] = getTime(src); - sortedIndices[dest / ARRAY_SIZE][dest % ARRAY_SIZE] = getValueIndex(src); - } - - @Override - public void saveAsPivot(int pos) { - pivotTime = getTime(pos); - pivotIndex = getValueIndex(pos); - } - - @Override - public void setFromSorted(int src, int dest) { - set( - dest, - sortedTimestamps[src / ARRAY_SIZE][src % ARRAY_SIZE], - sortedIndices[src / ARRAY_SIZE][src % ARRAY_SIZE]); - } - - @Override - public void setPivotTo(int pos) { - set(pos, pivotTime, pivotIndex); - } - - @Override - public void clearSortedTime() { - if (sortedTimestamps != null) { - sortedTimestamps = null; - } - } - - @Override - public void clearSortedValue() { - if (sortedIndices != null) { - sortedIndices = 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 = getValueIndex(lo); - long hiT = getTime(hi); - int hiV = getValueIndex(hi); - set(lo++, hiT, hiV); - set(hi--, loT, loV); - } - } - @Override public void clear() { super.clear(); - clearSortedTime(); - clearSortedValue(); + policy.clearSortedTime(); + policy.clearSortedValue(); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TimFloatTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TimFloatTVList.java index f5b0d03846358..6522fd61159e2 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TimFloatTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TimFloatTVList.java @@ -18,114 +18,28 @@ */ package org.apache.iotdb.db.utils.datastructure; -import org.apache.iotdb.db.storageengine.rescon.memory.PrimitiveArrayManager; +public class TimFloatTVList extends FloatTVList { + private final TimSort policy; -import org.apache.tsfile.enums.TSDataType; - -import static org.apache.iotdb.db.storageengine.rescon.memory.PrimitiveArrayManager.ARRAY_SIZE; - -public class TimFloatTVList extends FloatTVList implements TimSort { - private long[][] sortedTimestamps; - private int[][] sortedIndices; - - private long pivotTime; - private int pivotIndex; + TimFloatTVList() { + policy = new TimSort(this); + } @Override public void sort() { - if (sortedTimestamps == null - || sortedTimestamps.length < PrimitiveArrayManager.getArrayRowCount(rowCount)) { - sortedTimestamps = - (long[][]) PrimitiveArrayManager.createDataListsByType(TSDataType.INT64, rowCount); - } - if (sortedIndices == null - || sortedIndices.length < PrimitiveArrayManager.getArrayRowCount(rowCount)) { - sortedIndices = - (int[][]) PrimitiveArrayManager.createDataListsByType(TSDataType.INT32, rowCount); - } + policy.checkSortedTimestampsAndIndices(); if (!sorted) { - sort(0, rowCount); + policy.sort(0, rowCount); } - clearSortedValue(); - clearSortedTime(); + policy.clearSortedValue(); + policy.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 = getValueIndex(src); - set(dest, srcT, srcV); - } - - @Override - public void setToSorted(int src, int dest) { - sortedTimestamps[dest / ARRAY_SIZE][dest % ARRAY_SIZE] = getTime(src); - sortedIndices[dest / ARRAY_SIZE][dest % ARRAY_SIZE] = getValueIndex(src); - } - - @Override - public void saveAsPivot(int pos) { - pivotTime = getTime(pos); - pivotIndex = getValueIndex(pos); - } - - @Override - public void setFromSorted(int src, int dest) { - set( - dest, - sortedTimestamps[src / ARRAY_SIZE][src % ARRAY_SIZE], - sortedIndices[src / ARRAY_SIZE][src % ARRAY_SIZE]); - } - - @Override - public void setPivotTo(int pos) { - set(pos, pivotTime, pivotIndex); - } - - @Override - public void clearSortedTime() { - if (sortedTimestamps != null) { - sortedTimestamps = null; - } - } - - @Override - public void clearSortedValue() { - if (sortedIndices != null) { - sortedIndices = 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 = getValueIndex(lo); - long hiT = getTime(hi); - int hiV = getValueIndex(hi); - set(lo++, hiT, hiV); - set(hi--, loT, loV); - } - } - @Override public void clear() { super.clear(); - clearSortedTime(); - clearSortedValue(); + policy.clearSortedTime(); + policy.clearSortedValue(); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TimIntTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TimIntTVList.java index 421fbe2916d20..7bcd7073ee511 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TimIntTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TimIntTVList.java @@ -18,114 +18,28 @@ */ package org.apache.iotdb.db.utils.datastructure; -import org.apache.iotdb.db.storageengine.rescon.memory.PrimitiveArrayManager; +public class TimIntTVList extends IntTVList { + private final TimSort policy; -import org.apache.tsfile.enums.TSDataType; - -import static org.apache.iotdb.db.storageengine.rescon.memory.PrimitiveArrayManager.ARRAY_SIZE; - -public class TimIntTVList extends IntTVList implements TimSort { - private long[][] sortedTimestamps; - private int[][] sortedIndices; - - private int pivotIndex; - private long pivotTime; + TimIntTVList() { + policy = new TimSort(this); + } @Override public void sort() { - if (sortedTimestamps == null - || sortedTimestamps.length < PrimitiveArrayManager.getArrayRowCount(rowCount)) { - sortedTimestamps = - (long[][]) PrimitiveArrayManager.createDataListsByType(TSDataType.INT64, rowCount); - } - if (sortedIndices == null - || sortedIndices.length < PrimitiveArrayManager.getArrayRowCount(rowCount)) { - sortedIndices = - (int[][]) PrimitiveArrayManager.createDataListsByType(TSDataType.INT32, rowCount); - } + policy.checkSortedTimestampsAndIndices(); if (!sorted) { - sort(0, rowCount); + policy.sort(0, rowCount); } - clearSortedValue(); - clearSortedTime(); + policy.clearSortedValue(); + policy.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 = getValueIndex(src); - set(dest, srcT, srcV); - } - - @Override - public void setToSorted(int src, int dest) { - sortedTimestamps[dest / ARRAY_SIZE][dest % ARRAY_SIZE] = getTime(src); - sortedIndices[dest / ARRAY_SIZE][dest % ARRAY_SIZE] = getValueIndex(src); - } - - @Override - public void saveAsPivot(int pos) { - pivotTime = getTime(pos); - pivotIndex = getValueIndex(pos); - } - - @Override - public void setFromSorted(int src, int dest) { - set( - dest, - sortedTimestamps[src / ARRAY_SIZE][src % ARRAY_SIZE], - sortedIndices[src / ARRAY_SIZE][src % ARRAY_SIZE]); - } - - @Override - public void setPivotTo(int pos) { - set(pos, pivotTime, pivotIndex); - } - - @Override - public void clearSortedTime() { - if (sortedTimestamps != null) { - sortedTimestamps = null; - } - } - - @Override - public void clearSortedValue() { - if (sortedIndices != null) { - sortedIndices = 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 = getValueIndex(lo); - long hiT = getTime(hi); - int hiV = getValueIndex(hi); - set(lo++, hiT, hiV); - set(hi--, loT, loV); - } - } - @Override public void clear() { super.clear(); - clearSortedTime(); - clearSortedValue(); + policy.clearSortedTime(); + policy.clearSortedValue(); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TimLongTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TimLongTVList.java index 7c0f5640eeeb8..c42d7d8cc8c77 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TimLongTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TimLongTVList.java @@ -18,114 +18,28 @@ */ package org.apache.iotdb.db.utils.datastructure; -import org.apache.iotdb.db.storageengine.rescon.memory.PrimitiveArrayManager; +public class TimLongTVList extends LongTVList { + private final TimSort policy; -import org.apache.tsfile.enums.TSDataType; - -import static org.apache.iotdb.db.storageengine.rescon.memory.PrimitiveArrayManager.ARRAY_SIZE; - -public class TimLongTVList extends LongTVList implements TimSort { - private long[][] sortedTimestamps; - private int[][] sortedIndices; - - private long pivotTime; - private int pivotIndex; + TimLongTVList() { + policy = new TimSort(this); + } @Override public void sort() { - if (sortedTimestamps == null - || sortedTimestamps.length < PrimitiveArrayManager.getArrayRowCount(rowCount)) { - sortedTimestamps = - (long[][]) PrimitiveArrayManager.createDataListsByType(TSDataType.INT64, rowCount); - } - if (sortedIndices == null - || sortedIndices.length < PrimitiveArrayManager.getArrayRowCount(rowCount)) { - sortedIndices = - (int[][]) PrimitiveArrayManager.createDataListsByType(TSDataType.INT32, rowCount); - } + policy.checkSortedTimestampsAndIndices(); if (!sorted) { - sort(0, rowCount); + policy.sort(0, rowCount); } - clearSortedValue(); - clearSortedTime(); + policy.clearSortedValue(); + policy.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 = getValueIndex(src); - set(dest, srcT, srcV); - } - - @Override - public void setToSorted(int src, int dest) { - sortedTimestamps[dest / ARRAY_SIZE][dest % ARRAY_SIZE] = getTime(src); - sortedIndices[dest / ARRAY_SIZE][dest % ARRAY_SIZE] = getValueIndex(src); - } - - @Override - public void saveAsPivot(int pos) { - pivotTime = getTime(pos); - pivotIndex = getValueIndex(pos); - } - - @Override - public void setFromSorted(int src, int dest) { - set( - dest, - sortedTimestamps[src / ARRAY_SIZE][src % ARRAY_SIZE], - sortedIndices[src / ARRAY_SIZE][src % ARRAY_SIZE]); - } - - @Override - public void setPivotTo(int pos) { - set(pos, pivotTime, pivotIndex); - } - - @Override - public void clearSortedTime() { - if (sortedTimestamps != null) { - sortedTimestamps = null; - } - } - - @Override - public void clearSortedValue() { - if (sortedIndices != null) { - sortedIndices = 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 = getValueIndex(lo); - long hiT = getTime(hi); - int hiV = getValueIndex(hi); - set(lo++, hiT, hiV); - set(hi--, loT, loV); - } - } - @Override public void clear() { super.clear(); - clearSortedTime(); - clearSortedValue(); + policy.clearSortedTime(); + policy.clearSortedValue(); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TimSort.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TimSort.java index 63584429d4abe..82f8d5fdf16df 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TimSort.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TimSort.java @@ -16,46 +16,113 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.iotdb.db.utils.datastructure; -/** - * The interface refers to TimSort.java, and is used for sort the TVList Functions for tim_sort like - * merge, sort, binary_sort is implemented here as default, reuse code whenever possible. - */ -public interface TimSort { +import org.apache.iotdb.db.storageengine.rescon.memory.PrimitiveArrayManager; + +import org.apache.tsfile.enums.TSDataType; + +import static org.apache.iotdb.db.storageengine.rescon.memory.PrimitiveArrayManager.ARRAY_SIZE; + +public class TimSort { /** when array size <= 32, it's better to use binarysort. */ - int SMALL_ARRAY_LENGTH = 32; + public static int SMALL_ARRAY_LENGTH = 32; + + protected final TVList tvList; + + private long[][] sortedTimestamps; + private long pivotTime; + + private int[][] sortedIndices; + private int pivotIndex; + + public TimSort(TVList tvList) { + this.tvList = tvList; + } /** the same as the 'set' function in TVList, the reason is to avoid two equal functions. */ - void tim_set(int src, int dest); + public void tim_set(int src, int dest) { + tvList.set(src, dest); + } - void setFromSorted(int src, int dest); + public void setToSorted(int src, int dest) { + sortedTimestamps[dest / ARRAY_SIZE][dest % ARRAY_SIZE] = tvList.getTime(src); + sortedIndices[dest / ARRAY_SIZE][dest % ARRAY_SIZE] = tvList.getValueIndex(src); + } - void setToSorted(int src, int dest); + public void saveAsPivot(int pos) { + pivotTime = tvList.getTime(pos); + pivotIndex = tvList.getValueIndex(pos); + } - void setPivotTo(int pos); + public void setFromSorted(int src, int dest) { + tvList.set( + dest, + sortedTimestamps[src / ARRAY_SIZE][src % ARRAY_SIZE], + sortedIndices[src / ARRAY_SIZE][src % ARRAY_SIZE]); + } - void saveAsPivot(int pos); + public void setPivotTo(int pos) { + tvList.set(pos, pivotTime, pivotIndex); + } /** * The arrays for sorting are not including in write memory now, the memory usage is considered as * temporary memory. */ - void clearSortedTime(); + public void clearSortedTime() { + if (sortedTimestamps != null) { + sortedTimestamps = null; + } + } - void clearSortedValue(); + public void clearSortedValue() { + if (sortedIndices != null) { + sortedIndices = null; + } + } /** compare the timestamps in idx1 and idx2 */ - int compare(int idx1, int idx2); + public int compare(int idx1, int idx2) { + long t1 = tvList.getTime(idx1); + long t2 = tvList.getTime(idx2); + return Long.compare(t1, t2); + } /** From TimSort.java */ - void reverseRange(int lo, int hi); + public void reverseRange(int lo, int hi) { + hi--; + while (lo < hi) { + long loT = tvList.getTime(lo); + int loV = tvList.getValueIndex(lo); + long hiT = tvList.getTime(hi); + int hiV = tvList.getValueIndex(hi); + tvList.set(lo++, hiT, hiV); + tvList.set(hi--, loT, loV); + } + } + + public void checkSortedTimestampsAndIndices() { + if (sortedTimestamps == null + || sortedTimestamps.length < PrimitiveArrayManager.getArrayRowCount(tvList.rowCount())) { + sortedTimestamps = + (long[][]) + PrimitiveArrayManager.createDataListsByType(TSDataType.INT64, tvList.rowCount()); + } + if (sortedIndices == null + || sortedIndices.length < PrimitiveArrayManager.getArrayRowCount(tvList.rowCount())) { + sortedIndices = + (int[][]) + PrimitiveArrayManager.createDataListsByType(TSDataType.INT32, tvList.rowCount()); + } + } /** * the entrance of tim_sort; 1. array_size <= 32, use binary sort. 2. recursively invoke merge * sort. */ - default void sort(int lo, int hi) { + public void sort(int lo, int hi) { if (lo == hi) { return; } @@ -70,7 +137,7 @@ default void sort(int lo, int hi) { merge(lo, mid, hi); } - default int countRunAndMakeAscending(int lo, int hi) { + public int countRunAndMakeAscending(int lo, int hi) { assert lo < hi; int runHi = lo + 1; if (runHi == hi) { @@ -91,7 +158,7 @@ default int countRunAndMakeAscending(int lo, int hi) { return runHi - lo; } - default void binarySort(int lo, int hi, int start) { + public void binarySort(int lo, int hi, int start) { assert lo <= start && start <= hi; if (start == lo) { start++; @@ -137,7 +204,7 @@ default void binarySort(int lo, int hi, int start) { } /** merge arrays [lo, mid) [mid, hi] */ - default void merge(int lo, int mid, int hi) { + public void merge(int lo, int mid, int hi) { // end of sorting buffer int tmpIdx = 0; From a8beea5d77ade486082ef9df0ca02346ae554a61 Mon Sep 17 00:00:00 2001 From: shizy Date: Wed, 25 Dec 2024 19:28:57 +0800 Subject: [PATCH 26/54] refactor: synchronized tvlist method: sort, putXXX --- .../memtable/AlignedReadOnlyMemChunk.java | 4 +- .../memtable/AlignedWritableMemChunk.java | 16 +-- .../dataregion/memtable/ReadOnlyMemChunk.java | 4 +- .../dataregion/memtable/WritableMemChunk.java | 120 +++++++++--------- .../db/utils/datastructure/AlignedTVList.java | 4 +- .../datastructure/BackAlignedTVList.java | 3 +- .../utils/datastructure/BackBinaryTVList.java | 3 +- .../datastructure/BackBooleanTVList.java | 3 +- .../utils/datastructure/BackDoubleTVList.java | 3 +- .../utils/datastructure/BackFloatTVList.java | 3 +- .../db/utils/datastructure/BackIntTVList.java | 3 +- .../utils/datastructure/BackLongTVList.java | 3 +- .../db/utils/datastructure/BinaryTVList.java | 5 +- .../db/utils/datastructure/BooleanTVList.java | 5 +- .../db/utils/datastructure/DoubleTVList.java | 5 +- .../db/utils/datastructure/FloatTVList.java | 5 +- .../db/utils/datastructure/IntTVList.java | 4 +- .../db/utils/datastructure/LongTVList.java | 4 +- .../datastructure/QuickAlignedTVList.java | 3 +- .../datastructure/QuickBinaryTVList.java | 3 +- .../datastructure/QuickBooleanTVList.java | 3 +- .../datastructure/QuickDoubleTVList.java | 3 +- .../utils/datastructure/QuickFloatTVList.java | 3 +- .../utils/datastructure/QuickIntTVList.java | 3 +- .../utils/datastructure/QuickLongTVList.java | 3 +- .../iotdb/db/utils/datastructure/TVList.java | 5 - .../utils/datastructure/TimAlignedTVList.java | 3 +- .../utils/datastructure/TimBinaryTVList.java | 3 +- .../utils/datastructure/TimBooleanTVList.java | 3 +- .../utils/datastructure/TimDoubleTVList.java | 3 +- .../utils/datastructure/TimFloatTVList.java | 3 +- .../db/utils/datastructure/TimIntTVList.java | 3 +- .../db/utils/datastructure/TimLongTVList.java | 3 +- 33 files changed, 128 insertions(+), 116 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedReadOnlyMemChunk.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedReadOnlyMemChunk.java index 5c9ade1f0aa29..c8d2bfad39a57 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedReadOnlyMemChunk.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedReadOnlyMemChunk.java @@ -118,7 +118,7 @@ public void sortTvLists() { AlignedTVList alignedTvList = entry.getKey(); int queryRowCount = entry.getValue(); if (!alignedTvList.isSorted() && queryRowCount > alignedTvList.seqRowCount()) { - alignedTvList.safelySort(); + alignedTvList.sort(); } } } @@ -420,7 +420,7 @@ public IPointReader getPointReader() { AlignedTVList tvList = entry.getKey(); int queryLength = entry.getValue(); if (!tvList.isSorted() && queryLength > tvList.seqRowCount()) { - tvList.safelySort(); + tvList.sort(); } } TsBlock tsBlock = buildTsBlock(); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedWritableMemChunk.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedWritableMemChunk.java index 87388a876a668..b54ee8f37f2f9 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedWritableMemChunk.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedWritableMemChunk.java @@ -130,10 +130,8 @@ public boolean putBooleanWithFlushCheck(long t, boolean v) { @Override public boolean putAlignedValueWithFlushCheck(long t, Object[] v) { - synchronized (list) { - list.putAlignedValue(t, v); - return list.reachChunkSizeOrPointNumThreshold(); - } + list.putAlignedValue(t, v); + return list.reachChunkSizeOrPointNumThreshold(); } @Override @@ -171,10 +169,8 @@ public boolean putBooleansWithFlushCheck( @Override public boolean putAlignedValuesWithFlushCheck( long[] t, Object[] v, BitMap[] bitMaps, int start, int end, TSStatus[] results) { - synchronized (list) { - list.putAlignedValues(t, v, bitMaps, start, end, results); - return list.reachChunkSizeOrPointNumThreshold(); - } + list.putAlignedValues(t, v, bitMaps, start, end, results); + return list.reachChunkSizeOrPointNumThreshold(); } @Override @@ -195,7 +191,7 @@ protected void handoverAlignedTvList() { if (list.isSorted()) { sortedList.add(list); } else if (list.getQueryContextList().isEmpty()) { - list.safelySort(); + list.sort(); sortedList.add(list); } else { QueryContext firstQuery = list.getQueryContextList().get(0); @@ -384,7 +380,7 @@ private void sortTVList() { @Override public synchronized void sortTvListForFlush() { if (!list.isSorted()) { - list.safelySort(); + list.sort(); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/ReadOnlyMemChunk.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/ReadOnlyMemChunk.java index d16bea3d8192d..c65c007851cba 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/ReadOnlyMemChunk.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/ReadOnlyMemChunk.java @@ -132,7 +132,7 @@ public void sortTvLists() { TVList tvList = entry.getKey(); int queryRowCount = entry.getValue(); if (!tvList.isSorted() && queryRowCount > tvList.seqRowCount()) { - tvList.safelySort(); + tvList.sort(); } } } @@ -220,7 +220,7 @@ public IPointReader getPointReader() { TVList tvList = entry.getKey(); int queryLength = entry.getValue(); if (!tvList.isSorted() && queryLength > tvList.seqRowCount()) { - tvList.safelySort(); + tvList.sort(); } } TsBlock tsBlock = buildTsBlock(); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/WritableMemChunk.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/WritableMemChunk.java index 1e513085a8e06..19d51a7c9b193 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/WritableMemChunk.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/WritableMemChunk.java @@ -71,7 +71,7 @@ protected void handoverTvList() { if (list.isSorted()) { sortedList.add(list); } else if (list.getQueryContextList().isEmpty()) { - list.safelySort(); + list.sort(); sortedList.add(list); } else { QueryContext firstQuery = list.getQueryContextList().get(0); @@ -96,33 +96,31 @@ protected void handoverTvList() { @Override public boolean writeWithFlushCheck(long insertTime, Object objectValue) { boolean shouldFlush; - synchronized (list) { - switch (schema.getType()) { - case BOOLEAN: - shouldFlush = putBooleanWithFlushCheck(insertTime, (boolean) objectValue); - break; - case INT32: - case DATE: - shouldFlush = putIntWithFlushCheck(insertTime, (int) objectValue); - break; - case INT64: - case TIMESTAMP: - shouldFlush = putLongWithFlushCheck(insertTime, (long) objectValue); - break; - case FLOAT: - shouldFlush = putFloatWithFlushCheck(insertTime, (float) objectValue); - break; - case DOUBLE: - shouldFlush = putDoubleWithFlushCheck(insertTime, (double) objectValue); - break; - case TEXT: - case BLOB: - case STRING: - shouldFlush = putBinaryWithFlushCheck(insertTime, (Binary) objectValue); - break; - default: - throw new UnSupportedDataTypeException(UNSUPPORTED_TYPE + schema.getType().name()); - } + switch (schema.getType()) { + case BOOLEAN: + shouldFlush = putBooleanWithFlushCheck(insertTime, (boolean) objectValue); + break; + case INT32: + case DATE: + shouldFlush = putIntWithFlushCheck(insertTime, (int) objectValue); + break; + case INT64: + case TIMESTAMP: + shouldFlush = putLongWithFlushCheck(insertTime, (long) objectValue); + break; + case FLOAT: + shouldFlush = putFloatWithFlushCheck(insertTime, (float) objectValue); + break; + case DOUBLE: + shouldFlush = putDoubleWithFlushCheck(insertTime, (double) objectValue); + break; + case TEXT: + case BLOB: + case STRING: + shouldFlush = putBinaryWithFlushCheck(insertTime, (Binary) objectValue); + break; + default: + throw new UnSupportedDataTypeException(UNSUPPORTED_TYPE + schema.getType().name()); } if (shouldFlush) { return true; @@ -144,38 +142,36 @@ public boolean writeAlignedValueWithFlushCheck( public boolean writeWithFlushCheck( long[] times, Object valueList, BitMap bitMap, TSDataType dataType, int start, int end) { boolean shouldFlush; - synchronized (list) { - switch (dataType) { - case BOOLEAN: - boolean[] boolValues = (boolean[]) valueList; - shouldFlush = putBooleansWithFlushCheck(times, boolValues, bitMap, start, end); - break; - case INT32: - case DATE: - int[] intValues = (int[]) valueList; - shouldFlush = putIntsWithFlushCheck(times, intValues, bitMap, start, end); - break; - case INT64: - case TIMESTAMP: - long[] longValues = (long[]) valueList; - return putLongsWithFlushCheck(times, longValues, bitMap, start, end); - case FLOAT: - float[] floatValues = (float[]) valueList; - shouldFlush = putFloatsWithFlushCheck(times, floatValues, bitMap, start, end); - break; - case DOUBLE: - double[] doubleValues = (double[]) valueList; - shouldFlush = putDoublesWithFlushCheck(times, doubleValues, bitMap, start, end); - break; - case TEXT: - case BLOB: - case STRING: - Binary[] binaryValues = (Binary[]) valueList; - shouldFlush = putBinariesWithFlushCheck(times, binaryValues, bitMap, start, end); - break; - default: - throw new UnSupportedDataTypeException(UNSUPPORTED_TYPE + dataType.name()); - } + switch (dataType) { + case BOOLEAN: + boolean[] boolValues = (boolean[]) valueList; + shouldFlush = putBooleansWithFlushCheck(times, boolValues, bitMap, start, end); + break; + case INT32: + case DATE: + int[] intValues = (int[]) valueList; + shouldFlush = putIntsWithFlushCheck(times, intValues, bitMap, start, end); + break; + case INT64: + case TIMESTAMP: + long[] longValues = (long[]) valueList; + return putLongsWithFlushCheck(times, longValues, bitMap, start, end); + case FLOAT: + float[] floatValues = (float[]) valueList; + shouldFlush = putFloatsWithFlushCheck(times, floatValues, bitMap, start, end); + break; + case DOUBLE: + double[] doubleValues = (double[]) valueList; + shouldFlush = putDoublesWithFlushCheck(times, doubleValues, bitMap, start, end); + break; + case TEXT: + case BLOB: + case STRING: + Binary[] binaryValues = (Binary[]) valueList; + shouldFlush = putBinariesWithFlushCheck(times, binaryValues, bitMap, start, end); + break; + default: + throw new UnSupportedDataTypeException(UNSUPPORTED_TYPE + dataType.name()); } if (shouldFlush) { return true; @@ -305,14 +301,14 @@ private void sortTVList() { } if (!list.isSorted()) { - list.safelySort(); + list.sort(); } } @Override public synchronized void sortTvListForFlush() { if (!list.isSorted()) { - list.safelySort(); + list.sort(); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/AlignedTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/AlignedTVList.java index fa10f9b603337..6c191f0f0b3c6 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/AlignedTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/AlignedTVList.java @@ -179,7 +179,7 @@ public AlignedTVList clone() { @SuppressWarnings("squid:S3776") // Suppress high Cognitive Complexity warning @Override - public void putAlignedValue(long timestamp, Object[] value) { + public synchronized void putAlignedValue(long timestamp, Object[] value) { checkExpansion(); int arrayIndex = rowCount / ARRAY_SIZE; int elementIndex = rowCount % ARRAY_SIZE; @@ -764,7 +764,7 @@ public boolean reachChunkSizeOrPointNumThreshold() { @SuppressWarnings("squid:S3776") // Suppress high Cognitive Complexity warning @Override - public void putAlignedValues( + public synchronized void putAlignedValues( long[] time, Object[] value, BitMap[] bitMaps, int start, int end, TSStatus[] results) { checkExpansion(); int idx = start; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BackAlignedTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BackAlignedTVList.java index 26c1b7b8a8ded..2ad728a1a2643 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BackAlignedTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BackAlignedTVList.java @@ -32,11 +32,12 @@ public class BackAlignedTVList extends QuickAlignedTVList { } @Override - public void sort() { + public synchronized void sort() { if (!sorted) { policy.backwardSort(timestamps, rowCount); policy.clearTmp(); } sorted = true; + seqRowCount = rowCount; } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BackBinaryTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BackBinaryTVList.java index 48b0ecd4a199d..cd85976a58ef4 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BackBinaryTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BackBinaryTVList.java @@ -26,11 +26,12 @@ public class BackBinaryTVList extends QuickBinaryTVList { } @Override - public void sort() { + public synchronized void sort() { if (!sorted) { policy.backwardSort(timestamps, rowCount); policy.clearTmp(); } sorted = true; + seqRowCount = rowCount; } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BackBooleanTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BackBooleanTVList.java index e6e2153e3df5c..969d398128572 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BackBooleanTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BackBooleanTVList.java @@ -27,11 +27,12 @@ public class BackBooleanTVList extends QuickBooleanTVList { } @Override - public void sort() { + public synchronized void sort() { if (!sorted) { policy.backwardSort(timestamps, rowCount); policy.clearTmp(); } sorted = true; + seqRowCount = rowCount; } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BackDoubleTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BackDoubleTVList.java index 2915906e0e61d..6d79b9d0ab294 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BackDoubleTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BackDoubleTVList.java @@ -27,11 +27,12 @@ public class BackDoubleTVList extends QuickDoubleTVList { } @Override - public void sort() { + public synchronized void sort() { if (!sorted) { policy.backwardSort(timestamps, rowCount); policy.clearTmp(); } sorted = true; + seqRowCount = rowCount; } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BackFloatTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BackFloatTVList.java index 41a1acfa1bc3f..75b09507cc49a 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BackFloatTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BackFloatTVList.java @@ -27,11 +27,12 @@ public class BackFloatTVList extends QuickFloatTVList { } @Override - public void sort() { + public synchronized void sort() { if (!sorted) { policy.backwardSort(timestamps, rowCount); policy.clearTmp(); } sorted = true; + seqRowCount = rowCount; } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BackIntTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BackIntTVList.java index 776e065a4ac98..3ccc5cbc946f5 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BackIntTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BackIntTVList.java @@ -27,11 +27,12 @@ public class BackIntTVList extends QuickIntTVList { } @Override - public void sort() { + public synchronized void sort() { if (!sorted) { policy.backwardSort(timestamps, rowCount); policy.clearTmp(); } sorted = true; + seqRowCount = rowCount; } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BackLongTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BackLongTVList.java index b55e7c15df841..58d4a4e2fc8e0 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BackLongTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BackLongTVList.java @@ -26,11 +26,12 @@ public class BackLongTVList extends QuickLongTVList { } @Override - public void sort() { + public synchronized void sort() { if (!sorted) { policy.backwardSort(timestamps, rowCount); policy.clearTmp(); } sorted = true; + seqRowCount = rowCount; } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BinaryTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BinaryTVList.java index edf6b3ce3d34b..043228077bc69 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BinaryTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BinaryTVList.java @@ -87,7 +87,7 @@ private Binary[] cloneValue(Binary[] array) { } @Override - public void putBinary(long timestamp, Binary value) { + public synchronized void putBinary(long timestamp, Binary value) { checkExpansion(); int arrayIndex = rowCount / ARRAY_SIZE; int elementIndex = rowCount % ARRAY_SIZE; @@ -177,7 +177,8 @@ protected void releaseLastValueArray() { } @Override - public void putBinaries(long[] time, Binary[] value, BitMap bitMap, int start, int end) { + public synchronized void putBinaries( + long[] time, Binary[] value, BitMap bitMap, int start, int end) { checkExpansion(); int idx = start; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BooleanTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BooleanTVList.java index aa7e67da5ebf7..2206f658986a2 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BooleanTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BooleanTVList.java @@ -80,7 +80,7 @@ private boolean[] cloneValue(boolean[] array) { } @Override - public void putBoolean(long timestamp, boolean value) { + public synchronized void putBoolean(long timestamp, boolean value) { checkExpansion(); int arrayIndex = rowCount / ARRAY_SIZE; int elementIndex = rowCount % ARRAY_SIZE; @@ -164,7 +164,8 @@ protected void releaseLastValueArray() { } @Override - public void putBooleans(long[] time, boolean[] value, BitMap bitMap, int start, int end) { + public synchronized void putBooleans( + long[] time, boolean[] value, BitMap bitMap, int start, int end) { checkExpansion(); int idx = start; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/DoubleTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/DoubleTVList.java index 0e2080933f74e..6d1dc3a51bbe8 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/DoubleTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/DoubleTVList.java @@ -81,7 +81,7 @@ private double[] cloneValue(double[] array) { } @Override - public void putDouble(long timestamp, double value) { + public synchronized void putDouble(long timestamp, double value) { checkExpansion(); int arrayIndex = rowCount / ARRAY_SIZE; int elementIndex = rowCount % ARRAY_SIZE; @@ -170,7 +170,8 @@ protected void releaseLastValueArray() { } @Override - public void putDoubles(long[] time, double[] value, BitMap bitMap, int start, int end) { + public synchronized void putDoubles( + long[] time, double[] value, BitMap bitMap, int start, int end) { checkExpansion(); int idx = start; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/FloatTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/FloatTVList.java index e934f8fe4c378..f226236309afc 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/FloatTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/FloatTVList.java @@ -81,7 +81,7 @@ private float[] cloneValue(float[] array) { } @Override - public void putFloat(long timestamp, float value) { + public synchronized void putFloat(long timestamp, float value) { checkExpansion(); int arrayIndex = rowCount / ARRAY_SIZE; int elementIndex = rowCount % ARRAY_SIZE; @@ -170,7 +170,8 @@ protected void releaseLastValueArray() { } @Override - public void putFloats(long[] time, float[] value, BitMap bitMap, int start, int end) { + public synchronized void putFloats( + long[] time, float[] value, BitMap bitMap, int start, int end) { checkExpansion(); int idx = start; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/IntTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/IntTVList.java index 30fdf7b3b4cb9..4fb9cd3e184f8 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/IntTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/IntTVList.java @@ -80,7 +80,7 @@ private int[] cloneValue(int[] array) { } @Override - public void putInt(long timestamp, int value) { + public synchronized void putInt(long timestamp, int value) { checkExpansion(); int arrayIndex = rowCount / ARRAY_SIZE; int elementIndex = rowCount % ARRAY_SIZE; @@ -163,7 +163,7 @@ protected void releaseLastValueArray() { } @Override - public void putInts(long[] time, int[] value, BitMap bitMap, int start, int end) { + public synchronized void putInts(long[] time, int[] value, BitMap bitMap, int start, int end) { checkExpansion(); int idx = start; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/LongTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/LongTVList.java index 8d49e739bef3e..f182352ca8bec 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/LongTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/LongTVList.java @@ -80,7 +80,7 @@ private long[] cloneValue(long[] array) { } @Override - public void putLong(long timestamp, long value) { + public synchronized void putLong(long timestamp, long value) { checkExpansion(); int arrayIndex = rowCount / ARRAY_SIZE; int elementIndex = rowCount % ARRAY_SIZE; @@ -163,7 +163,7 @@ protected void releaseLastValueArray() { } @Override - public void putLongs(long[] time, long[] value, BitMap bitMap, int start, int end) { + public synchronized void putLongs(long[] time, long[] value, BitMap bitMap, int start, int end) { checkExpansion(); int idx = start; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/QuickAlignedTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/QuickAlignedTVList.java index ff11bdcd4a123..69f24527d6799 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/QuickAlignedTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/QuickAlignedTVList.java @@ -31,10 +31,11 @@ public class QuickAlignedTVList extends AlignedTVList { } @Override - public void sort() { + public synchronized void sort() { if (!sorted) { policy.qsort(0, rowCount - 1); } sorted = true; + seqRowCount = rowCount; } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/QuickBinaryTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/QuickBinaryTVList.java index f5bb5e65e34b6..645d0dbce5a26 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/QuickBinaryTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/QuickBinaryTVList.java @@ -26,10 +26,11 @@ public class QuickBinaryTVList extends BinaryTVList { } @Override - public void sort() { + public synchronized void sort() { if (!sorted) { policy.qsort(0, rowCount - 1); } sorted = true; + seqRowCount = rowCount; } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/QuickBooleanTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/QuickBooleanTVList.java index 0278378ea8663..779159ca71979 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/QuickBooleanTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/QuickBooleanTVList.java @@ -26,10 +26,11 @@ public class QuickBooleanTVList extends BooleanTVList { } @Override - public void sort() { + public synchronized void sort() { if (!sorted) { policy.qsort(0, rowCount - 1); } sorted = true; + seqRowCount = rowCount; } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/QuickDoubleTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/QuickDoubleTVList.java index 7f0f43d0b2d97..58f3cc1ce2f87 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/QuickDoubleTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/QuickDoubleTVList.java @@ -26,10 +26,11 @@ public class QuickDoubleTVList extends DoubleTVList { } @Override - public void sort() { + public synchronized void sort() { if (!sorted) { policy.qsort(0, rowCount - 1); } sorted = true; + seqRowCount = rowCount; } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/QuickFloatTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/QuickFloatTVList.java index 896135dae8c1b..2fc22ab29902d 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/QuickFloatTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/QuickFloatTVList.java @@ -26,10 +26,11 @@ public class QuickFloatTVList extends FloatTVList { } @Override - public void sort() { + public synchronized void sort() { if (!sorted) { policy.qsort(0, rowCount - 1); } sorted = true; + seqRowCount = rowCount; } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/QuickIntTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/QuickIntTVList.java index afca18e18318a..e83646e32024b 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/QuickIntTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/QuickIntTVList.java @@ -26,10 +26,11 @@ public class QuickIntTVList extends IntTVList { } @Override - public void sort() { + public synchronized void sort() { if (!sorted) { policy.qsort(0, rowCount - 1); } sorted = true; + seqRowCount = rowCount; } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/QuickLongTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/QuickLongTVList.java index dfe5da1f7ad34..5f629720c5d64 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/QuickLongTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/QuickLongTVList.java @@ -26,10 +26,11 @@ public class QuickLongTVList extends LongTVList { } @Override - public void sort() { + public synchronized void sort() { if (!sorted) { policy.qsort(0, rowCount - 1); } sorted = true; + seqRowCount = rowCount; } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java index f24f8d671708d..dada6298f828e 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java @@ -149,11 +149,6 @@ public synchronized boolean isSorted() { public abstract void sort(); - public synchronized void safelySort() { - sort(); - seqRowCount = rowCount; - } - public void increaseReferenceCount() { referenceCount.incrementAndGet(); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TimAlignedTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TimAlignedTVList.java index 558291dc02079..479e0a969a39f 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TimAlignedTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TimAlignedTVList.java @@ -31,7 +31,7 @@ public class TimAlignedTVList extends AlignedTVList { } @Override - public void sort() { + public synchronized void sort() { policy.checkSortedTimestampsAndIndices(); if (!sorted) { policy.sort(0, rowCount); @@ -39,6 +39,7 @@ public void sort() { policy.clearSortedValue(); policy.clearSortedTime(); sorted = true; + seqRowCount = rowCount; } @Override diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TimBinaryTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TimBinaryTVList.java index 379b9248b1b75..c78d67034da0b 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TimBinaryTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TimBinaryTVList.java @@ -26,7 +26,7 @@ public class TimBinaryTVList extends BinaryTVList { } @Override - public void sort() { + public synchronized void sort() { policy.checkSortedTimestampsAndIndices(); if (!sorted) { policy.sort(0, rowCount); @@ -34,6 +34,7 @@ public void sort() { policy.clearSortedValue(); policy.clearSortedTime(); sorted = true; + seqRowCount = rowCount; } @Override diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TimBooleanTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TimBooleanTVList.java index 1f45779fc3485..0e7dc48faaf35 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TimBooleanTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TimBooleanTVList.java @@ -26,7 +26,7 @@ public class TimBooleanTVList extends BooleanTVList { } @Override - public void sort() { + public synchronized void sort() { policy.checkSortedTimestampsAndIndices(); if (!sorted) { policy.sort(0, rowCount); @@ -34,6 +34,7 @@ public void sort() { policy.clearSortedValue(); policy.clearSortedTime(); sorted = true; + seqRowCount = rowCount; } @Override diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TimDoubleTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TimDoubleTVList.java index 74a5db378facf..29fc62d91c6e3 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TimDoubleTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TimDoubleTVList.java @@ -26,7 +26,7 @@ public class TimDoubleTVList extends DoubleTVList { } @Override - public void sort() { + public synchronized void sort() { policy.checkSortedTimestampsAndIndices(); if (!sorted) { policy.sort(0, rowCount); @@ -34,6 +34,7 @@ public void sort() { policy.clearSortedValue(); policy.clearSortedTime(); sorted = true; + seqRowCount = rowCount; } @Override diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TimFloatTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TimFloatTVList.java index 6522fd61159e2..83ad6aca1f676 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TimFloatTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TimFloatTVList.java @@ -26,7 +26,7 @@ public class TimFloatTVList extends FloatTVList { } @Override - public void sort() { + public synchronized void sort() { policy.checkSortedTimestampsAndIndices(); if (!sorted) { policy.sort(0, rowCount); @@ -34,6 +34,7 @@ public void sort() { policy.clearSortedValue(); policy.clearSortedTime(); sorted = true; + seqRowCount = rowCount; } @Override diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TimIntTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TimIntTVList.java index 7bcd7073ee511..8216f2eb2961d 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TimIntTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TimIntTVList.java @@ -26,7 +26,7 @@ public class TimIntTVList extends IntTVList { } @Override - public void sort() { + public synchronized void sort() { policy.checkSortedTimestampsAndIndices(); if (!sorted) { policy.sort(0, rowCount); @@ -34,6 +34,7 @@ public void sort() { policy.clearSortedValue(); policy.clearSortedTime(); sorted = true; + seqRowCount = rowCount; } @Override diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TimLongTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TimLongTVList.java index c42d7d8cc8c77..bcd5c3022f9c7 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TimLongTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TimLongTVList.java @@ -26,7 +26,7 @@ public class TimLongTVList extends LongTVList { } @Override - public void sort() { + public synchronized void sort() { policy.checkSortedTimestampsAndIndices(); if (!sorted) { policy.sort(0, rowCount); @@ -34,6 +34,7 @@ public void sort() { policy.clearSortedValue(); policy.clearSortedTime(); sorted = true; + seqRowCount = rowCount; } @Override From 6369293fc6071bc354feb5ffb50e3f51b7ec3281 Mon Sep 17 00:00:00 2001 From: shizy Date: Wed, 25 Dec 2024 21:27:47 +0800 Subject: [PATCH 27/54] refactor: change list to array in AlignedTVList iterator --- .../db/utils/datastructure/AlignedTVList.java | 50 +++++++++---------- .../MergeSortAlignedTVListIterator.java | 14 +++--- 2 files changed, 32 insertions(+), 32 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/AlignedTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/AlignedTVList.java index 6c191f0f0b3c6..d35ff7a5fbd09 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/AlignedTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/AlignedTVList.java @@ -51,6 +51,7 @@ import java.util.List; import java.util.Objects; import java.util.concurrent.CopyOnWriteArrayList; +import java.util.stream.IntStream; import static org.apache.iotdb.db.storageengine.rescon.memory.PrimitiveArrayManager.ARRAY_SIZE; import static org.apache.iotdb.db.storageengine.rescon.memory.PrimitiveArrayManager.TVLIST_SORT_ALGORITHM; @@ -1489,11 +1490,11 @@ public AlignedTVListIterator iterator( public class AlignedTVListIterator extends TVListIterator { private final BitMap allValueColDeletedMap; - private final List dataTypeList; + private final TSDataType[] dataTypeArray; + private final int[] columnIndexArray; private final Integer floatPrecision; - private final List encodingList; + private final TSEncoding[] encodingArray; - List columnIndexList; private final int[] validRowIndex; public AlignedTVListIterator( @@ -1503,11 +1504,14 @@ public AlignedTVListIterator( Integer floatPrecision, List encodingList) { super(null, null); - this.dataTypeList = dataTypeList; - this.columnIndexList = columnIndexList; + this.dataTypeArray = dataTypeList.toArray(new TSDataType[0]); + this.columnIndexArray = + (columnIndexList == null) + ? IntStream.range(0, dataTypes.size()).toArray() + : columnIndexList.stream().mapToInt(Integer::intValue).toArray(); this.allValueColDeletedMap = ignoreAllNullRows ? getAllValueColDeletedMap() : null; this.floatPrecision = floatPrecision; - this.encodingList = encodingList; + this.encodingArray = encodingList == null ? null : encodingList.toArray(new TSEncoding[0]); this.validRowIndex = new int[dataTypeList.size()]; } @@ -1539,7 +1543,7 @@ private void prepareNext() { // skip all-Null rows if allValueColDeletedMap exits int rowIndex = getValueIndex(index); if (allValueColDeletedMap == null || !allValueColDeletedMap.isMarked(rowIndex)) { - for (int columnIndex = 0; columnIndex < dataTypeList.size(); columnIndex++) { + for (int columnIndex = 0; columnIndex < dataTypeArray.length; columnIndex++) { // update currTvPair if the column is not null if (!isNull(rowIndex, columnIndex)) { validRowIndex[columnIndex] = rowIndex; @@ -1569,9 +1573,8 @@ public TimeValuePair next() { return null; } - TsPrimitiveType[] vector = new TsPrimitiveType[dataTypeList.size()]; - for (int columnIndex = 0; columnIndex < dataTypeList.size(); columnIndex++) { - // update currTvPair if the column is not null + TsPrimitiveType[] vector = new TsPrimitiveType[dataTypeArray.length]; + for (int columnIndex = 0; columnIndex < dataTypeArray.length; columnIndex++) { vector[columnIndex] = getPrimitiveObject(validRowIndex[columnIndex], columnIndex); } TimeValuePair tvPair = @@ -1586,17 +1589,15 @@ public TimeValuePair current() { if (!hasCurrent()) { return null; } - TsPrimitiveType[] vector = new TsPrimitiveType[dataTypeList.size()]; - for (int columnIndex = 0; columnIndex < dataTypeList.size(); columnIndex++) { - // update currTvPair if the column is not null + TsPrimitiveType[] vector = new TsPrimitiveType[dataTypeArray.length]; + for (int columnIndex = 0; columnIndex < dataTypeArray.length; columnIndex++) { vector[columnIndex] = getPrimitiveObject(validRowIndex[columnIndex], columnIndex); } return new TimeValuePair(currentTime, TsPrimitiveType.getByType(TSDataType.VECTOR, vector)); } public boolean isNull(int rowIndex, int columnIndex) { - int validColumnIndex = - (columnIndexList == null) ? columnIndex : columnIndexList.get(columnIndex); + int validColumnIndex = columnIndexArray[columnIndex]; if (validColumnIndex < 0 || validColumnIndex >= dataTypes.size()) { return true; } @@ -1607,15 +1608,14 @@ public TsPrimitiveType getPrimitiveObject(int rowIndex, int columnIndex) { if (rowIndex < 0 || rowIndex >= rows) { return null; } - int validColumnIndex = - (columnIndexList == null) ? columnIndex : columnIndexList.get(columnIndex); + int validColumnIndex = columnIndexArray[columnIndex]; if (validColumnIndex < 0 || validColumnIndex >= dataTypes.size()) { return null; } if (isNullValue(rowIndex, validColumnIndex)) { return null; } - switch (dataTypeList.get(columnIndex)) { + switch (dataTypeArray[columnIndex]) { case BOOLEAN: return TsPrimitiveType.getByType( TSDataType.BOOLEAN, getBooleanByValueIndex(rowIndex, validColumnIndex)); @@ -1630,20 +1630,20 @@ public TsPrimitiveType getPrimitiveObject(int rowIndex, int columnIndex) { case FLOAT: float valueF = getFloatByValueIndex(rowIndex, validColumnIndex); if (floatPrecision != null - && encodingList != null + && encodingArray != null && !Float.isNaN(valueF) - && (encodingList.get(columnIndex) == TSEncoding.RLE - || encodingList.get(columnIndex) == TSEncoding.TS_2DIFF)) { + && (encodingArray[columnIndex] == TSEncoding.RLE + || encodingArray[columnIndex] == TSEncoding.TS_2DIFF)) { valueF = MathUtils.roundWithGivenPrecision(valueF, floatPrecision); } return TsPrimitiveType.getByType(TSDataType.FLOAT, valueF); case DOUBLE: double valueD = getDoubleByValueIndex(rowIndex, validColumnIndex); if (floatPrecision != null - && encodingList != null + && encodingArray != null && !Double.isNaN(valueD) - && (encodingList.get(columnIndex) == TSEncoding.RLE - || encodingList.get(columnIndex) == TSEncoding.TS_2DIFF)) { + && (encodingArray[columnIndex] == TSEncoding.RLE + || encodingArray[columnIndex] == TSEncoding.TS_2DIFF)) { valueD = MathUtils.roundWithGivenPrecision(valueD, floatPrecision); } return TsPrimitiveType.getByType(TSDataType.DOUBLE, valueD); @@ -1654,7 +1654,7 @@ public TsPrimitiveType getPrimitiveObject(int rowIndex, int columnIndex) { TSDataType.TEXT, getBinaryByValueIndex(rowIndex, validColumnIndex)); default: throw new UnSupportedDataTypeException( - String.format("Data type %s is not supported.", dataTypeList.get(columnIndex))); + String.format("Data type %s is not supported.", dataTypeArray[columnIndex])); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MergeSortAlignedTVListIterator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MergeSortAlignedTVListIterator.java index 9988874086100..3b02a92e1c53f 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MergeSortAlignedTVListIterator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MergeSortAlignedTVListIterator.java @@ -31,7 +31,7 @@ public class MergeSortAlignedTVListIterator implements IPointReader { private final AlignedTVList.AlignedTVListIterator[] alignedTvListIterators; - private final List tsDataTypes; + private final int columnNum; private boolean probeNext = false; private boolean hasNext = false; @@ -58,12 +58,12 @@ public MergeSortAlignedTVListIterator( tsDataTypes, columnIndexList, ignoreAllNullRows, floatPrecision, encodingList); } this.alignedTvListOffsets = new int[alignedTvLists.size()]; - this.tsDataTypes = tsDataTypes; - this.columnAccessInfo = new int[tsDataTypes.size()][]; + this.columnNum = tsDataTypes.size(); + this.columnAccessInfo = new int[columnNum][]; for (int i = 0; i < columnAccessInfo.length; i++) { columnAccessInfo[i] = new int[2]; } - this.bitMap = new BitMap(tsDataTypes.size()); + this.bitMap = new BitMap(columnNum); } private void prepareNextRow() { @@ -72,7 +72,7 @@ private void prepareNextRow() { AlignedTVList.AlignedTVListIterator iterator = alignedTvListIterators[i]; if (iterator.hasNext() && iterator.currentTime() <= time) { if (i == 0 || iterator.currentTime() < time) { - for (int columnIndex = 0; columnIndex < tsDataTypes.size(); columnIndex++) { + for (int columnIndex = 0; columnIndex < columnNum; columnIndex++) { int rowIndex = iterator.getValidRowIndex(columnIndex); columnAccessInfo[columnIndex][0] = i; columnAccessInfo[columnIndex][1] = rowIndex; @@ -82,7 +82,7 @@ private void prepareNextRow() { } time = iterator.currentTime(); } else { - for (int columnIndex = 0; columnIndex < tsDataTypes.size(); columnIndex++) { + for (int columnIndex = 0; columnIndex < columnNum; columnIndex++) { int rowIndex = iterator.getValidRowIndex(columnIndex); // update if the column is not null if (!iterator.isNull(rowIndex, columnIndex)) { @@ -125,7 +125,7 @@ public TimeValuePair currentTimeValuePair() { } private TimeValuePair buildTimeValuePair() { - TsPrimitiveType[] vector = new TsPrimitiveType[tsDataTypes.size()]; + TsPrimitiveType[] vector = new TsPrimitiveType[columnNum]; for (int columnIndex = 0; columnIndex < vector.length; columnIndex++) { int[] accessInfo = columnAccessInfo[columnIndex]; AlignedTVList.AlignedTVListIterator iterator = alignedTvListIterators[accessInfo[0]]; From a80bcc04f70f930a6aefa92711b7560f586ba3a6 Mon Sep 17 00:00:00 2001 From: shizy Date: Thu, 26 Dec 2024 11:26:19 +0800 Subject: [PATCH 28/54] revert: remove CopyOnWriteArrayList --- .../iotdb/db/utils/datastructure/AlignedTVList.java | 11 +++++------ .../iotdb/db/utils/datastructure/BinaryTVList.java | 4 ++-- .../iotdb/db/utils/datastructure/BooleanTVList.java | 4 ++-- .../iotdb/db/utils/datastructure/DoubleTVList.java | 4 ++-- .../iotdb/db/utils/datastructure/FloatTVList.java | 4 ++-- .../iotdb/db/utils/datastructure/IntTVList.java | 4 ++-- .../iotdb/db/utils/datastructure/LongTVList.java | 4 ++-- .../apache/iotdb/db/utils/datastructure/TVList.java | 9 ++++----- 8 files changed, 21 insertions(+), 23 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/AlignedTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/AlignedTVList.java index d35ff7a5fbd09..988d54af55aaa 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/AlignedTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/AlignedTVList.java @@ -50,7 +50,6 @@ import java.util.Arrays; import java.util.List; import java.util.Objects; -import java.util.concurrent.CopyOnWriteArrayList; import java.util.stream.IntStream; import static org.apache.iotdb.db.storageengine.rescon.memory.PrimitiveArrayManager.ARRAY_SIZE; @@ -93,7 +92,7 @@ public abstract class AlignedTVList extends TVList { values = new ArrayList<>(types.size()); for (int i = 0; i < types.size(); i++) { - values.add(new CopyOnWriteArrayList<>()); + values.add(new ArrayList<>()); } } @@ -167,7 +166,7 @@ public AlignedTVList clone() { } } if (cloneList.bitMaps.get(i) == null) { - List cloneColumnBitMaps = new CopyOnWriteArrayList<>(); + List cloneColumnBitMaps = new ArrayList<>(); for (BitMap bitMap : columnBitMaps) { cloneColumnBitMaps.add(bitMap == null ? null : bitMap.clone()); } @@ -351,8 +350,8 @@ public void extendColumn(TSDataType dataType) { bitMaps.add(null); } } - List columnValue = new CopyOnWriteArrayList<>(); - List columnBitMaps = new CopyOnWriteArrayList<>(); + List columnValue = new ArrayList<>(); + List columnBitMaps = new ArrayList<>(); for (int i = 0; i < timestamps.size(); i++) { switch (dataType) { case TEXT: @@ -880,7 +879,7 @@ private void markNullValue(int columnIndex, int arrayIndex, int elementIndex) { // if the bitmap in columnIndex is null, init the bitmap of this column from the beginning if (bitMaps.get(columnIndex) == null) { - List columnBitMaps = new CopyOnWriteArrayList<>(); + List columnBitMaps = new ArrayList<>(); for (int i = 0; i < values.get(columnIndex).size(); i++) { columnBitMaps.add(new BitMap(ARRAY_SIZE)); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BinaryTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BinaryTVList.java index 043228077bc69..4a04fb7017caa 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BinaryTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BinaryTVList.java @@ -34,8 +34,8 @@ import java.io.DataInputStream; import java.io.IOException; +import java.util.ArrayList; import java.util.List; -import java.util.concurrent.CopyOnWriteArrayList; import java.util.stream.IntStream; import static org.apache.iotdb.db.storageengine.rescon.memory.PrimitiveArrayManager.ARRAY_SIZE; @@ -53,7 +53,7 @@ public abstract class BinaryTVList extends TVList { BinaryTVList() { super(); - values = new CopyOnWriteArrayList<>(); + values = new ArrayList<>(); memoryBinaryChunkSize = 0; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BooleanTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BooleanTVList.java index 2206f658986a2..16caedc83ce89 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BooleanTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BooleanTVList.java @@ -33,8 +33,8 @@ import java.io.DataInputStream; import java.io.IOException; +import java.util.ArrayList; import java.util.List; -import java.util.concurrent.CopyOnWriteArrayList; import java.util.stream.IntStream; import static org.apache.iotdb.db.storageengine.rescon.memory.PrimitiveArrayManager.ARRAY_SIZE; @@ -48,7 +48,7 @@ public abstract class BooleanTVList extends TVList { BooleanTVList() { super(); - values = new CopyOnWriteArrayList<>(); + values = new ArrayList<>(); } public static BooleanTVList newList() { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/DoubleTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/DoubleTVList.java index 6d1dc3a51bbe8..6682435e04722 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/DoubleTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/DoubleTVList.java @@ -34,8 +34,8 @@ import java.io.DataInputStream; import java.io.IOException; +import java.util.ArrayList; import java.util.List; -import java.util.concurrent.CopyOnWriteArrayList; import java.util.stream.IntStream; import static org.apache.iotdb.db.storageengine.rescon.memory.PrimitiveArrayManager.ARRAY_SIZE; @@ -49,7 +49,7 @@ public abstract class DoubleTVList extends TVList { DoubleTVList() { super(); - values = new CopyOnWriteArrayList<>(); + values = new ArrayList<>(); } public static DoubleTVList newList() { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/FloatTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/FloatTVList.java index f226236309afc..986a662bb350d 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/FloatTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/FloatTVList.java @@ -34,8 +34,8 @@ import java.io.DataInputStream; import java.io.IOException; +import java.util.ArrayList; import java.util.List; -import java.util.concurrent.CopyOnWriteArrayList; import java.util.stream.IntStream; import static org.apache.iotdb.db.storageengine.rescon.memory.PrimitiveArrayManager.ARRAY_SIZE; @@ -49,7 +49,7 @@ public abstract class FloatTVList extends TVList { FloatTVList() { super(); - values = new CopyOnWriteArrayList<>(); + values = new ArrayList<>(); } public static FloatTVList newList() { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/IntTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/IntTVList.java index 4fb9cd3e184f8..106b3bb740243 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/IntTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/IntTVList.java @@ -33,8 +33,8 @@ import java.io.DataInputStream; import java.io.IOException; +import java.util.ArrayList; import java.util.List; -import java.util.concurrent.CopyOnWriteArrayList; import java.util.stream.IntStream; import static org.apache.iotdb.db.storageengine.rescon.memory.PrimitiveArrayManager.ARRAY_SIZE; @@ -48,7 +48,7 @@ public abstract class IntTVList extends TVList { IntTVList() { super(); - values = new CopyOnWriteArrayList<>(); + values = new ArrayList<>(); } public static IntTVList newList() { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/LongTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/LongTVList.java index f182352ca8bec..37b21699de584 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/LongTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/LongTVList.java @@ -33,8 +33,8 @@ import java.io.DataInputStream; import java.io.IOException; +import java.util.ArrayList; import java.util.List; -import java.util.concurrent.CopyOnWriteArrayList; import java.util.stream.IntStream; import static org.apache.iotdb.db.storageengine.rescon.memory.PrimitiveArrayManager.ARRAY_SIZE; @@ -48,7 +48,7 @@ public abstract class LongTVList extends TVList { LongTVList() { super(); - values = new CopyOnWriteArrayList<>(); + values = new ArrayList<>(); } public static LongTVList newList() { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java index dada6298f828e..a2efacbd6cd22 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java @@ -43,7 +43,6 @@ import java.util.Arrays; import java.util.Collections; import java.util.List; -import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.locks.ReentrantLock; @@ -91,8 +90,8 @@ public abstract class TVList implements WALEntryValue { private long version; protected TVList() { - timestamps = new CopyOnWriteArrayList<>(); - indices = new CopyOnWriteArrayList<>(); + timestamps = new ArrayList<>(); + indices = new ArrayList<>(); rowCount = 0; seqRowCount = 0; maxTime = Long.MIN_VALUE; @@ -224,7 +223,7 @@ public int getValueIndex(int index) { protected void markNullValue(int arrayIndex, int elementIndex) { // init bitMap if doesn't have if (bitMap == null) { - bitMap = new CopyOnWriteArrayList<>(); + bitMap = new ArrayList<>(); for (int i = 0; i < timestamps.size(); i++) { bitMap.add(new BitMap(ARRAY_SIZE)); } @@ -263,7 +262,7 @@ protected void cloneSlicesAndBitMap(TVList cloneList) { } } if (bitMap != null) { - cloneList.bitMap = new CopyOnWriteArrayList<>(); + cloneList.bitMap = new ArrayList<>(); for (BitMap bm : bitMap) { cloneList.bitMap.add(bm == null ? null : bm.clone()); } From 3e0b9040d79b5fa2dd861d60d45d7442c5732def Mon Sep 17 00:00:00 2001 From: shizy Date: Fri, 27 Dec 2024 10:37:39 +0800 Subject: [PATCH 29/54] refactor: clone MergeSort iterator from ReadOnlyChunk --- .../memtable/AlignedReadOnlyMemChunk.java | 9 +++--- .../dataregion/memtable/ReadOnlyMemChunk.java | 10 +++---- .../reader/chunk/MemAlignedChunkReader.java | 13 +------- .../read/reader/chunk/MemChunkReader.java | 7 +---- .../db/utils/datastructure/AlignedTVList.java | 30 +++++++++++++++---- .../MergeSortAlignedTVListIterator.java | 25 ++++++++++++---- .../MergeSortTvListIterator.java | 17 +++++++---- .../iotdb/db/utils/datastructure/TVList.java | 23 +++++++++----- 8 files changed, 80 insertions(+), 54 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedReadOnlyMemChunk.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedReadOnlyMemChunk.java index c8d2bfad39a57..01558daad9346 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedReadOnlyMemChunk.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedReadOnlyMemChunk.java @@ -77,7 +77,7 @@ public class AlignedReadOnlyMemChunk extends ReadOnlyMemChunk { private final List columnIndexList; - private int workingTVListRows; + private MergeSortAlignedTVListIterator timeValuePairIterator; /** * The constructor for Aligned type. @@ -264,7 +264,7 @@ public void initChunkMetaFromTvLists() { // create MergeSortAlignedTVListIterator List alignedTvLists = new ArrayList<>(alignedTvListQueryMap.keySet()); - MergeSortAlignedTVListIterator timeValuePairIterator = + timeValuePairIterator = new MergeSortAlignedTVListIterator( alignedTvLists, dataTypes, @@ -272,7 +272,6 @@ public void initChunkMetaFromTvLists() { floatPrecision, encodingList, context.isIgnoreAllNullRows()); - this.workingTVListRows = timeValuePairIterator.getRowsForWorkingTVListIterator(); int[] alignedTvListOffsets = timeValuePairIterator.getAlignedTVListOffsets(); // iterate to build column access info and split pages @@ -565,7 +564,7 @@ public List[]> getValuesStatisticsList() { return valueStatisticsList; } - public int workingTVListRows() { - return workingTVListRows; + public MergeSortAlignedTVListIterator getMergeSortAlignedTVListIterator() { + return timeValuePairIterator; } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/ReadOnlyMemChunk.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/ReadOnlyMemChunk.java index c65c007851cba..4f3aeaa7752fa 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/ReadOnlyMemChunk.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/ReadOnlyMemChunk.java @@ -83,7 +83,7 @@ public class ReadOnlyMemChunk { // tvlist rowCount during query private Map tvListQueryMap; - private int workingTVListRows; + private MergeSortTvListIterator timeValuePairIterator; protected ReadOnlyMemChunk(QueryContext context) { this.context = context; @@ -143,9 +143,7 @@ public void initChunkMetaFromTvLists() { int cnt = 0; int[] deleteCursor = {0}; List tvLists = new ArrayList<>(tvListQueryMap.keySet()); - MergeSortTvListIterator timeValuePairIterator = - new MergeSortTvListIterator(tvLists, floatPrecision, encoding); - this.workingTVListRows = timeValuePairIterator.getRowsForWorkingTVListIterator(); + timeValuePairIterator = new MergeSortTvListIterator(tvLists, floatPrecision, encoding); int[] tvListOffsets = timeValuePairIterator.getTVListOffsets(); while (timeValuePairIterator.hasNextTimeValuePair()) { if (cnt % MAX_NUMBER_OF_POINTS_IN_PAGE == 0) { @@ -330,7 +328,7 @@ public TsBlock getTsBlock() { return null; } - public int workingTVListRows() { - return workingTVListRows; + public MergeSortTvListIterator getMergeSortTVListIterator() { + return timeValuePairIterator; } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemAlignedChunkReader.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemAlignedChunkReader.java index 75cb1c51e1abf..8ff72e1638ba3 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemAlignedChunkReader.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemAlignedChunkReader.java @@ -20,7 +20,6 @@ package org.apache.iotdb.db.storageengine.dataregion.read.reader.chunk; import org.apache.iotdb.db.storageengine.dataregion.memtable.AlignedReadOnlyMemChunk; -import org.apache.iotdb.db.utils.datastructure.AlignedTVList; import org.apache.iotdb.db.utils.datastructure.MergeSortAlignedTVListIterator; import org.apache.iotdb.db.utils.datastructure.PageColumnAccessInfo; @@ -55,17 +54,7 @@ public class MemAlignedChunkReader implements IChunkReader { public MemAlignedChunkReader(AlignedReadOnlyMemChunk readableChunk, Filter globalTimeFilter) { this.readableChunk = readableChunk; - List alignedTVLists = - new ArrayList<>(readableChunk.getAligendTvListQueryMap().keySet()); - timeValuePairIterator = - new MergeSortAlignedTVListIterator( - alignedTVLists, - readableChunk.getDataTypes(), - readableChunk.getColumnIndexList(), - readableChunk.getFloatPrecision(), - readableChunk.getEncodingList(), - readableChunk.getContext().isIgnoreAllNullRows()); - timeValuePairIterator.setRowsForWorkingTVListIterator(readableChunk.workingTVListRows()); + timeValuePairIterator = readableChunk.getMergeSortAlignedTVListIterator().clone(); this.globalTimeFilter = globalTimeFilter; this.pageReaderList = new ArrayList<>(); initAllPageReaders( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemChunkReader.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemChunkReader.java index e511432c8a85e..595d22d6c28ec 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemChunkReader.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemChunkReader.java @@ -21,7 +21,6 @@ import org.apache.iotdb.db.storageengine.dataregion.memtable.ReadOnlyMemChunk; import org.apache.iotdb.db.utils.datastructure.MergeSortTvListIterator; -import org.apache.iotdb.db.utils.datastructure.TVList; import org.apache.tsfile.enums.TSDataType; import org.apache.tsfile.file.metadata.IChunkMetadata; @@ -57,11 +56,7 @@ public class MemChunkReader implements IChunkReader, IPointReader { public MemChunkReader(ReadOnlyMemChunk readableChunk, Filter globalTimeFilter) { this.readableChunk = readableChunk; - List tvLists = new ArrayList<>(readableChunk.getTvListQueryMap().keySet()); - timeValuePairIterator = - new MergeSortTvListIterator( - tvLists, readableChunk.getFloatPrecision(), readableChunk.getEncoding()); - timeValuePairIterator.setRowsForWorkingTVListIterator(readableChunk.workingTVListRows()); + timeValuePairIterator = readableChunk.getMergeSortTVListIterator().clone(); this.globalTimeFilter = globalTimeFilter; this.pageReaderList = new ArrayList<>(); initAllPageReaders( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/AlignedTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/AlignedTVList.java index 988d54af55aaa..4cae56f51fed5 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/AlignedTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/AlignedTVList.java @@ -1487,14 +1487,18 @@ public AlignedTVListIterator iterator( /* AlignedTVList Iterator */ public class AlignedTVListIterator extends TVListIterator { - private final BitMap allValueColDeletedMap; + private BitMap allValueColDeletedMap; - private final TSDataType[] dataTypeArray; - private final int[] columnIndexArray; - private final Integer floatPrecision; - private final TSEncoding[] encodingArray; + private TSDataType[] dataTypeArray; + private int[] columnIndexArray; + private Integer floatPrecision; + private TSEncoding[] encodingArray; - private final int[] validRowIndex; + private int[] validRowIndex; + + public AlignedTVListIterator() { + super(); + } public AlignedTVListIterator( List dataTypeList, @@ -1660,5 +1664,19 @@ public TsPrimitiveType getPrimitiveObject(int rowIndex, int columnIndex) { public int getValidRowIndex(int columnIndex) { return validRowIndex[columnIndex]; } + + @Override + public AlignedTVListIterator clone() { + AlignedTVListIterator iterator = new AlignedTVListIterator(); + iterator.rows = rows; + iterator.dataTypeArray = dataTypeArray; + iterator.columnIndexArray = columnIndexArray; + iterator.allValueColDeletedMap = allValueColDeletedMap; + iterator.floatPrecision = floatPrecision; + iterator.encodingArray = this.encodingArray; + iterator.validRowIndex = new int[dataTypeArray.length]; + iterator.reset(); + return iterator; + } } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MergeSortAlignedTVListIterator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MergeSortAlignedTVListIterator.java index 3b02a92e1c53f..12c098e6eab3e 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MergeSortAlignedTVListIterator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MergeSortAlignedTVListIterator.java @@ -66,6 +66,22 @@ public MergeSortAlignedTVListIterator( this.bitMap = new BitMap(columnNum); } + public MergeSortAlignedTVListIterator( + AlignedTVList.AlignedTVListIterator[] alignedTvListIterators, int columnNum) { + this.alignedTvListIterators = + new AlignedTVList.AlignedTVListIterator[alignedTvListIterators.length]; + for (int i = 0; i < alignedTvListIterators.length; i++) { + this.alignedTvListIterators[i] = alignedTvListIterators[i].clone(); + } + this.alignedTvListOffsets = new int[alignedTvListIterators.length]; + this.columnNum = columnNum; + this.columnAccessInfo = new int[columnNum][]; + for (int i = 0; i < columnAccessInfo.length; i++) { + columnAccessInfo[i] = new int[2]; + } + this.bitMap = new BitMap(columnNum); + } + private void prepareNextRow() { time = Long.MAX_VALUE; for (int i = 0; i < alignedTvListIterators.length; i++) { @@ -190,11 +206,8 @@ public BitMap getBitmap() { return bitMap; } - public int getRowsForWorkingTVListIterator() { - return alignedTvListIterators[alignedTvListIterators.length - 1].getRows(); - } - - public void setRowsForWorkingTVListIterator(int rows) { - alignedTvListIterators[alignedTvListIterators.length - 1].setRows(rows); + @Override + public MergeSortAlignedTVListIterator clone() { + return new MergeSortAlignedTVListIterator(alignedTvListIterators, columnNum); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MergeSortTvListIterator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MergeSortTvListIterator.java index 6b9468936c9ce..37ae45121593c 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MergeSortTvListIterator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MergeSortTvListIterator.java @@ -51,6 +51,14 @@ public MergeSortTvListIterator( this.tvListOffsets = new int[tvLists.size()]; } + public MergeSortTvListIterator(TVList.TVListIterator[] tvListIterators) { + this.tvListIterators = new TVList.TVListIterator[tvListIterators.length]; + for (int i = 0; i < tvListIterators.length; i++) { + this.tvListIterators[i] = tvListIterators[i].clone(); + } + this.tvListOffsets = new int[tvListIterators.length]; + } + private void prepareNextRow() { long time = Long.MAX_VALUE; selectedTVListIndex = -1; @@ -126,11 +134,8 @@ public void setTVListOffsets(int[] tvListOffsets) { selectedTVListIndex = -1; } - public int getRowsForWorkingTVListIterator() { - return tvListIterators[tvListIterators.length - 1].getRows(); - } - - public void setRowsForWorkingTVListIterator(int rows) { - tvListIterators[tvListIterators.length - 1].setRows(rows); + @Override + public MergeSortTvListIterator clone() { + return new MergeSortTvListIterator(tvListIterators); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java index a2efacbd6cd22..f73ef9afccfb9 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java @@ -611,8 +611,10 @@ public class TVListIterator { protected int rows; protected long currentTime; protected boolean probeNext; - private final Integer floatPrecision; - private final TSEncoding encoding; + private Integer floatPrecision; + private TSEncoding encoding; + + public TVListIterator() {} public TVListIterator(Integer floatPrecision, TSEncoding encoding) { this.index = 0; @@ -694,13 +696,20 @@ protected void step() { currentTime = index < rows ? getTime(index) : Long.MIN_VALUE; } - public int getRows() { - return rows; + public void reset() { + index = 0; + currentTime = index < rows ? getTime(index) : Long.MIN_VALUE; + probeNext = false; } - public void setRows(int rows) { - this.rows = rows; - this.currentTime = index < rows ? getTime(index) : Long.MIN_VALUE; + @Override + public TVListIterator clone() { + TVListIterator iterator = new TVListIterator(); + iterator.rows = rows; + iterator.floatPrecision = floatPrecision; + iterator.encoding = encoding; + iterator.reset(); + return iterator; } } } From ca08605fe5e6f54ac1ec23603365c928e99ecf2f Mon Sep 17 00:00:00 2001 From: shizy Date: Sat, 28 Dec 2024 11:05:58 +0800 Subject: [PATCH 30/54] fix: clone working tvlist during flush if there is query on it --- .../utils/ResourceByPathUtils.java | 14 +++++++++---- .../memtable/AlignedWritableMemChunk.java | 21 +++++++++++++++++++ .../dataregion/memtable/WritableMemChunk.java | 21 +++++++++++++++++++ 3 files changed, 52 insertions(+), 4 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/utils/ResourceByPathUtils.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/utils/ResourceByPathUtils.java index fff51bb95bbd2..946672433baf7 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/utils/ResourceByPathUtils.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/utils/ResourceByPathUtils.java @@ -213,6 +213,7 @@ private Map prepareAlignedTvListMapForQuery( // mutable aligned TVList AlignedTVList list = alignedMemChunk.getWorkingTVList(); + AlignedTVList cloneList = null; list.lockQueryList(); try { if (!isWorkMemTable) { @@ -242,15 +243,17 @@ private Map prepareAlignedTvListMapForQuery( list.setOwnerQuery(firstQuery); // clone TVList - AlignedTVList cloneList = list.clone(); + cloneList = list.clone(); cloneList.getQueryContextList().add(context); alignedTvListQueryMap.put(cloneList, cloneList.rowCount()); - alignedMemChunk.setWorkingTVList(cloneList); } } } finally { list.unlockQueryList(); } + if (cloneList != null) { + alignedMemChunk.setWorkingTVList(cloneList); + } return alignedTvListQueryMap; } @@ -475,6 +478,7 @@ private Map prepareTvListMapForQuery( // mutable tvlist TVList list = memChunk.getWorkingTVList(); + TVList cloneList = null; list.lockQueryList(); try { if (!isWorkMemTable) { @@ -504,15 +508,17 @@ private Map prepareTvListMapForQuery( list.setOwnerQuery(firstQuery); // clone TVList - TVList cloneList = list.clone(); + cloneList = list.clone(); cloneList.getQueryContextList().add(context); tvListQueryMap.put(cloneList, cloneList.rowCount()); - memChunk.setWorkingTVList(cloneList); } } } finally { list.unlockQueryList(); } + if (cloneList != null) { + memChunk.setWorkingTVList(cloneList); + } return tvListQueryMap; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedWritableMemChunk.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedWritableMemChunk.java index b54ee8f37f2f9..c5d916ba10c17 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedWritableMemChunk.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedWritableMemChunk.java @@ -379,6 +379,27 @@ private void sortTVList() { @Override public synchronized void sortTvListForFlush() { + AlignedTVList cloneList = null; + list.lockQueryList(); + try { + if (!list.isSorted() && !list.getQueryContextList().isEmpty()) { + QueryContext firstQuery = list.getQueryContextList().get(0); + // reserve query memory + if (firstQuery instanceof FragmentInstanceContext) { + MemoryReservationManager memoryReservationManager = + ((FragmentInstanceContext) firstQuery).getMemoryReservationContext(); + memoryReservationManager.reserveMemoryCumulatively(list.calculateRamSize()); + } + list.setOwnerQuery(firstQuery); + cloneList = list.clone(); + } + } finally { + list.unlockQueryList(); + } + if (cloneList != null) { + setWorkingTVList(cloneList); + } + if (!list.isSorted()) { list.sort(); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/WritableMemChunk.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/WritableMemChunk.java index 19d51a7c9b193..b1d6d1b0599c5 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/WritableMemChunk.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/WritableMemChunk.java @@ -307,6 +307,27 @@ private void sortTVList() { @Override public synchronized void sortTvListForFlush() { + TVList cloneList = null; + list.lockQueryList(); + try { + if (!list.isSorted() && !list.getQueryContextList().isEmpty()) { + QueryContext firstQuery = list.getQueryContextList().get(0); + // reserve query memory + if (firstQuery instanceof FragmentInstanceContext) { + MemoryReservationManager memoryReservationManager = + ((FragmentInstanceContext) firstQuery).getMemoryReservationContext(); + memoryReservationManager.reserveMemoryCumulatively(list.calculateRamSize()); + } + list.setOwnerQuery(firstQuery); + cloneList = list.clone(); + } + } finally { + list.unlockQueryList(); + } + if (cloneList != null) { + setWorkingTVList(cloneList); + } + if (!list.isSorted()) { list.sort(); } From f23a72beefb7ebc9c1d00907f425ff30a6b09982 Mon Sep 17 00:00:00 2001 From: shizy Date: Mon, 30 Dec 2024 15:06:10 +0800 Subject: [PATCH 31/54] fix: writable mem chunk flush conditions --- .../memtable/AlignedWritableMemChunk.java | 21 +++++++++- .../memtable/IWritableMemChunk.java | 4 ++ .../dataregion/memtable/WritableMemChunk.java | 40 +++++++++++-------- .../db/utils/datastructure/AlignedTVList.java | 15 +------ .../db/utils/datastructure/BinaryTVList.java | 4 +- .../iotdb/db/utils/datastructure/TVList.java | 13 ++---- 6 files changed, 55 insertions(+), 42 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedWritableMemChunk.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedWritableMemChunk.java index c5d916ba10c17..158fb05e11499 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedWritableMemChunk.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedWritableMemChunk.java @@ -131,7 +131,7 @@ public boolean putBooleanWithFlushCheck(long t, boolean v) { @Override public boolean putAlignedValueWithFlushCheck(long t, Object[] v) { list.putAlignedValue(t, v); - return list.reachChunkSizeOrPointNumThreshold(); + return reachChunkSizeOrPointNumThreshold(); } @Override @@ -170,7 +170,7 @@ public boolean putBooleansWithFlushCheck( public boolean putAlignedValuesWithFlushCheck( long[] t, Object[] v, BitMap[] bitMaps, int start, int end, TSStatus[] results) { list.putAlignedValues(t, v, bitMaps, start, end, results); - return list.reachChunkSizeOrPointNumThreshold(); + return reachChunkSizeOrPointNumThreshold(); } @Override @@ -831,4 +831,21 @@ public List buildColumnIndexList(List schemaList) { } return columnIndexList; } + + private boolean reachChunkSizeOrPointNumThreshold() { + if (rowCount() >= MAX_SERIES_POINT_NUMBER) { + return true; + } + long[] totalBinaryChunkSize = new long[dataTypes.size()]; + for (AlignedTVList alignedTvList : sortedList) { + long[] binaryChunkSize = alignedTvList.memoryBinaryChunkSize(); + for (int i = 0; i < binaryChunkSize.length; i++) { + totalBinaryChunkSize[i] += binaryChunkSize[i]; + if (totalBinaryChunkSize[i] >= TARGET_CHUNK_SIZE) { + return true; + } + } + } + return false; + } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/IWritableMemChunk.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/IWritableMemChunk.java index 25749ad58aa54..993794cce40c3 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/IWritableMemChunk.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/IWritableMemChunk.java @@ -37,6 +37,10 @@ public interface IWritableMemChunk extends WALEntryValue { int MAX_NUMBER_OF_POINTS_IN_PAGE = TSFileDescriptor.getInstance().getConfig().getMaxNumberOfPointsInPage(); + long MAX_SERIES_POINT_NUMBER = + IoTDBDescriptor.getInstance().getConfig().getAvgSeriesPointNumberThreshold(); + long TARGET_CHUNK_SIZE = IoTDBDescriptor.getInstance().getConfig().getTargetChunkSize(); + boolean putLongWithFlushCheck(long t, long v); boolean putIntWithFlushCheck(long t, int v); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/WritableMemChunk.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/WritableMemChunk.java index b1d6d1b0599c5..0a06362bdcf02 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/WritableMemChunk.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/WritableMemChunk.java @@ -19,8 +19,6 @@ package org.apache.iotdb.db.storageengine.dataregion.memtable; import org.apache.iotdb.common.rpc.thrift.TSStatus; -import org.apache.iotdb.db.conf.IoTDBConfig; -import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.queryengine.execution.fragment.FragmentInstanceContext; import org.apache.iotdb.db.queryengine.execution.fragment.QueryContext; import org.apache.iotdb.db.queryengine.plan.planner.memory.MemoryReservationManager; @@ -54,8 +52,6 @@ public class WritableMemChunk implements IWritableMemChunk { private static final String UNSUPPORTED_TYPE = "Unsupported data type:"; private static final Logger LOGGER = LoggerFactory.getLogger(WritableMemChunk.class); - private static final IoTDBConfig CONFIG = IoTDBDescriptor.getInstance().getConfig(); - public WritableMemChunk(IMeasurementSchema schema) { this.schema = schema; this.list = TVList.newList(schema.getType()); @@ -198,37 +194,37 @@ public boolean writeAlignedValuesWithFlushCheck( @Override public boolean putLongWithFlushCheck(long t, long v) { list.putLong(t, v); - return list.reachChunkSizeOrPointNumThreshold(); + return reachChunkSizeOrPointNumThreshold(); } @Override public boolean putIntWithFlushCheck(long t, int v) { list.putInt(t, v); - return list.reachChunkSizeOrPointNumThreshold(); + return reachChunkSizeOrPointNumThreshold(); } @Override public boolean putFloatWithFlushCheck(long t, float v) { list.putFloat(t, v); - return list.reachChunkSizeOrPointNumThreshold(); + return reachChunkSizeOrPointNumThreshold(); } @Override public boolean putDoubleWithFlushCheck(long t, double v) { list.putDouble(t, v); - return list.reachChunkSizeOrPointNumThreshold(); + return reachChunkSizeOrPointNumThreshold(); } @Override public boolean putBinaryWithFlushCheck(long t, Binary v) { list.putBinary(t, v); - return list.reachChunkSizeOrPointNumThreshold(); + return reachChunkSizeOrPointNumThreshold(); } @Override public boolean putBooleanWithFlushCheck(long t, boolean v) { list.putBoolean(t, v); - return list.reachChunkSizeOrPointNumThreshold(); + return reachChunkSizeOrPointNumThreshold(); } @Override @@ -239,39 +235,39 @@ public boolean putAlignedValueWithFlushCheck(long t, Object[] v) { @Override public boolean putLongsWithFlushCheck(long[] t, long[] v, BitMap bitMap, int start, int end) { list.putLongs(t, v, bitMap, start, end); - return list.reachChunkSizeOrPointNumThreshold(); + return reachChunkSizeOrPointNumThreshold(); } @Override public boolean putIntsWithFlushCheck(long[] t, int[] v, BitMap bitMap, int start, int end) { list.putInts(t, v, bitMap, start, end); - return list.reachChunkSizeOrPointNumThreshold(); + return reachChunkSizeOrPointNumThreshold(); } @Override public boolean putFloatsWithFlushCheck(long[] t, float[] v, BitMap bitMap, int start, int end) { list.putFloats(t, v, bitMap, start, end); - return list.reachChunkSizeOrPointNumThreshold(); + return reachChunkSizeOrPointNumThreshold(); } @Override public boolean putDoublesWithFlushCheck(long[] t, double[] v, BitMap bitMap, int start, int end) { list.putDoubles(t, v, bitMap, start, end); - return list.reachChunkSizeOrPointNumThreshold(); + return reachChunkSizeOrPointNumThreshold(); } @Override public boolean putBinariesWithFlushCheck( long[] t, Binary[] v, BitMap bitMap, int start, int end) { list.putBinaries(t, v, bitMap, start, end); - return list.reachChunkSizeOrPointNumThreshold(); + return reachChunkSizeOrPointNumThreshold(); } @Override public boolean putBooleansWithFlushCheck( long[] t, boolean[] v, BitMap bitMap, int start, int end) { list.putBooleans(t, v, bitMap, start, end); - return list.reachChunkSizeOrPointNumThreshold(); + return reachChunkSizeOrPointNumThreshold(); } @Override @@ -360,6 +356,14 @@ public long rowCount() { return rowCount; } + public long chunkSize() { + long chunkSize = list.chunkSize(); + for (TVList tvList : sortedList) { + chunkSize += tvList.chunkSize(); + } + return chunkSize; + } + @Override public IMeasurementSchema getSchema() { return schema; @@ -631,6 +635,10 @@ public static WritableMemChunk deserialize(DataInputStream stream) throws IOExce return memChunk; } + private boolean reachChunkSizeOrPointNumThreshold() { + return rowCount() >= MAX_SERIES_POINT_NUMBER || chunkSize() >= TARGET_CHUNK_SIZE; + } + public List getSortedList() { return sortedList; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/AlignedTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/AlignedTVList.java index 4cae56f51fed5..15f8ca321af2d 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/AlignedTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/AlignedTVList.java @@ -76,9 +76,6 @@ public abstract class AlignedTVList extends TVList { // Index relation: columnIndex(dataTypeIndex) -> arrayIndex -> elementIndex protected List> bitMaps; - // If a sensor chunk size of Text datatype reaches the threshold, this flag will be set true - boolean reachMaxChunkSizeFlag; - // not null when constructed by queries for tree model BitMap allValueColDeletedMap; // constructed after deletion @@ -88,7 +85,6 @@ public abstract class AlignedTVList extends TVList { super(); dataTypes = types; memoryBinaryChunkSize = new long[dataTypes.size()]; - reachMaxChunkSizeFlag = false; values = new ArrayList<>(types.size()); for (int i = 0; i < types.size(); i++) { @@ -202,9 +198,6 @@ public synchronized void putAlignedValue(long timestamp, Object[] value) { columnValue != null ? getBinarySize((Binary) columnValue) : getBinarySize(Binary.EMPTY_VALUE); - if (memoryBinaryChunkSize[i] >= TARGET_CHUNK_SIZE) { - reachMaxChunkSizeFlag = true; - } break; case FLOAT: ((float[]) columnValues.get(arrayIndex))[elementIndex] = @@ -757,9 +750,8 @@ protected void releaseLastValueArray() { } } - @Override - public boolean reachChunkSizeOrPointNumThreshold() { - return reachMaxChunkSizeFlag || rowCount >= MAX_SERIES_POINT_NUMBER; + public long[] memoryBinaryChunkSize() { + return memoryBinaryChunkSize; } @SuppressWarnings("squid:S3776") // Suppress high Cognitive Complexity warning @@ -836,9 +828,6 @@ private void arrayCopy(Object[] value, int idx, int arrayIndex, int elementIndex memoryBinaryChunkSize[i] += arrayT[elementIndex + i1] != null ? getBinarySize(arrayT[elementIndex + i1]) : 0; } - if (memoryBinaryChunkSize[i] > TARGET_CHUNK_SIZE) { - reachMaxChunkSizeFlag = true; - } break; case FLOAT: float[] arrayF = ((float[]) columnValues.get(arrayIndex)); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BinaryTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BinaryTVList.java index 4a04fb7017caa..e66fc611e4a9e 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BinaryTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BinaryTVList.java @@ -108,8 +108,8 @@ public synchronized void putBinary(long timestamp, Binary value) { } @Override - public boolean reachChunkSizeOrPointNumThreshold() { - return memoryBinaryChunkSize >= TARGET_CHUNK_SIZE || rowCount >= MAX_SERIES_POINT_NUMBER; + public long chunkSize() { + return memoryBinaryChunkSize; } @Override diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java index f73ef9afccfb9..1a446c56fc325 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java @@ -21,7 +21,6 @@ import org.apache.iotdb.common.rpc.thrift.TSStatus; import org.apache.iotdb.commons.utils.TestOnly; -import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.queryengine.execution.fragment.QueryContext; import org.apache.iotdb.db.storageengine.dataregion.wal.buffer.WALEntryValue; import org.apache.iotdb.db.storageengine.rescon.memory.PrimitiveArrayManager; @@ -52,10 +51,6 @@ public abstract class TVList implements WALEntryValue { protected static final String ERR_DATATYPE_NOT_CONSISTENT = "DataType not consistent"; - protected static final long TARGET_CHUNK_SIZE = - IoTDBDescriptor.getInstance().getConfig().getTargetChunkSize(); - protected static final long MAX_SERIES_POINT_NUMBER = - IoTDBDescriptor.getInstance().getConfig().getAvgSeriesPointNumberThreshold(); // list of timestamp array, add 1 when expanded -> data point timestamp array // index relation: arrayIndex -> elementIndex protected List timestamps; @@ -164,6 +159,10 @@ public int seqRowCount() { return seqRowCount; } + public long chunkSize() { + return 0; + } + public int count() { if (bitMap == null) { return rowCount; @@ -308,10 +307,6 @@ public void putBinary(long time, Binary value) { throw new UnsupportedOperationException(ERR_DATATYPE_NOT_CONSISTENT); } - public boolean reachChunkSizeOrPointNumThreshold() { - return rowCount >= MAX_SERIES_POINT_NUMBER; - } - public void putBoolean(long time, boolean value) { throw new UnsupportedOperationException(ERR_DATATYPE_NOT_CONSISTENT); } From 8c7db5582b0d2852ff9e2d8400ae3854f9247dba Mon Sep 17 00:00:00 2001 From: shizy Date: Tue, 31 Dec 2024 19:33:52 +0800 Subject: [PATCH 32/54] refactor: add annotation and variable/function rename --- .../org/apache/iotdb/db/conf/IoTDBConfig.java | 5 +- .../fragment/FragmentInstanceContext.java | 9 +- .../execution/fragment/QueryContext.java | 2 +- .../utils/ResourceByPathUtils.java | 307 ++++++++++-------- .../memtable/AlignedReadOnlyMemChunk.java | 277 ++++++++-------- .../memtable/AlignedWritableMemChunk.java | 3 +- .../dataregion/memtable/ReadOnlyMemChunk.java | 20 +- .../dataregion/memtable/WritableMemChunk.java | 26 +- .../read/reader/chunk/MemChunkReader.java | 5 +- .../db/utils/datastructure/AlignedTVList.java | 21 +- .../MergeSortAlignedTVListIterator.java | 7 +- .../datastructure/PageColumnAccessInfo.java | 2 +- .../chunk/MemAlignedChunkLoaderTest.java | 22 +- .../read/reader/chunk/MemChunkLoaderTest.java | 111 +++++-- 14 files changed, 466 insertions(+), 351 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java index 507c6aaf5a201..57ddbe888ba2d 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java @@ -430,7 +430,10 @@ public class IoTDBConfig { /** The sort algorithm used in TVList */ private TVListSortAlgorithm tvListSortAlgorithm = TVListSortAlgorithm.TIM; - /** the threshold when working TVList is sorted and handover in writable memtable */ + /** + * the threshold when working TVList is sorted and added into immutable TVList list in the + * writable memtable + */ private int tvListSortThreshold = 0; /** When average series point number reaches this, flush the memtable to disk */ diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceContext.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceContext.java index 73e4b0b5f28b0..2b12262aaaecd 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceContext.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceContext.java @@ -650,6 +650,12 @@ public void releaseResourceWhenAllDriversAreClosed() { releaseResource(); } + /** + * It checks all referenced TVList by the query: 1. If current is not the owner, just remove + * itself from query context list 2. If current query is the owner and no other query use it now, + * release the TVList 3. If current query is the owner and other queries still use it, set the + * next query as owner + */ private void releaseTVListOwnedByQuery() { for (TVList tvList : tvListSet) { tvList.lockQueryList(); @@ -657,7 +663,6 @@ private void releaseTVListOwnedByQuery() { try { queryContextList.remove(this); if (tvList.getOwnerQuery() == this) { - tvList.setOwnerQuery(null); if (queryContextList.isEmpty()) { LOGGER.debug( "TVList {} is released by the query, FragmentInstance Id is {}", @@ -667,7 +672,7 @@ private void releaseTVListOwnedByQuery() { memoryReservationManager.releaseMemoryCumulatively(tvList.calculateRamSize()); } else { LOGGER.debug( - "TVList {} is owned by another query, FragmentInstance Id is {}", + "TVList {} is now owned by another query, FragmentInstance Id is {}", tvList, ((FragmentInstanceContext) queryContextList.get(0)).getId()); tvList.setOwnerQuery(queryContextList.get(0)); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/QueryContext.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/QueryContext.java index e0abc8e424786..fed6e7e9f61e6 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/QueryContext.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/QueryContext.java @@ -74,7 +74,7 @@ public class QueryContext { private final Set nonExistentModFiles = new CopyOnWriteArraySet<>(); - // accessed tvlists for the query + // referenced TVLists for the query protected final Set tvListSet = new HashSet<>(); public QueryContext() {} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/utils/ResourceByPathUtils.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/utils/ResourceByPathUtils.java index 946672433baf7..2b95bb20f0617 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/utils/ResourceByPathUtils.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/utils/ResourceByPathUtils.java @@ -187,76 +187,6 @@ public AlignedTimeSeriesMetadata generateTimeSeriesMetadata( return new AlignedTimeSeriesMetadata(timeTimeSeriesMetadata, valueTimeSeriesMetadataList); } - private Map prepareAlignedTvListMapForQuery( - QueryContext context, - AlignedWritableMemChunk alignedMemChunk, - boolean isWorkMemTable, - Filter globalTimeFilter) { - Map alignedTvListQueryMap = new LinkedHashMap<>(); - // immutable aligned TVList - for (AlignedTVList alignedTvList : alignedMemChunk.getSortedList()) { - if (globalTimeFilter != null - && !globalTimeFilter.satisfyStartEndTime( - alignedTvList.getMinTime(), alignedTvList.getMaxTime())) { - continue; - } - alignedTvList.lockQueryList(); - try { - LOGGER.debug( - "Flushing/Working MemTable - add current query context to immutable AlignedTVList's query list"); - alignedTvList.getQueryContextList().add(context); - alignedTvListQueryMap.put(alignedTvList, alignedTvList.rowCount()); - } finally { - alignedTvList.unlockQueryList(); - } - } - - // mutable aligned TVList - AlignedTVList list = alignedMemChunk.getWorkingTVList(); - AlignedTVList cloneList = null; - list.lockQueryList(); - try { - if (!isWorkMemTable) { - if (globalTimeFilter == null - || globalTimeFilter.satisfyStartEndTime(list.getMinTime(), list.getMaxTime())) { - LOGGER.debug( - "Flushing MemTable - add current query context to mutable AlignedTVList's query list"); - list.getQueryContextList().add(context); - alignedTvListQueryMap.put(list, list.rowCount()); - } - } else { - if (list.isSorted() || list.getQueryContextList().isEmpty()) { - LOGGER.debug( - "Working MemTable - add current query context to mutable AlignedTVList's query list when it's sorted or no other query on it"); - list.getQueryContextList().add(context); - alignedTvListQueryMap.put(list, list.rowCount()); - } else { - LOGGER.debug( - "Working MemTable - clone mutable AlignedTVList and replace old AlignedTVList in working MemTable"); - QueryContext firstQuery = list.getQueryContextList().get(0); - // reserve query memory - if (firstQuery instanceof FragmentInstanceContext) { - MemoryReservationManager memoryReservationManager = - ((FragmentInstanceContext) firstQuery).getMemoryReservationContext(); - memoryReservationManager.reserveMemoryCumulatively(list.calculateRamSize()); - } - list.setOwnerQuery(firstQuery); - - // clone TVList - cloneList = list.clone(); - cloneList.getQueryContextList().add(context); - alignedTvListQueryMap.put(cloneList, cloneList.rowCount()); - } - } - } finally { - list.unlockQueryList(); - } - if (cloneList != null) { - alignedMemChunk.setWorkingTVList(cloneList); - } - return alignedTvListQueryMap; - } - @Override public ReadOnlyMemChunk getReadOnlyMemChunkFromMemTable( QueryContext context, @@ -288,8 +218,7 @@ public ReadOnlyMemChunk getReadOnlyMemChunkFromMemTable( } } - // prepare AlignedTVList for query. It should clone and sort TVList if necessary. - // Also, the map keeps AlignedTVList length at this moment. + // prepare AlignedTVList for query. It should clone TVList if necessary. Map alignedTvListQueryMap = prepareAlignedTvListMapForQuery( context, alignedMemChunk, modsToMemtable == null, globalTimeFilter); @@ -407,6 +336,85 @@ public List getVisibleMetadataListFromWriter( chunkMetadataList.removeIf(x -> x.getEndTime() < timeLowerBound); return new ArrayList<>(chunkMetadataList); } + + /** + * Prepare the AlignedTVList references for the query. Same logic as prepareTvListMapForQuery + * + * @param context query context + * @param alignedMemChunk aligned writable memchunk + * @param isWorkMemTable in working or flushing memtable + * @param globalTimeFilter global time filter + * @return Map + */ + private Map prepareAlignedTvListMapForQuery( + QueryContext context, + AlignedWritableMemChunk alignedMemChunk, + boolean isWorkMemTable, + Filter globalTimeFilter) { + Map alignedTvListQueryMap = new LinkedHashMap<>(); + // immutable aligned TVList + for (AlignedTVList alignedTvList : alignedMemChunk.getSortedList()) { + if (globalTimeFilter != null + && !globalTimeFilter.satisfyStartEndTime( + alignedTvList.getMinTime(), alignedTvList.getMaxTime())) { + continue; + } + alignedTvList.lockQueryList(); + try { + LOGGER.debug( + "Flushing/Working MemTable - add current query context to immutable AlignedTVList's query list"); + alignedTvList.getQueryContextList().add(context); + alignedTvListQueryMap.put(alignedTvList, alignedTvList.rowCount()); + } finally { + alignedTvList.unlockQueryList(); + } + } + + // mutable aligned TVList + AlignedTVList list = alignedMemChunk.getWorkingTVList(); + AlignedTVList cloneList = null; + list.lockQueryList(); + try { + if (!isWorkMemTable) { + if (globalTimeFilter == null + || globalTimeFilter.satisfyStartEndTime(list.getMinTime(), list.getMaxTime())) { + LOGGER.debug( + "Flushing MemTable - add current query context to mutable AlignedTVList's query list"); + list.getQueryContextList().add(context); + alignedTvListQueryMap.put(list, list.rowCount()); + } + } else { + if (list.isSorted() || list.getQueryContextList().isEmpty()) { + LOGGER.debug( + "Working MemTable - add current query context to mutable AlignedTVList's query list when it's sorted or no other query on it"); + list.getQueryContextList().add(context); + alignedTvListQueryMap.put(list, list.rowCount()); + } else { + LOGGER.debug( + "Working MemTable - clone mutable AlignedTVList and replace old AlignedTVList in working MemTable"); + QueryContext firstQuery = list.getQueryContextList().get(0); + // reserve query memory + if (firstQuery instanceof FragmentInstanceContext) { + MemoryReservationManager memoryReservationManager = + ((FragmentInstanceContext) firstQuery).getMemoryReservationContext(); + memoryReservationManager.reserveMemoryCumulatively(list.calculateRamSize()); + } + list.setOwnerQuery(firstQuery); + + // clone TVList + cloneList = list.clone(); + cloneList.getQueryContextList().add(context); + alignedTvListQueryMap.put(cloneList, cloneList.rowCount()); + } + } + } finally { + list.unlockQueryList(); + } + if (cloneList != null) { + alignedMemChunk.setWorkingTVList(cloneList); + } + return alignedTvListQueryMap; + } } class MeasurementResourceByPathUtils extends ResourceByPathUtils { @@ -453,10 +461,85 @@ public ITimeSeriesMetadata generateTimeSeriesMetadata( return timeSeriesMetadata; } + @Override + public ReadOnlyMemChunk getReadOnlyMemChunkFromMemTable( + QueryContext context, + IMemTable memTable, + List> modsToMemtable, + long timeLowerBound, + Filter globalTimeFilter) + throws QueryProcessException, IOException { + Map memTableMap = memTable.getMemTableMap(); + IDeviceID deviceID = fullPath.getDeviceId(); + // check If Memtable Contains this path + if (!memTableMap.containsKey(deviceID) + || !memTableMap.get(deviceID).contains(fullPath.getMeasurement())) { + return null; + } + WritableMemChunk memChunk = + (WritableMemChunk) + memTableMap.get(deviceID).getMemChunkMap().get(fullPath.getMeasurement()); + // prepare TVList for query. It should clone TVList if necessary. + Map tvListQueryMap = + prepareTvListMapForQuery(context, memChunk, modsToMemtable == null, globalTimeFilter); + List deletionList = null; + if (modsToMemtable != null) { + deletionList = + ModificationUtils.constructDeletionList( + fullPath.getDeviceId(), + fullPath.getMeasurement(), + memTable, + modsToMemtable, + timeLowerBound); + } + return new ReadOnlyMemChunk( + context, + fullPath.getMeasurement(), + fullPath.getMeasurementSchema().getType(), + fullPath.getMeasurementSchema().getEncodingType(), + tvListQueryMap, + fullPath.getMeasurementSchema().getProps(), + deletionList); + } + + @Override + public List getVisibleMetadataListFromWriter( + RestorableTsFileIOWriter writer, + TsFileResource tsFileResource, + QueryContext context, + long timeLowerBound) { + List modifications = + context.getPathModifications( + tsFileResource, fullPath.getDeviceId(), fullPath.getMeasurement()); + + List chunkMetadataList = + new ArrayList<>( + writer.getVisibleMetadataList( + fullPath.getDeviceId(), + fullPath.getMeasurement(), + fullPath.getMeasurementSchema().getType())); + + ModificationUtils.modifyChunkMetaData(chunkMetadataList, modifications); + chunkMetadataList.removeIf(x -> x.getEndTime() < timeLowerBound); + return chunkMetadataList; + } + + /** + * Prepare the TVList references for the query. We remember TVLists' row count here and determine + * whether the TVLists needs sorting later during operator execution based on it. It need not + * protect sorted list. Sorted list is changed in the handover process of inserting, which holds + * the data region write lock. At this moment, query thread holds the data region read lock. + * + * @param context query context + * @param memChunk writable memchunk + * @param isWorkMemTable in working or flushing memtable + * @param globalTimeFilter global time filter + * @return Map + */ private Map prepareTvListMapForQuery( + QueryContext context, WritableMemChunk memChunk, boolean isWorkMemTable, - QueryContext context, Filter globalTimeFilter) { Map tvListQueryMap = new LinkedHashMap<>(); // immutable sorted lists @@ -496,6 +579,20 @@ private Map prepareTvListMapForQuery( list.getQueryContextList().add(context); tvListQueryMap.put(list, list.rowCount()); } else { + /* + * +----------------------+ + * | MemTable | + * | | + * | +------------+ | +-----------------+ + * | | TVList |<---+--+ +---+ Previous Query | + * | +-----^------+ | | | +-----------------+ + * | | | | | + * +----------+-----------+ | | +----------------+ + * | Clone +---+---+ Current Query | + * +-----+------+ | +----------------+ + * | TVList | <---------+ + * +------------+ + */ LOGGER.debug( "Working MemTable - clone mutable TVList and replace old TVList in working MemTable"); QueryContext firstQuery = list.getQueryContextList().get(0); @@ -521,68 +618,4 @@ private Map prepareTvListMapForQuery( } return tvListQueryMap; } - - @Override - public ReadOnlyMemChunk getReadOnlyMemChunkFromMemTable( - QueryContext context, - IMemTable memTable, - List> modsToMemtable, - long timeLowerBound, - Filter globalTimeFilter) - throws QueryProcessException, IOException { - Map memTableMap = memTable.getMemTableMap(); - IDeviceID deviceID = fullPath.getDeviceId(); - // check If Memtable Contains this path - if (!memTableMap.containsKey(deviceID) - || !memTableMap.get(deviceID).contains(fullPath.getMeasurement())) { - return null; - } - WritableMemChunk memChunk = - (WritableMemChunk) - memTableMap.get(deviceID).getMemChunkMap().get(fullPath.getMeasurement()); - // prepare TVList for query. It should clone and sort TVList if necessary. - // Also, the map keeps TVlist length at this moment. - Map tvListQueryMap = - prepareTvListMapForQuery(memChunk, modsToMemtable == null, context, globalTimeFilter); - List deletionList = null; - if (modsToMemtable != null) { - deletionList = - ModificationUtils.constructDeletionList( - fullPath.getDeviceId(), - fullPath.getMeasurement(), - memTable, - modsToMemtable, - timeLowerBound); - } - return new ReadOnlyMemChunk( - context, - fullPath.getMeasurement(), - fullPath.getMeasurementSchema().getType(), - fullPath.getMeasurementSchema().getEncodingType(), - tvListQueryMap, - fullPath.getMeasurementSchema().getProps(), - deletionList); - } - - @Override - public List getVisibleMetadataListFromWriter( - RestorableTsFileIOWriter writer, - TsFileResource tsFileResource, - QueryContext context, - long timeLowerBound) { - List modifications = - context.getPathModifications( - tsFileResource, fullPath.getDeviceId(), fullPath.getMeasurement()); - - List chunkMetadataList = - new ArrayList<>( - writer.getVisibleMetadataList( - fullPath.getDeviceId(), - fullPath.getMeasurement(), - fullPath.getMeasurementSchema().getType())); - - ModificationUtils.modifyChunkMetaData(chunkMetadataList, modifications); - chunkMetadataList.removeIf(x -> x.getEndTime() < timeLowerBound); - return chunkMetadataList; - } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedReadOnlyMemChunk.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedReadOnlyMemChunk.java index 01558daad9346..f685d67f3d0cd 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedReadOnlyMemChunk.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedReadOnlyMemChunk.java @@ -19,7 +19,6 @@ package org.apache.iotdb.db.storageengine.dataregion.memtable; -import org.apache.iotdb.db.exception.query.QueryProcessException; import org.apache.iotdb.db.queryengine.execution.fragment.QueryContext; import org.apache.iotdb.db.storageengine.dataregion.read.reader.chunk.MemAlignedChunkLoader; import org.apache.iotdb.db.utils.datastructure.AlignedTVList; @@ -75,6 +74,8 @@ public class AlignedReadOnlyMemChunk extends ReadOnlyMemChunk { // AlignedTVList rowCount during query protected Map alignedTvListQueryMap; + // For example, it stores time series [s1, s2, s3] in AlignedWritableMemChunk. + // When we select two of time series [s1, s3], the column index list should be [0, 2] private final List columnIndexList; private MergeSortAlignedTVListIterator timeValuePairIterator; @@ -83,11 +84,11 @@ public class AlignedReadOnlyMemChunk extends ReadOnlyMemChunk { * The constructor for Aligned type. * * @param context query context + * @param columnIndexList column index list * @param schema VectorMeasurementSchema - * @param alignedTvListQueryMap VectorTvList + * @param alignedTvListQueryMap AlignedTvList map * @param timeColumnDeletion The timeRange of deletionList * @param valueColumnsDeletionList time value column deletionList - * @throws QueryProcessException if there is unsupported data type. */ public AlignedReadOnlyMemChunk( QueryContext context, @@ -123,135 +124,6 @@ public void sortTvLists() { } } - private void updateTimeStatistics( - long[] time, - int count, - Statistics chunkTimeStatistics, - Statistics pageTimeStatistics) { - // update time statistics - for (int index = 0; index < count; index++) { - chunkTimeStatistics.update(time[index]); - pageTimeStatistics.update(time[index]); - } - pageTimeStatistics.setEmpty(count == 0); - } - - private void checkValueStatistics( - Statistics[] chunkValueStatistics, - Statistics[] pageValueStatistics, - List dataTypes, - int columnIndex) { - if (pageValueStatistics[columnIndex] == null) { - Statistics valueStatistics = - Statistics.getStatsByType(dataTypes.get(columnIndex)); - pageValueStatistics[columnIndex] = valueStatistics; - } - if (chunkValueStatistics[columnIndex] == null) { - Statistics chunkValueStats = - Statistics.getStatsByType(dataTypes.get(columnIndex)); - chunkValueStatistics[columnIndex] = chunkValueStats; - } - } - - private void updateValueStatistics( - long[] time, - PageColumnAccessInfo[] columnAccessInfo, - Statistics[] chunkValueStatistics, - Statistics[] pageValueStatistics, - MergeSortAlignedTVListIterator timeValuePairIterator) { - // update value statistics - for (int columnIndex = 0; columnIndex < dataTypes.size(); columnIndex++) { - PageColumnAccessInfo pageAccessInfo = columnAccessInfo[columnIndex]; - switch (dataTypes.get(columnIndex)) { - case BOOLEAN: - for (int index = 0; index < pageAccessInfo.count(); index++) { - int[] accessInfo = pageAccessInfo.get(index); - TsPrimitiveType value = - timeValuePairIterator.getPrimitiveObject(accessInfo, columnIndex); - if (value != null) { - checkValueStatistics( - pageValueStatistics, chunkValueStatistics, dataTypes, columnIndex); - pageValueStatistics[columnIndex].update(time[index], value.getBoolean()); - chunkValueStatistics[columnIndex].update(time[index], value.getBoolean()); - } - } - break; - case INT32: - case DATE: - for (int index = 0; index < pageAccessInfo.count(); index++) { - int[] accessInfo = pageAccessInfo.get(index); - TsPrimitiveType value = - timeValuePairIterator.getPrimitiveObject(accessInfo, columnIndex); - if (value != null) { - checkValueStatistics( - pageValueStatistics, chunkValueStatistics, dataTypes, columnIndex); - pageValueStatistics[columnIndex].update(time[index], value.getInt()); - chunkValueStatistics[columnIndex].update(time[index], value.getInt()); - } - } - break; - case INT64: - case TIMESTAMP: - for (int index = 0; index < pageAccessInfo.count(); index++) { - int[] accessInfo = pageAccessInfo.get(index); - TsPrimitiveType value = - timeValuePairIterator.getPrimitiveObject(accessInfo, columnIndex); - if (value != null) { - checkValueStatistics( - pageValueStatistics, chunkValueStatistics, dataTypes, columnIndex); - pageValueStatistics[columnIndex].update(time[index], value.getLong()); - chunkValueStatistics[columnIndex].update(time[index], value.getLong()); - } - } - break; - case FLOAT: - for (int index = 0; index < pageAccessInfo.count(); index++) { - int[] accessInfo = pageAccessInfo.get(index); - TsPrimitiveType value = - timeValuePairIterator.getPrimitiveObject(accessInfo, columnIndex); - if (value != null) { - checkValueStatistics( - pageValueStatistics, chunkValueStatistics, dataTypes, columnIndex); - pageValueStatistics[columnIndex].update(time[index], value.getFloat()); - chunkValueStatistics[columnIndex].update(time[index], value.getFloat()); - } - } - break; - case DOUBLE: - for (int index = 0; index < pageAccessInfo.count(); index++) { - int[] accessInfo = pageAccessInfo.get(index); - TsPrimitiveType value = - timeValuePairIterator.getPrimitiveObject(accessInfo, columnIndex); - if (value != null) { - checkValueStatistics( - pageValueStatistics, chunkValueStatistics, dataTypes, columnIndex); - pageValueStatistics[columnIndex].update(time[index], value.getDouble()); - chunkValueStatistics[columnIndex].update(time[index], value.getDouble()); - } - } - break; - case TEXT: - case BLOB: - case STRING: - for (int index = 0; index < pageAccessInfo.count(); index++) { - int[] accessInfo = pageAccessInfo.get(index); - TsPrimitiveType value = - timeValuePairIterator.getPrimitiveObject(accessInfo, columnIndex); - if (value != null) { - checkValueStatistics( - pageValueStatistics, chunkValueStatistics, dataTypes, columnIndex); - pageValueStatistics[columnIndex].update(time[index], value.getBinary()); - chunkValueStatistics[columnIndex].update(time[index], value.getBinary()); - } - } - break; - default: - throw new UnSupportedDataTypeException( - String.format("Data type %s is not supported.", dataTypes.get(columnIndex))); - } - } - } - @Override public void initChunkMetaFromTvLists() { // init chunk meta @@ -290,12 +162,14 @@ public void initChunkMetaFromTvLists() { while (timeValuePairIterator.hasNextTimeValuePair()) { long timestamp = timeValuePairIterator.getTime(); + // ignore deleted row if (timeColumnDeletion != null && isPointDeleted(timestamp, timeColumnDeletion, timeDeleteCursor)) { timeValuePairIterator.step(); continue; } + // ignore all-null row BitMap bitMap = timeValuePairIterator.getBitmap(); if (valueColumnsDeletionList != null) { for (int columnIndex = 0; columnIndex < dataTypes.size(); columnIndex++) { @@ -312,9 +186,8 @@ && isPointDeleted(timestamp, timeColumnDeletion, timeDeleteCursor)) { continue; } - // Split pages + // create pageTimeStatistics and pageValueStatistics for new page if (pointsInPage == 0) { - // Add new pageTimeStatistics array and pageValueStatistics into list Statistics pageTimeStatistics = Statistics.getStatsByType(TSDataType.VECTOR); timeStatisticsList.add(pageTimeStatistics); @@ -326,10 +199,10 @@ && isPointDeleted(timestamp, timeColumnDeletion, timeDeleteCursor)) { } // prepare column access info for current page - int[][] accessInfo = timeValuePairIterator.getColumnAccessInfo(); + int[][] columnAccessInfo = timeValuePairIterator.getColumnAccessInfo(); for (int i = 0; i < dataTypes.size(); i++) { time[pointsInPage] = timeValuePairIterator.getTime(); - pageColumnAccessInfo[i].add(accessInfo[i]); + pageColumnAccessInfo[i].add(columnAccessInfo[i]); } timeValuePairIterator.step(); pointsInPage++; @@ -351,8 +224,8 @@ && isPointDeleted(timestamp, timeColumnDeletion, timeDeleteCursor)) { timeValuePairIterator); // reset - for (PageColumnAccessInfo columnAccessInfo : pageColumnAccessInfo) { - columnAccessInfo.reset(); + for (PageColumnAccessInfo p : pageColumnAccessInfo) { + p.reset(); } chunkTimeStatistics.setEmpty(false); pointsInPage = 0; @@ -567,4 +440,132 @@ public List[]> getValuesStatisticsList() { public MergeSortAlignedTVListIterator getMergeSortAlignedTVListIterator() { return timeValuePairIterator; } + + private void updateTimeStatistics( + long[] time, + int count, + Statistics chunkTimeStatistics, + Statistics pageTimeStatistics) { + for (int index = 0; index < count; index++) { + chunkTimeStatistics.update(time[index]); + pageTimeStatistics.update(time[index]); + } + pageTimeStatistics.setEmpty(count == 0); + } + + private void createValueStatisticsIfNotExists( + Statistics[] chunkValueStatistics, + Statistics[] pageValueStatistics, + List dataTypes, + int columnIndex) { + if (pageValueStatistics[columnIndex] == null) { + Statistics valueStatistics = + Statistics.getStatsByType(dataTypes.get(columnIndex)); + pageValueStatistics[columnIndex] = valueStatistics; + } + if (chunkValueStatistics[columnIndex] == null) { + Statistics chunkValueStats = + Statistics.getStatsByType(dataTypes.get(columnIndex)); + chunkValueStatistics[columnIndex] = chunkValueStats; + } + } + + private void updateValueStatistics( + long[] time, + PageColumnAccessInfo[] columnAccessInfo, + Statistics[] chunkValueStatistics, + Statistics[] pageValueStatistics, + MergeSortAlignedTVListIterator timeValuePairIterator) { + // update value statistics + for (int columnIndex = 0; columnIndex < dataTypes.size(); columnIndex++) { + PageColumnAccessInfo pageAccessInfo = columnAccessInfo[columnIndex]; + switch (dataTypes.get(columnIndex)) { + case BOOLEAN: + for (int index = 0; index < pageAccessInfo.count(); index++) { + int[] accessInfo = pageAccessInfo.get(index); + TsPrimitiveType value = + timeValuePairIterator.getPrimitiveObject(accessInfo, columnIndex); + if (value != null) { + createValueStatisticsIfNotExists( + pageValueStatistics, chunkValueStatistics, dataTypes, columnIndex); + pageValueStatistics[columnIndex].update(time[index], value.getBoolean()); + chunkValueStatistics[columnIndex].update(time[index], value.getBoolean()); + } + } + break; + case INT32: + case DATE: + for (int index = 0; index < pageAccessInfo.count(); index++) { + int[] accessInfo = pageAccessInfo.get(index); + TsPrimitiveType value = + timeValuePairIterator.getPrimitiveObject(accessInfo, columnIndex); + if (value != null) { + createValueStatisticsIfNotExists( + pageValueStatistics, chunkValueStatistics, dataTypes, columnIndex); + pageValueStatistics[columnIndex].update(time[index], value.getInt()); + chunkValueStatistics[columnIndex].update(time[index], value.getInt()); + } + } + break; + case INT64: + case TIMESTAMP: + for (int index = 0; index < pageAccessInfo.count(); index++) { + int[] accessInfo = pageAccessInfo.get(index); + TsPrimitiveType value = + timeValuePairIterator.getPrimitiveObject(accessInfo, columnIndex); + if (value != null) { + createValueStatisticsIfNotExists( + pageValueStatistics, chunkValueStatistics, dataTypes, columnIndex); + pageValueStatistics[columnIndex].update(time[index], value.getLong()); + chunkValueStatistics[columnIndex].update(time[index], value.getLong()); + } + } + break; + case FLOAT: + for (int index = 0; index < pageAccessInfo.count(); index++) { + int[] accessInfo = pageAccessInfo.get(index); + TsPrimitiveType value = + timeValuePairIterator.getPrimitiveObject(accessInfo, columnIndex); + if (value != null) { + createValueStatisticsIfNotExists( + pageValueStatistics, chunkValueStatistics, dataTypes, columnIndex); + pageValueStatistics[columnIndex].update(time[index], value.getFloat()); + chunkValueStatistics[columnIndex].update(time[index], value.getFloat()); + } + } + break; + case DOUBLE: + for (int index = 0; index < pageAccessInfo.count(); index++) { + int[] accessInfo = pageAccessInfo.get(index); + TsPrimitiveType value = + timeValuePairIterator.getPrimitiveObject(accessInfo, columnIndex); + if (value != null) { + createValueStatisticsIfNotExists( + pageValueStatistics, chunkValueStatistics, dataTypes, columnIndex); + pageValueStatistics[columnIndex].update(time[index], value.getDouble()); + chunkValueStatistics[columnIndex].update(time[index], value.getDouble()); + } + } + break; + case TEXT: + case BLOB: + case STRING: + for (int index = 0; index < pageAccessInfo.count(); index++) { + int[] accessInfo = pageAccessInfo.get(index); + TsPrimitiveType value = + timeValuePairIterator.getPrimitiveObject(accessInfo, columnIndex); + if (value != null) { + createValueStatisticsIfNotExists( + pageValueStatistics, chunkValueStatistics, dataTypes, columnIndex); + pageValueStatistics[columnIndex].update(time[index], value.getBinary()); + chunkValueStatistics[columnIndex].update(time[index], value.getBinary()); + } + } + break; + default: + throw new UnSupportedDataTypeException( + String.format("Data type %s is not supported.", dataTypes.get(columnIndex))); + } + } + } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedWritableMemChunk.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedWritableMemChunk.java index 158fb05e11499..0faeac575c1b1 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedWritableMemChunk.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedWritableMemChunk.java @@ -205,6 +205,7 @@ protected void handoverAlignedTvList() { list.setOwnerQuery(firstQuery); // clone tv list AlignedTVList cloneList = list.clone(); + cloneList.sort(); sortedList.add(cloneList); } } finally { @@ -671,9 +672,9 @@ private void writePageValuesIntoWriter( } } - @SuppressWarnings({"squid:S6541", "squid:S3776"}) @Override public void encode(IChunkWriter chunkWriter) { + // use original encode method when TVList handover never happens. It runs a little faster. if (TVLIST_SORT_THRESHOLD == 0) { encodeWorkingAlignedTVList(chunkWriter); return; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/ReadOnlyMemChunk.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/ReadOnlyMemChunk.java index 4f3aeaa7752fa..7cb73cf54e88c 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/ReadOnlyMemChunk.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/ReadOnlyMemChunk.java @@ -44,6 +44,7 @@ import org.slf4j.LoggerFactory; import java.io.IOException; +import java.io.Serializable; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -74,13 +75,15 @@ public class ReadOnlyMemChunk { private List deletionList; // Read only chunk is now regarded as multiple pages. Apart from chunk statistics, - // we need to collect page statistic and MergeSortTvListIterator offset for each page. - private List pageStatisticsList; + // we need to collect page statistic. + private List> pageStatisticsList; - // page offsets + // page offsets in the MemChunk. For example, there are two TVList/AlignedTVList. TVList1: 1, + // 2, 3, 4, 5, 6, 7 TVList2: 1, 3, 5, 7, 8, 9 If each page has 3 points in it. The page offset + // lists would be [[0, 0], [3, 2], [6, 3], [7, 6]]. [7, 6] means the end. protected List pageOffsetsList; - // tvlist rowCount during query + // TVList and its rowCount during query private Map tvListQueryMap; private MergeSortTvListIterator timeValuePairIterator; @@ -139,7 +142,7 @@ public void sortTvLists() { public void initChunkMetaFromTvLists() { // create chunk statistics - Statistics chunkStatistics = Statistics.getStatsByType(dataType); + Statistics chunkStatistics = Statistics.getStatsByType(dataType); int cnt = 0; int[] deleteCursor = {0}; List tvLists = new ArrayList<>(tvListQueryMap.keySet()); @@ -147,13 +150,14 @@ public void initChunkMetaFromTvLists() { int[] tvListOffsets = timeValuePairIterator.getTVListOffsets(); while (timeValuePairIterator.hasNextTimeValuePair()) { if (cnt % MAX_NUMBER_OF_POINTS_IN_PAGE == 0) { - Statistics stats = Statistics.getStatsByType(dataType); + Statistics stats = Statistics.getStatsByType(dataType); pageStatisticsList.add(stats); pageOffsetsList.add(Arrays.copyOf(tvListOffsets, tvListOffsets.length)); } TimeValuePair tvPair = timeValuePairIterator.nextTimeValuePair(); if (!isPointDeleted(tvPair.getTimestamp(), deletionList, deleteCursor)) { - Statistics pageStatistics = pageStatisticsList.get(pageStatisticsList.size() - 1); + Statistics pageStatistics = + pageStatisticsList.get(pageStatisticsList.size() - 1); switch (dataType) { case BOOLEAN: chunkStatistics.update(tvPair.getTimestamp(), tvPair.getValue().getBoolean()); @@ -311,7 +315,7 @@ public List getDeletionList() { return deletionList; } - public List getPageStatisticsList() { + public List> getPageStatisticsList() { return pageStatisticsList; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/WritableMemChunk.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/WritableMemChunk.java index 0a06362bdcf02..e0cafa2f03812 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/WritableMemChunk.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/WritableMemChunk.java @@ -70,6 +70,20 @@ protected void handoverTvList() { list.sort(); sortedList.add(list); } else { + /* + * +----------------------+ + * | MemTable | + * | | + * | +---------------+ | +----------+ + * | | sorted TVList | | +---+ Query | + * | +------^--------+ | | +----------+ + * | | | | + * +----------+-----------+ | + * | Clone + Sort | + * +-----+------+ | + * | TVList | <---------+ + * +------------+ + */ QueryContext firstQuery = list.getQueryContextList().get(0); // reserve query memory if (firstQuery instanceof FragmentInstanceContext) { @@ -81,6 +95,7 @@ protected void handoverTvList() { list.setOwnerQuery(firstQuery); // clone tv list TVList cloneList = list.clone(); + cloneList.sort(); sortedList.add(cloneList); } } finally { @@ -306,6 +321,8 @@ public synchronized void sortTvListForFlush() { TVList cloneList = null; list.lockQueryList(); try { + // During flush, if the working TVList is not sorted and referenced by some query, we need to + // clone it. The query still refer to original unsorted TVList. if (!list.isSorted() && !list.getQueryContextList().isEmpty()) { QueryContext firstQuery = list.getQueryContextList().get(0); // reserve query memory @@ -543,6 +560,7 @@ private void writeData(ChunkWriterImpl chunkWriterImpl, TimeValuePair tvPair) { @Override public void encode(IChunkWriter chunkWriter) { + // use original encode method when TVList handover never happens. if (TVLIST_SORT_THRESHOLD == 0) { encodeWorkingTVList(chunkWriter); return; @@ -573,10 +591,10 @@ public void encode(IChunkWriter chunkWriter) { } /** - * Release process for memtable flush. Release the TVList if there is no query on it, otherwise - * set query owner and release the TVList until query finishes. + * Release the TVList after flush if there is no more query on it, otherwise set query owner and + * the unfinished query is responsible to release the TVList. * - * @param tvList + * @param tvList TVList */ private void maybeReleaseTvList(TVList tvList) { tvList.lockQueryList(); @@ -586,7 +604,7 @@ private void maybeReleaseTvList(TVList tvList) { } else { QueryContext firstQuery = tvList.getQueryContextList().get(0); // transfer memory from write process to read process. Here it reserves read memory and - // releaseFlushedMemTable will release write memory. + // releaseFlushedMemTable will release write memory correspondingly. if (firstQuery instanceof FragmentInstanceContext) { MemoryReservationManager memoryReservationManager = ((FragmentInstanceContext) firstQuery).getMemoryReservationContext(); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemChunkReader.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemChunkReader.java index 595d22d6c28ec..0f80b6a45ebfc 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemChunkReader.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemChunkReader.java @@ -35,6 +35,7 @@ import org.apache.tsfile.read.reader.IPointReader; import java.io.IOException; +import java.io.Serializable; import java.util.ArrayList; import java.util.Collections; import java.util.List; @@ -66,7 +67,9 @@ public MemChunkReader(ReadOnlyMemChunk readableChunk, Filter globalTimeFilter) { } private void initAllPageReaders( - IChunkMetadata metadata, List pageStats, List pageOffsetsList) { + IChunkMetadata metadata, + List> pageStats, + List pageOffsetsList) { Supplier tsBlockSupplier = new TsBlockSupplier(); for (int i = 0; i < pageStats.size(); i++) { MemPageReader pageReader = diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/AlignedTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/AlignedTVList.java index 15f8ca321af2d..274b773750bcc 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/AlignedTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/AlignedTVList.java @@ -1483,7 +1483,10 @@ public class AlignedTVListIterator extends TVListIterator { private Integer floatPrecision; private TSEncoding[] encodingArray; - private int[] validRowIndex; + // remember the selected index of last not-null value for each column during prepareNext phase. + // It is already transferred by getValueIndex method, so it can be directly used in + // getPrimitiveObject method. + private int[] selectedIndex; public AlignedTVListIterator() { super(); @@ -1504,7 +1507,7 @@ public AlignedTVListIterator( this.allValueColDeletedMap = ignoreAllNullRows ? getAllValueColDeletedMap() : null; this.floatPrecision = floatPrecision; this.encodingArray = encodingList == null ? null : encodingList.toArray(new TSEncoding[0]); - this.validRowIndex = new int[dataTypeList.size()]; + this.selectedIndex = new int[dataTypeList.size()]; } private void prepareNext() { @@ -1525,7 +1528,7 @@ private void prepareNext() { probeNext = true; return; } - Arrays.fill(validRowIndex, rowIndex); + Arrays.fill(selectedIndex, rowIndex); findValidRow = true; } @@ -1538,7 +1541,7 @@ private void prepareNext() { for (int columnIndex = 0; columnIndex < dataTypeArray.length; columnIndex++) { // update currTvPair if the column is not null if (!isNull(rowIndex, columnIndex)) { - validRowIndex[columnIndex] = rowIndex; + selectedIndex[columnIndex] = rowIndex; } } } @@ -1567,7 +1570,7 @@ public TimeValuePair next() { TsPrimitiveType[] vector = new TsPrimitiveType[dataTypeArray.length]; for (int columnIndex = 0; columnIndex < dataTypeArray.length; columnIndex++) { - vector[columnIndex] = getPrimitiveObject(validRowIndex[columnIndex], columnIndex); + vector[columnIndex] = getPrimitiveObject(selectedIndex[columnIndex], columnIndex); } TimeValuePair tvPair = new TimeValuePair(currentTime, TsPrimitiveType.getByType(TSDataType.VECTOR, vector)); @@ -1583,7 +1586,7 @@ public TimeValuePair current() { } TsPrimitiveType[] vector = new TsPrimitiveType[dataTypeArray.length]; for (int columnIndex = 0; columnIndex < dataTypeArray.length; columnIndex++) { - vector[columnIndex] = getPrimitiveObject(validRowIndex[columnIndex], columnIndex); + vector[columnIndex] = getPrimitiveObject(selectedIndex[columnIndex], columnIndex); } return new TimeValuePair(currentTime, TsPrimitiveType.getByType(TSDataType.VECTOR, vector)); } @@ -1650,8 +1653,8 @@ public TsPrimitiveType getPrimitiveObject(int rowIndex, int columnIndex) { } } - public int getValidRowIndex(int columnIndex) { - return validRowIndex[columnIndex]; + public int getSelectedIndex(int columnIndex) { + return selectedIndex[columnIndex]; } @Override @@ -1663,7 +1666,7 @@ public AlignedTVListIterator clone() { iterator.allValueColDeletedMap = allValueColDeletedMap; iterator.floatPrecision = floatPrecision; iterator.encodingArray = this.encodingArray; - iterator.validRowIndex = new int[dataTypeArray.length]; + iterator.selectedIndex = new int[dataTypeArray.length]; iterator.reset(); return iterator; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MergeSortAlignedTVListIterator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MergeSortAlignedTVListIterator.java index 12c098e6eab3e..bf0dfe16c2c4d 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MergeSortAlignedTVListIterator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MergeSortAlignedTVListIterator.java @@ -38,7 +38,10 @@ public class MergeSortAlignedTVListIterator implements IPointReader { private final int[] alignedTvListOffsets; + // Remember the selected columns for prepareNext + // column index -> [selectedTVList, selectedIndex] private final int[][] columnAccessInfo; + private long time; private final BitMap bitMap; @@ -89,7 +92,7 @@ private void prepareNextRow() { if (iterator.hasNext() && iterator.currentTime() <= time) { if (i == 0 || iterator.currentTime() < time) { for (int columnIndex = 0; columnIndex < columnNum; columnIndex++) { - int rowIndex = iterator.getValidRowIndex(columnIndex); + int rowIndex = iterator.getSelectedIndex(columnIndex); columnAccessInfo[columnIndex][0] = i; columnAccessInfo[columnIndex][1] = rowIndex; if (iterator.isNull(rowIndex, columnIndex)) { @@ -99,7 +102,7 @@ private void prepareNextRow() { time = iterator.currentTime(); } else { for (int columnIndex = 0; columnIndex < columnNum; columnIndex++) { - int rowIndex = iterator.getValidRowIndex(columnIndex); + int rowIndex = iterator.getSelectedIndex(columnIndex); // update if the column is not null if (!iterator.isNull(rowIndex, columnIndex)) { columnAccessInfo[columnIndex][0] = i; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/PageColumnAccessInfo.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/PageColumnAccessInfo.java index 1e84934741c74..518bfdc7d010e 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/PageColumnAccessInfo.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/PageColumnAccessInfo.java @@ -22,7 +22,7 @@ import static org.apache.iotdb.db.storageengine.dataregion.memtable.IWritableMemChunk.MAX_NUMBER_OF_POINTS_IN_PAGE; public class PageColumnAccessInfo { - // time -> (selectedTVList, rowIndex) + // time -> (selectedTVList, selectedIndex) private final int[][] indices; private int count; diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemAlignedChunkLoaderTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemAlignedChunkLoaderTest.java index c4297c9b071d2..75a70a4e41956 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemAlignedChunkLoaderTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemAlignedChunkLoaderTest.java @@ -22,13 +22,14 @@ import org.apache.iotdb.db.queryengine.execution.fragment.QueryContext; import org.apache.iotdb.db.storageengine.dataregion.memtable.AlignedReadOnlyMemChunk; import org.apache.iotdb.db.utils.datastructure.AlignedTVList; +import org.apache.iotdb.db.utils.datastructure.MergeSortAlignedTVListIterator; import org.apache.tsfile.common.conf.TSFileConfig; import org.apache.tsfile.enums.TSDataType; import org.apache.tsfile.file.metadata.AlignedChunkMetadata; import org.apache.tsfile.file.metadata.ChunkMetadata; -import org.apache.tsfile.file.metadata.enums.TSEncoding; import org.apache.tsfile.file.metadata.statistics.Statistics; +import org.apache.tsfile.file.metadata.statistics.TimeStatistics; import org.apache.tsfile.read.common.BatchData; import org.apache.tsfile.read.common.block.TsBlock; import org.apache.tsfile.read.reader.IPageReader; @@ -70,7 +71,7 @@ public void testMemAlignedChunkLoader() throws IOException { // Mock getTimeStatisticsList & getValuesStatisticsList List> timeStatitsticsList = new ArrayList<>(); - Statistics timeStatistics = Mockito.mock(Statistics.class); + Statistics timeStatistics = Mockito.mock(TimeStatistics.class); Mockito.when(timeStatistics.getCount()).thenReturn(2L); timeStatitsticsList.add(timeStatistics); Mockito.when(chunk.getTimeStatisticsList()).thenReturn(timeStatitsticsList); @@ -101,18 +102,19 @@ public void testMemAlignedChunkLoader() throws IOException { // Mock AlignedReadOnlyMemChunk Getter List pageOffsets = Arrays.asList(new int[] {0, 0}, new int[] {2, 1}); Mockito.when(chunk.getPageOffsetsList()).thenReturn(pageOffsets); - - List encodingList = - Arrays.asList(new TSEncoding[] {null, null, null, null, null, null}); - - Mockito.when(chunk.getEncodingList()).thenReturn(encodingList); - Mockito.when(chunk.getDataTypes()).thenReturn(buildTsDataTypes()); - Mockito.when(chunk.getColumnIndexList()).thenReturn(null); + List dataTypes = buildTsDataTypes(); + Mockito.when(chunk.getDataTypes()).thenReturn(dataTypes); Mockito.when(chunk.getTimeColumnDeletion()).thenReturn(null); Mockito.when(chunk.getValueColumnsDeletionList()).thenReturn(null); - Mockito.when(chunk.getAligendTvListQueryMap()).thenReturn(buildAlignedTvListMap()); Mockito.when(chunk.getContext()).thenReturn(ctx); + Map alignedTvListMap = buildAlignedTvListMap(); + Mockito.when(chunk.getAligendTvListQueryMap()).thenReturn(alignedTvListMap); + List alignedTvLists = new ArrayList<>(alignedTvListMap.keySet()); + MergeSortAlignedTVListIterator timeValuePairIterator = + new MergeSortAlignedTVListIterator(alignedTvLists, dataTypes, null, null, null, false); + Mockito.when(chunk.getMergeSortAlignedTVListIterator()).thenReturn(timeValuePairIterator); + AlignedChunkMetadata chunkMetadata1 = Mockito.mock(AlignedChunkMetadata.class); Mockito.when(chunk.getChunkMetaData()).thenReturn(chunkMetadata1); diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemChunkLoaderTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemChunkLoaderTest.java index ce715e21fa611..003f88ffef918 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemChunkLoaderTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemChunkLoaderTest.java @@ -21,11 +21,18 @@ import org.apache.iotdb.db.queryengine.execution.fragment.QueryContext; import org.apache.iotdb.db.storageengine.dataregion.memtable.ReadOnlyMemChunk; +import org.apache.iotdb.db.utils.datastructure.MergeSortTvListIterator; import org.apache.iotdb.db.utils.datastructure.TVList; import org.apache.tsfile.common.conf.TSFileConfig; import org.apache.tsfile.enums.TSDataType; import org.apache.tsfile.file.metadata.ChunkMetadata; +import org.apache.tsfile.file.metadata.statistics.BinaryStatistics; +import org.apache.tsfile.file.metadata.statistics.BooleanStatistics; +import org.apache.tsfile.file.metadata.statistics.DoubleStatistics; +import org.apache.tsfile.file.metadata.statistics.FloatStatistics; +import org.apache.tsfile.file.metadata.statistics.IntegerStatistics; +import org.apache.tsfile.file.metadata.statistics.LongStatistics; import org.apache.tsfile.file.metadata.statistics.Statistics; import org.apache.tsfile.read.common.BatchData; import org.apache.tsfile.read.common.block.TsBlock; @@ -35,6 +42,8 @@ import org.mockito.Mockito; import java.io.IOException; +import java.io.Serializable; +import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; import java.util.LinkedHashMap; @@ -55,9 +64,14 @@ public class MemChunkLoaderTest { public void testBooleanMemChunkLoader() throws IOException { ReadOnlyMemChunk chunk = Mockito.mock(ReadOnlyMemChunk.class); Mockito.when(chunk.getDataType()).thenReturn(TSDataType.BOOLEAN); - Mockito.when(chunk.getTvListQueryMap()).thenReturn(buildBooleanTvListMap()); - ChunkMetadata chunkMetadata = Mockito.mock(ChunkMetadata.class); + Map booleanTvListMap = buildBooleanTvListMap(); + Mockito.when(chunk.getTvListQueryMap()).thenReturn(booleanTvListMap); + List booleanTvLists = new ArrayList<>(booleanTvListMap.keySet()); + MergeSortTvListIterator timeValuePairIterator = + new MergeSortTvListIterator(booleanTvLists, null, null); + Mockito.when(chunk.getMergeSortTVListIterator()).thenReturn(timeValuePairIterator); + ChunkMetadata chunkMetadata = Mockito.mock(ChunkMetadata.class); MemChunkLoader memChunkLoader = new MemChunkLoader(new QueryContext(), chunk); try { memChunkLoader.loadChunk(chunkMetadata); @@ -66,8 +80,8 @@ public void testBooleanMemChunkLoader() throws IOException { assertNull(e.getMessage()); } - Statistics pageStatitstics = Mockito.mock(Statistics.class); - List pageStats = Arrays.asList(pageStatitstics); + Statistics pageStatistics = Mockito.mock(BooleanStatistics.class); + List> pageStats = Collections.singletonList(pageStatistics); List pageOffsets = Arrays.asList(new int[] {0, 0}, new int[] {2, 1}); Mockito.when(chunk.getPageStatisticsList()).thenReturn(pageStats); Mockito.when(chunk.getPageOffsetsList()).thenReturn(pageOffsets); @@ -75,10 +89,10 @@ public void testBooleanMemChunkLoader() throws IOException { ChunkMetadata chunkMetadata1 = Mockito.mock(ChunkMetadata.class); Mockito.when(chunk.getChunkMetaData()).thenReturn(chunkMetadata1); Mockito.when(chunk.getPointReader()).thenReturn(null); - Statistics statistics = Mockito.mock(Statistics.class); + Statistics statistics = Mockito.mock(BooleanStatistics.class); Mockito.when(statistics.getCount()).thenReturn(3L); - Mockito.when(chunkMetadata1.getStatistics()).thenReturn(statistics); + Mockito.doReturn(statistics).when(chunkMetadata1).getStatistics(); Mockito.when(chunkMetadata1.getDataType()).thenReturn(TSDataType.BOOLEAN); MemChunkReader chunkReader = @@ -130,9 +144,14 @@ private Map buildBooleanTvListMap() { public void testInt32MemChunkLoader() throws IOException { ReadOnlyMemChunk chunk = Mockito.mock(ReadOnlyMemChunk.class); Mockito.when(chunk.getDataType()).thenReturn(TSDataType.INT32); - Mockito.when(chunk.getTvListQueryMap()).thenReturn(buildInt32TvListMap()); - ChunkMetadata chunkMetadata = Mockito.mock(ChunkMetadata.class); + Map int32TvListMap = buildInt32TvListMap(); + Mockito.when(chunk.getTvListQueryMap()).thenReturn(int32TvListMap); + List int32TvLists = new ArrayList<>(int32TvListMap.keySet()); + MergeSortTvListIterator timeValuePairIterator = + new MergeSortTvListIterator(int32TvLists, null, null); + Mockito.when(chunk.getMergeSortTVListIterator()).thenReturn(timeValuePairIterator); + ChunkMetadata chunkMetadata = Mockito.mock(ChunkMetadata.class); MemChunkLoader memChunkLoader = new MemChunkLoader(new QueryContext(), chunk); try { memChunkLoader.loadChunk(chunkMetadata); @@ -141,8 +160,8 @@ public void testInt32MemChunkLoader() throws IOException { assertNull(e.getMessage()); } - Statistics pageStatitstics = Mockito.mock(Statistics.class); - List pageStats = Arrays.asList(pageStatitstics); + Statistics pageStatistics = Mockito.mock(IntegerStatistics.class); + List> pageStats = Collections.singletonList(pageStatistics); List pageOffsets = Arrays.asList(new int[] {0, 0}, new int[] {2, 1}); Mockito.when(chunk.getPageStatisticsList()).thenReturn(pageStats); Mockito.when(chunk.getPageOffsetsList()).thenReturn(pageOffsets); @@ -150,10 +169,10 @@ public void testInt32MemChunkLoader() throws IOException { ChunkMetadata chunkMetadata1 = Mockito.mock(ChunkMetadata.class); Mockito.when(chunk.getChunkMetaData()).thenReturn(chunkMetadata1); Mockito.when(chunk.getPointReader()).thenReturn(null); - Statistics statistics = Mockito.mock(Statistics.class); + Statistics statistics = Mockito.mock(IntegerStatistics.class); Mockito.when(statistics.getCount()).thenReturn(2L); - Mockito.when(chunkMetadata1.getStatistics()).thenReturn(statistics); + Mockito.doReturn(statistics).when(chunkMetadata1).getStatistics(); Mockito.when(chunkMetadata1.getDataType()).thenReturn(TSDataType.INT32); MemChunkReader chunkReader = @@ -205,9 +224,14 @@ private Map buildInt32TvListMap() { public void testInt64MemChunkLoader() throws IOException { ReadOnlyMemChunk chunk = Mockito.mock(ReadOnlyMemChunk.class); Mockito.when(chunk.getDataType()).thenReturn(TSDataType.INT64); - Mockito.when(chunk.getTvListQueryMap()).thenReturn(buildInt64TvListMap()); - ChunkMetadata chunkMetadata = Mockito.mock(ChunkMetadata.class); + Map int64TvListMap = buildInt64TvListMap(); + Mockito.when(chunk.getTvListQueryMap()).thenReturn(int64TvListMap); + List int64TvLists = new ArrayList<>(int64TvListMap.keySet()); + MergeSortTvListIterator timeValuePairIterator = + new MergeSortTvListIterator(int64TvLists, null, null); + Mockito.when(chunk.getMergeSortTVListIterator()).thenReturn(timeValuePairIterator); + ChunkMetadata chunkMetadata = Mockito.mock(ChunkMetadata.class); MemChunkLoader memChunkLoader = new MemChunkLoader(new QueryContext(), chunk); try { memChunkLoader.loadChunk(chunkMetadata); @@ -216,8 +240,8 @@ public void testInt64MemChunkLoader() throws IOException { assertNull(e.getMessage()); } - Statistics pageStatitstics = Mockito.mock(Statistics.class); - List pageStats = Arrays.asList(pageStatitstics); + Statistics pageStatistics = Mockito.mock(LongStatistics.class); + List> pageStats = Collections.singletonList(pageStatistics); List pageOffsets = Arrays.asList(new int[] {0, 0}, new int[] {2, 1}); Mockito.when(chunk.getPageStatisticsList()).thenReturn(pageStats); Mockito.when(chunk.getPageOffsetsList()).thenReturn(pageOffsets); @@ -225,10 +249,10 @@ public void testInt64MemChunkLoader() throws IOException { ChunkMetadata chunkMetadata1 = Mockito.mock(ChunkMetadata.class); Mockito.when(chunk.getChunkMetaData()).thenReturn(chunkMetadata1); Mockito.when(chunk.getPointReader()).thenReturn(null); - Statistics statistics = Mockito.mock(Statistics.class); + Statistics statistics = Mockito.mock(LongStatistics.class); Mockito.when(statistics.getCount()).thenReturn(2L); - Mockito.when(chunkMetadata1.getStatistics()).thenReturn(statistics); + Mockito.doReturn(statistics).when(chunkMetadata1).getStatistics(); Mockito.when(chunkMetadata1.getDataType()).thenReturn(TSDataType.INT64); MemChunkReader chunkReader = @@ -280,9 +304,14 @@ private Map buildInt64TvListMap() { public void testFloatMemChunkLoader() throws IOException { ReadOnlyMemChunk chunk = Mockito.mock(ReadOnlyMemChunk.class); Mockito.when(chunk.getDataType()).thenReturn(TSDataType.FLOAT); - Mockito.when(chunk.getTvListQueryMap()).thenReturn(buildFloatTvListMap()); - ChunkMetadata chunkMetadata = Mockito.mock(ChunkMetadata.class); + Map floatTvListMap = buildFloatTvListMap(); + Mockito.when(chunk.getTvListQueryMap()).thenReturn(floatTvListMap); + List floatTvLists = new ArrayList<>(floatTvListMap.keySet()); + MergeSortTvListIterator timeValuePairIterator = + new MergeSortTvListIterator(floatTvLists, null, null); + Mockito.when(chunk.getMergeSortTVListIterator()).thenReturn(timeValuePairIterator); + ChunkMetadata chunkMetadata = Mockito.mock(ChunkMetadata.class); MemChunkLoader memChunkLoader = new MemChunkLoader(new QueryContext(), chunk); try { memChunkLoader.loadChunk(chunkMetadata); @@ -291,8 +320,8 @@ public void testFloatMemChunkLoader() throws IOException { assertNull(e.getMessage()); } - Statistics pageStatitstics = Mockito.mock(Statistics.class); - List pageStats = Arrays.asList(pageStatitstics); + Statistics pageStatistics = Mockito.mock(FloatStatistics.class); + List> pageStats = Collections.singletonList(pageStatistics); List pageOffsets = Arrays.asList(new int[] {0, 0}, new int[] {2, 1}); Mockito.when(chunk.getPageStatisticsList()).thenReturn(pageStats); Mockito.when(chunk.getPageOffsetsList()).thenReturn(pageOffsets); @@ -300,10 +329,10 @@ public void testFloatMemChunkLoader() throws IOException { ChunkMetadata chunkMetadata1 = Mockito.mock(ChunkMetadata.class); Mockito.when(chunk.getChunkMetaData()).thenReturn(chunkMetadata1); Mockito.when(chunk.getPointReader()).thenReturn(null); - Statistics statistics = Mockito.mock(Statistics.class); + Statistics statistics = Mockito.mock(FloatStatistics.class); Mockito.when(statistics.getCount()).thenReturn(2L); - Mockito.when(chunkMetadata1.getStatistics()).thenReturn(statistics); + Mockito.doReturn(statistics).when(chunkMetadata1).getStatistics(); Mockito.when(chunkMetadata1.getDataType()).thenReturn(TSDataType.FLOAT); MemChunkReader chunkReader = @@ -355,9 +384,14 @@ private Map buildFloatTvListMap() { public void testDoubleMemChunkLoader() throws IOException { ReadOnlyMemChunk chunk = Mockito.mock(ReadOnlyMemChunk.class); Mockito.when(chunk.getDataType()).thenReturn(TSDataType.DOUBLE); - Mockito.when(chunk.getTvListQueryMap()).thenReturn(buildDoubleTvListMap()); - ChunkMetadata chunkMetadata = Mockito.mock(ChunkMetadata.class); + Map doubleTvListMap = buildDoubleTvListMap(); + Mockito.when(chunk.getTvListQueryMap()).thenReturn(doubleTvListMap); + List doubleTvLists = new ArrayList<>(doubleTvListMap.keySet()); + MergeSortTvListIterator timeValuePairIterator = + new MergeSortTvListIterator(doubleTvLists, null, null); + Mockito.when(chunk.getMergeSortTVListIterator()).thenReturn(timeValuePairIterator); + ChunkMetadata chunkMetadata = Mockito.mock(ChunkMetadata.class); MemChunkLoader memChunkLoader = new MemChunkLoader(new QueryContext(), chunk); try { memChunkLoader.loadChunk(chunkMetadata); @@ -366,8 +400,8 @@ public void testDoubleMemChunkLoader() throws IOException { assertNull(e.getMessage()); } - Statistics pageStatitstics = Mockito.mock(Statistics.class); - List pageStats = Arrays.asList(pageStatitstics); + Statistics pageStatistics = Mockito.mock(DoubleStatistics.class); + List> pageStats = Collections.singletonList(pageStatistics); List pageOffsets = Arrays.asList(new int[] {0, 0}, new int[] {2, 1}); Mockito.when(chunk.getPageStatisticsList()).thenReturn(pageStats); Mockito.when(chunk.getPageOffsetsList()).thenReturn(pageOffsets); @@ -375,10 +409,10 @@ public void testDoubleMemChunkLoader() throws IOException { ChunkMetadata chunkMetadata1 = Mockito.mock(ChunkMetadata.class); Mockito.when(chunk.getChunkMetaData()).thenReturn(chunkMetadata1); Mockito.when(chunk.getPointReader()).thenReturn(null); - Statistics statistics = Mockito.mock(Statistics.class); + Statistics statistics = Mockito.mock(DoubleStatistics.class); Mockito.when(statistics.getCount()).thenReturn(2L); - Mockito.when(chunkMetadata1.getStatistics()).thenReturn(statistics); + Mockito.doReturn(statistics).when(chunkMetadata1).getStatistics(); Mockito.when(chunkMetadata1.getDataType()).thenReturn(TSDataType.DOUBLE); MemChunkReader chunkReader = @@ -430,9 +464,14 @@ private Map buildDoubleTvListMap() { public void testTextMemChunkLoader() throws IOException { ReadOnlyMemChunk chunk = Mockito.mock(ReadOnlyMemChunk.class); Mockito.when(chunk.getDataType()).thenReturn(TSDataType.TEXT); - Mockito.when(chunk.getTvListQueryMap()).thenReturn(buildTextTvListMap()); - ChunkMetadata chunkMetadata = Mockito.mock(ChunkMetadata.class); + Map textTvListMap = buildTextTvListMap(); + Mockito.when(chunk.getTvListQueryMap()).thenReturn(textTvListMap); + List textTvLists = new ArrayList<>(textTvListMap.keySet()); + MergeSortTvListIterator timeValuePairIterator = + new MergeSortTvListIterator(textTvLists, null, null); + Mockito.when(chunk.getMergeSortTVListIterator()).thenReturn(timeValuePairIterator); + ChunkMetadata chunkMetadata = Mockito.mock(ChunkMetadata.class); MemChunkLoader memChunkLoader = new MemChunkLoader(new QueryContext(), chunk); try { memChunkLoader.loadChunk(chunkMetadata); @@ -441,8 +480,8 @@ public void testTextMemChunkLoader() throws IOException { assertNull(e.getMessage()); } - Statistics pageStatitstics = Mockito.mock(Statistics.class); - List pageStats = Arrays.asList(pageStatitstics); + Statistics pageStatistics = Mockito.mock(BinaryStatistics.class); + List> pageStats = Collections.singletonList(pageStatistics); List pageOffsets = Arrays.asList(new int[] {0, 0}, new int[] {2, 1}); Mockito.when(chunk.getPageStatisticsList()).thenReturn(pageStats); Mockito.when(chunk.getPageOffsetsList()).thenReturn(pageOffsets); @@ -450,10 +489,10 @@ public void testTextMemChunkLoader() throws IOException { ChunkMetadata chunkMetadata1 = Mockito.mock(ChunkMetadata.class); Mockito.when(chunk.getChunkMetaData()).thenReturn(chunkMetadata1); Mockito.when(chunk.getPointReader()).thenReturn(null); - Statistics statistics = Mockito.mock(Statistics.class); + Statistics statistics = Mockito.mock(BinaryStatistics.class); Mockito.when(statistics.getCount()).thenReturn(2L); - Mockito.when(chunkMetadata1.getStatistics()).thenReturn(statistics); + Mockito.doReturn(statistics).when(chunkMetadata1).getStatistics(); Mockito.when(chunkMetadata1.getDataType()).thenReturn(TSDataType.TEXT); MemChunkReader chunkReader = From 36aa04d97216d6a450fb9b72804cf3d83ed23db9 Mon Sep 17 00:00:00 2001 From: shizy Date: Thu, 2 Jan 2025 15:23:51 +0800 Subject: [PATCH 33/54] fix: * remove delete method in BinaryTVList * filter deleted data in WritableMemChunk encode --- .../dataregion/memtable/WritableMemChunk.java | 4 +++ .../db/utils/datastructure/BinaryTVList.java | 26 ------------------- 2 files changed, 4 insertions(+), 26 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/WritableMemChunk.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/WritableMemChunk.java index 23d53ebb715cb..a142de8932c56 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/WritableMemChunk.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/WritableMemChunk.java @@ -459,6 +459,10 @@ public void encodeWorkingTVList(BlockingQueue ioTaskQueue) { long dataSizeInCurrentChunk = 0; int pointNumInCurrentChunk = 0; for (int sortedRowIndex = 0; sortedRowIndex < list.rowCount(); sortedRowIndex++) { + if (list.isNullValue(list.getValueIndex(sortedRowIndex))) { + continue; + } + long time = list.getTime(sortedRowIndex); // skip duplicated data diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BinaryTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BinaryTVList.java index a298ced525629..c7453f9f76815 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BinaryTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BinaryTVList.java @@ -100,32 +100,6 @@ public synchronized void putBinary(long timestamp, Binary value) { } } - @Override - public int delete(long lowerBound, long upperBound) { - int newSize = 0; - maxTime = Long.MIN_VALUE; - for (int i = 0; i < rowCount; i++) { - long time = getTime(i); - if (time < lowerBound || time > upperBound) { - set(i, newSize++); - maxTime = Math.max(maxTime, time); - } - } - int deletedNumber = rowCount - newSize; - rowCount = newSize; - // release primitive arrays that are empty - int newArrayNum = newSize / ARRAY_SIZE; - if (newSize % ARRAY_SIZE != 0) { - newArrayNum++; - } - int oldArrayNum = timestamps.size(); - for (int releaseIdx = newArrayNum; releaseIdx < oldArrayNum; releaseIdx++) { - releaseLastTimeArray(); - releaseLastValueArray(); - } - return deletedNumber; - } - @Override public Binary getBinary(int index) { if (index >= rowCount) { From 809ebd59097e40b7e3e06af48976fce6e6abebaa Mon Sep 17 00:00:00 2001 From: shizy Date: Thu, 2 Jan 2025 18:23:47 +0800 Subject: [PATCH 34/54] fix: remove getSortedTvListForQuery in SeriesRegionScan --- .../dataregion/memtable/AbstractMemTable.java | 89 ++++++------------- .../memtable/AlignedWritableMemChunk.java | 67 ++++++++++++++ .../dataregion/memtable/WritableMemChunk.java | 41 +++++++++ .../impl/MemAlignedChunkHandleImpl.java | 21 ++--- 4 files changed, 140 insertions(+), 78 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AbstractMemTable.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AbstractMemTable.java index fbd4a3104c5b8..4859cf77b80e4 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AbstractMemTable.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AbstractMemTable.java @@ -43,8 +43,6 @@ import org.apache.iotdb.db.storageengine.dataregion.wal.utils.WALWriteUtils; import org.apache.iotdb.db.utils.MemUtils; import org.apache.iotdb.db.utils.ModificationUtils; -import org.apache.iotdb.db.utils.datastructure.AlignedTVList; -import org.apache.iotdb.db.utils.datastructure.TVList; import org.apache.tsfile.enums.TSDataType; import org.apache.tsfile.file.metadata.ChunkMetadata; @@ -64,7 +62,6 @@ import java.io.DataInputStream; import java.io.IOException; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.List; @@ -74,9 +71,6 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; import java.util.stream.Collectors; -import java.util.stream.LongStream; - -import static org.apache.iotdb.db.storageengine.rescon.memory.PrimitiveArrayManager.ARRAY_SIZE; public abstract class AbstractMemTable implements IMemTable { @@ -537,10 +531,10 @@ private void getMemChunkHandleFromMemTable( Map> memChunkHandleMap, List deletionList) { - IWritableMemChunk memChunk = memTableMap.get(deviceID).getMemChunkMap().get(measurementId); + WritableMemChunk memChunk = + (WritableMemChunk) memTableMap.get(deviceID).getMemChunkMap().get(measurementId); - TVList tvListCopy = memChunk.getSortedTvListForQuery(); - long[] timestamps = filterDeletedTimestamp(tvListCopy, deletionList); + long[] timestamps = memChunk.getFilteredTimestamp(deletionList); chunkMetadataMap .computeIfAbsent(measurementId, k -> new ArrayList<>()) @@ -548,7 +542,7 @@ private void getMemChunkHandleFromMemTable( buildChunkMetaDataForMemoryChunk( measurementId, timestamps[0], - timestamps[tvListCopy.count() - 1], + timestamps[timestamps.length - 1], Collections.emptyList())); memChunkHandleMap .computeIfAbsent(measurementId, k -> new ArrayList<>()) @@ -576,12 +570,13 @@ private void getMemAlignedChunkHandleFromMemTable( return; } - AlignedTVList alignedTVListCopy = - (AlignedTVList) alignedMemChunk.getSortedTvListForQuery(schemaList, true); + List bitMaps = new ArrayList<>(); + long[] timestamps = alignedMemChunk.getFilteredTimestamp(deletionList, bitMaps, true); buildAlignedMemChunkHandle( deviceID, - alignedTVListCopy, + timestamps, + bitMaps, deletionList, schemaList, chunkMetadataList, @@ -599,9 +594,6 @@ private void getMemAlignedChunkHandleFromMemTable( AlignedWritableMemChunk memChunk = writableMemChunkGroup.getAlignedMemChunk(); List schemaList = memChunk.getSchemaList(); - AlignedTVList alignedTVListCopy = - (AlignedTVList) memChunk.getSortedTvListForQuery(schemaList, true); - List> deletionList = new ArrayList<>(); if (modsToMemTabled != null) { for (IMeasurementSchema schema : schemaList) { @@ -610,9 +602,13 @@ private void getMemAlignedChunkHandleFromMemTable( deviceID, schema.getMeasurementName(), this, modsToMemTabled, ttlLowerBound)); } } + + List bitMaps = new ArrayList<>(); + long[] timestamps = memChunk.getFilteredTimestamp(deletionList, bitMaps, true); buildAlignedMemChunkHandle( deviceID, - alignedTVListCopy, + timestamps, + bitMaps, deletionList, schemaList, chunkMetadataList, @@ -631,8 +627,7 @@ private void getMemChunkHandleFromMemTable( writableMemChunkGroup.getMemChunkMap().entrySet()) { String measurementId = entry.getKey(); - IWritableMemChunk writableMemChunk = entry.getValue(); - TVList tvListCopy = writableMemChunk.getSortedTvListForQuery(); + WritableMemChunk writableMemChunk = (WritableMemChunk) entry.getValue(); List deletionList = new ArrayList<>(); if (modsToMemTabled != null) { @@ -640,14 +635,14 @@ private void getMemChunkHandleFromMemTable( ModificationUtils.constructDeletionList( deviceID, measurementId, this, modsToMemTabled, ttlLowerBound); } - long[] timestamps = filterDeletedTimestamp(tvListCopy, deletionList); + long[] timestamps = writableMemChunk.getFilteredTimestamp(deletionList); chunkMetadataMap .computeIfAbsent(measurementId, k -> new ArrayList<>()) .add( buildChunkMetaDataForMemoryChunk( measurementId, timestamps[0], - timestamps[tvListCopy.count() - 1], + timestamps[timestamps.length - 1], Collections.emptyList())); memChunkHandleMap .computeIfAbsent(measurementId, k -> new ArrayList<>()) @@ -657,26 +652,21 @@ private void getMemChunkHandleFromMemTable( private void buildAlignedMemChunkHandle( IDeviceID deviceID, - AlignedTVList alignedTVList, + long[] timestamps, + List bitMaps, List> deletionList, List schemaList, Map> chunkMetadataList, Map> chunkHandleMap) { - List> bitMaps = alignedTVList.getBitMaps(); - long[] timestamps = - alignedTVList.getTimestamps().stream().flatMapToLong(LongStream::of).toArray(); - timestamps = Arrays.copyOfRange(timestamps, 0, alignedTVList.rowCount()); - - for (int i = 0; i < schemaList.size(); i++) { - String measurement = schemaList.get(i).getMeasurementName(); - List curBitMap = bitMaps == null ? Collections.emptyList() : bitMaps.get(i); + for (int column = 0; column < schemaList.size(); column++) { + String measurement = schemaList.get(column).getMeasurementName(); List deletion = deletionList == null || deletionList.isEmpty() ? Collections.emptyList() - : deletionList.get(i); + : deletionList.get(column); - long[] startEndTime = calculateStartEndTime(timestamps, curBitMap); + long[] startEndTime = calculateStartEndTime(timestamps, bitMaps, column); chunkMetadataList .computeIfAbsent(measurement, k -> new ArrayList<>()) .add( @@ -686,28 +676,24 @@ private void buildAlignedMemChunkHandle( .computeIfAbsent(measurement, k -> new ArrayList<>()) .add( new MemAlignedChunkHandleImpl( - deviceID, measurement, timestamps, curBitMap, deletion, startEndTime)); + deviceID, measurement, timestamps, bitMaps, column, startEndTime)); } } - private long[] calculateStartEndTime(long[] timestamps, List bitMaps) { + private long[] calculateStartEndTime(long[] timestamps, List bitMaps, int column) { if (bitMaps.isEmpty()) { return new long[] {timestamps[0], timestamps[timestamps.length - 1]}; } long startTime = -1, endTime = -1; for (int i = 0; i < timestamps.length; i++) { - int arrayIndex = i / ARRAY_SIZE; - int elementIndex = i % ARRAY_SIZE; - if (!bitMaps.get(arrayIndex).isMarked(elementIndex)) { + if (!bitMaps.get(i).isMarked(column)) { startTime = timestamps[i]; break; } } for (int i = timestamps.length - 1; i >= 0; i--) { - int arrayIndex = i / ARRAY_SIZE; - int elementIndex = i % ARRAY_SIZE; - if (!bitMaps.get(arrayIndex).isMarked(elementIndex)) { + if (!bitMaps.get(i).isMarked(column)) { endTime = timestamps[i]; break; } @@ -736,29 +722,6 @@ private IChunkMetadata buildChunkMetaDataForMemoryChunk( return chunkMetadata; } - private long[] filterDeletedTimestamp(TVList tvList, List deletionList) { - if (tvList.getBitMap() == null && deletionList.isEmpty()) { - long[] timestamps = tvList.getTimestamps().stream().flatMapToLong(LongStream::of).toArray(); - return Arrays.copyOfRange(timestamps, 0, tvList.rowCount()); - } - - long lastTime = -1; - int[] deletionCursor = {0}; - int rowCount = tvList.rowCount(); - List result = new ArrayList<>(); - - for (int i = 0; i < rowCount; i++) { - long curTime = tvList.getTime(i); - if (!tvList.isNullValue(i) - && !ModificationUtils.isPointDeleted(curTime, deletionList, deletionCursor) - && (i == rowCount - 1 || curTime != lastTime)) { - result.add(curTime); - } - lastTime = curTime; - } - return result.stream().mapToLong(Long::longValue).toArray(); - } - @Override public long delete(ModEntry modEntry) { List> targetDeviceList = new ArrayList<>(); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedWritableMemChunk.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedWritableMemChunk.java index 72511ca8f85fa..142cacebcaf76 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedWritableMemChunk.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedWritableMemChunk.java @@ -33,6 +33,7 @@ import org.apache.iotdb.db.utils.datastructure.TVList; import org.apache.tsfile.enums.TSDataType; +import org.apache.tsfile.read.common.TimeRange; import org.apache.tsfile.utils.Binary; import org.apache.tsfile.utils.BitMap; import org.apache.tsfile.utils.Pair; @@ -54,8 +55,11 @@ import java.util.Map; import java.util.Objects; import java.util.Set; +import java.util.TreeMap; import java.util.concurrent.BlockingQueue; +import static org.apache.iotdb.db.utils.ModificationUtils.isPointDeleted; + public class AlignedWritableMemChunk implements IWritableMemChunk { private final Map measurementIndexMap; @@ -285,6 +289,69 @@ private Pair checkAndReorderColumnValuesInInsertPlan( return new Pair<>(reorderedColumnValues, reorderedBitMaps); } + private void filterDeletedTimeStamp( + AlignedTVList alignedTVList, + List> valueColumnsDeletionList, + boolean ignoreAllNullRows, + Map timestampWithBitmap) { + BitMap allValueColDeletedMap = alignedTVList.getAllValueColDeletedMap(); + + int rowCount = alignedTVList.rowCount(); + List valueColumnDeleteCursor = new ArrayList<>(); + if (valueColumnsDeletionList != null) { + valueColumnsDeletionList.forEach(x -> valueColumnDeleteCursor.add(new int[] {0})); + } + + for (int row = 0; row < rowCount; row++) { + // the row is deleted + if (allValueColDeletedMap != null && allValueColDeletedMap.isMarked(row)) { + continue; + } + long timestamp = alignedTVList.getTime(row); + + BitMap bitMap = new BitMap(schemaList.size()); + bitMap.markAll(); + for (int column = 0; column < schemaList.size(); column++) { + if (!alignedTVList.isNullValue(alignedTVList.getValueIndex(row), column)) { + bitMap.unmark(column); + } + + // skip deleted row + if (valueColumnsDeletionList != null + && !valueColumnsDeletionList.isEmpty() + && isPointDeleted( + timestamp, + valueColumnsDeletionList.get(column), + valueColumnDeleteCursor.get(column))) { + bitMap.mark(column); + } + + // skip all-null row + if (ignoreAllNullRows && bitMap.isAllMarked()) { + continue; + } + timestampWithBitmap.put(timestamp, bitMap); + } + } + } + + public long[] getFilteredTimestamp( + List> deletionList, List bitMaps, boolean ignoreAllNullRows) { + Map timestampWithBitmap = new TreeMap<>(); + + filterDeletedTimeStamp(list, deletionList, ignoreAllNullRows, timestampWithBitmap); + for (AlignedTVList alignedTVList : sortedList) { + filterDeletedTimeStamp(alignedTVList, deletionList, ignoreAllNullRows, timestampWithBitmap); + } + + List filteredTimestamps = new ArrayList<>(); + for (Map.Entry entry : timestampWithBitmap.entrySet()) { + filteredTimestamps.add(entry.getKey()); + bitMaps.add(entry.getValue()); + } + return filteredTimestamps.stream().mapToLong(Long::valueOf).toArray(); + } + @Override public AlignedTVList getWorkingTVList() { return list; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/WritableMemChunk.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/WritableMemChunk.java index a142de8932c56..c2fe8f1e1033d 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/WritableMemChunk.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/WritableMemChunk.java @@ -24,11 +24,13 @@ import org.apache.iotdb.db.queryengine.execution.fragment.QueryContext; import org.apache.iotdb.db.queryengine.plan.planner.memory.MemoryReservationManager; import org.apache.iotdb.db.storageengine.dataregion.wal.buffer.IWALByteBufferView; +import org.apache.iotdb.db.utils.ModificationUtils; import org.apache.iotdb.db.utils.datastructure.MergeSortTvListIterator; import org.apache.iotdb.db.utils.datastructure.TVList; import org.apache.tsfile.enums.TSDataType; import org.apache.tsfile.read.TimeValuePair; +import org.apache.tsfile.read.common.TimeRange; import org.apache.tsfile.utils.Binary; import org.apache.tsfile.utils.BitMap; import org.apache.tsfile.utils.Pair; @@ -43,8 +45,10 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.util.ArrayList; +import java.util.Arrays; import java.util.List; import java.util.concurrent.BlockingQueue; +import java.util.stream.Collectors; import static org.apache.iotdb.db.utils.MemUtils.getBinarySize; @@ -330,6 +334,43 @@ public synchronized void sortTvListForFlush() { } } + private void filterDeletedTimestamp( + TVList tvlist, List deletionList, List timestampList) { + long lastTime = Long.MIN_VALUE; + int[] deletionCursor = {0}; + int rowCount = tvlist.rowCount(); + for (int i = 0; i < rowCount; i++) { + if (tvlist.getBitMap() != null && tvlist.isNullValue(i)) { + continue; + } + long curTime = tvlist.getTime(i); + if (deletionList != null + && ModificationUtils.isPointDeleted(curTime, deletionList, deletionCursor)) { + continue; + } + + if (i == rowCount - 1 || curTime != lastTime) { + timestampList.add(curTime); + } + lastTime = curTime; + } + } + + public long[] getFilteredTimestamp(List deletionList) { + List timestampList = new ArrayList<>(); + filterDeletedTimestamp(list, deletionList, timestampList); + for (TVList tvList : sortedList) { + filterDeletedTimestamp(tvList, deletionList, timestampList); + } + + // remove duplicated time + List distinctTimestamps = timestampList.stream().distinct().collect(Collectors.toList()); + // sort timestamps + long[] filteredTimestamps = distinctTimestamps.stream().mapToLong(Long::longValue).toArray(); + Arrays.sort(filteredTimestamps); + return filteredTimestamps; + } + @Override public TVList getWorkingTVList() { return list; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/filescan/impl/MemAlignedChunkHandleImpl.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/filescan/impl/MemAlignedChunkHandleImpl.java index f2ad84d2d4809..b3f9fc1f2f228 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/filescan/impl/MemAlignedChunkHandleImpl.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/filescan/impl/MemAlignedChunkHandleImpl.java @@ -19,22 +19,17 @@ package org.apache.iotdb.db.storageengine.dataregion.read.filescan.impl; -import org.apache.iotdb.db.utils.ModificationUtils; - import org.apache.tsfile.file.metadata.IDeviceID; -import org.apache.tsfile.read.common.TimeRange; import org.apache.tsfile.utils.BitMap; import java.io.IOException; import java.util.ArrayList; import java.util.List; -import static org.apache.iotdb.db.storageengine.rescon.memory.PrimitiveArrayManager.ARRAY_SIZE; - public class MemAlignedChunkHandleImpl extends MemChunkHandleImpl { + private final int column; private final List bitMapOfValue; - private final List deletionList; // start time and end time of the chunk according to bitMap private final long[] startEndTime; @@ -43,11 +38,11 @@ public MemAlignedChunkHandleImpl( String measurement, long[] dataOfTimestamp, List bitMapOfValue, - List deletionList, + int column, long[] startEndTime) { super(deviceID, measurement, dataOfTimestamp); this.bitMapOfValue = bitMapOfValue; - this.deletionList = deletionList; + this.column = column; this.startEndTime = startEndTime; } @@ -59,17 +54,13 @@ public long[] getPageStatisticsTime() { @Override public long[] getDataTime() throws IOException { List timeList = new ArrayList<>(); - int[] deletionCursor = {0}; for (int i = 0; i < dataOfTimestamp.length; i++) { - if (!bitMapOfValue.isEmpty()) { - int arrayIndex = i / ARRAY_SIZE; - int elementIndex = i % ARRAY_SIZE; - if (bitMapOfValue.get(arrayIndex).isMarked(elementIndex)) { + if (bitMapOfValue != null) { + if (bitMapOfValue.get(i).isMarked(column)) { continue; } } - if (!ModificationUtils.isPointDeleted(dataOfTimestamp[i], deletionList, deletionCursor) - && (i == dataOfTimestamp.length - 1 || dataOfTimestamp[i] != dataOfTimestamp[i + 1])) { + if (i == dataOfTimestamp.length - 1 || dataOfTimestamp[i] != dataOfTimestamp[i + 1]) { timeList.add(dataOfTimestamp[i]); } } From ab17aea9720106be50db51fefa3589b80f1e554c Mon Sep 17 00:00:00 2001 From: shizy Date: Thu, 2 Jan 2025 22:16:20 +0800 Subject: [PATCH 35/54] fix: TsFileProcessorTest unit test --- .../dataregion/memtable/AlignedWritableMemChunk.java | 1 + .../dataregion/memtable/IWritableMemChunk.java | 3 --- .../storageengine/dataregion/memtable/WritableMemChunk.java | 6 ++++++ 3 files changed, 7 insertions(+), 3 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedWritableMemChunk.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedWritableMemChunk.java index 142cacebcaf76..a65154b624ba6 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedWritableMemChunk.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedWritableMemChunk.java @@ -70,6 +70,7 @@ public class AlignedWritableMemChunk implements IWritableMemChunk { private final boolean ignoreAllNullRows; private static final IoTDBConfig CONFIG = IoTDBDescriptor.getInstance().getConfig(); + private final long TARGET_CHUNK_SIZE = CONFIG.getTargetChunkSize(); private long maxNumberOfPointsInChunk = CONFIG.getTargetChunkPointNum(); private static final String UNSUPPORTED_TYPE = "Unsupported data type:"; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/IWritableMemChunk.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/IWritableMemChunk.java index b72adb66df4a0..cee4e2e302460 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/IWritableMemChunk.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/IWritableMemChunk.java @@ -37,9 +37,6 @@ public interface IWritableMemChunk extends WALEntryValue { int TVLIST_SORT_THRESHOLD = IoTDBDescriptor.getInstance().getConfig().getTvListSortThreshold(); int MAX_NUMBER_OF_POINTS_IN_PAGE = TSFileDescriptor.getInstance().getConfig().getMaxNumberOfPointsInPage(); - long TARGET_CHUNK_SIZE = IoTDBDescriptor.getInstance().getConfig().getTargetChunkSize(); - long MAX_NUMBER_OF_POINTS_IN_CHUNK = - IoTDBDescriptor.getInstance().getConfig().getTargetChunkPointNum(); void putLong(long t, long v); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/WritableMemChunk.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/WritableMemChunk.java index c2fe8f1e1033d..303b9ba76e728 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/WritableMemChunk.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/WritableMemChunk.java @@ -20,6 +20,8 @@ package org.apache.iotdb.db.storageengine.dataregion.memtable; import org.apache.iotdb.common.rpc.thrift.TSStatus; +import org.apache.iotdb.db.conf.IoTDBConfig; +import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.queryengine.execution.fragment.FragmentInstanceContext; import org.apache.iotdb.db.queryengine.execution.fragment.QueryContext; import org.apache.iotdb.db.queryengine.plan.planner.memory.MemoryReservationManager; @@ -61,6 +63,10 @@ public class WritableMemChunk implements IWritableMemChunk { private static final Logger LOGGER = LoggerFactory.getLogger(WritableMemChunk.class); + private static final IoTDBConfig CONFIG = IoTDBDescriptor.getInstance().getConfig(); + private final long TARGET_CHUNK_SIZE = CONFIG.getTargetChunkSize(); + private final long MAX_NUMBER_OF_POINTS_IN_CHUNK = CONFIG.getTargetChunkPointNum(); + public WritableMemChunk(IMeasurementSchema schema) { this.schema = schema; this.list = TVList.newList(schema.getType()); From 044d1ab181ff46d0e2aa442c84b2f086112ff183 Mon Sep 17 00:00:00 2001 From: shizy Date: Thu, 2 Jan 2025 23:26:16 +0800 Subject: [PATCH 36/54] fix: IoTDBNullIdQueryIT.noMeasurementColumnsSelectTest --- .../dataregion/memtable/AlignedReadOnlyMemChunk.java | 2 +- .../dataregion/read/reader/chunk/MemAlignedChunkReader.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedReadOnlyMemChunk.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedReadOnlyMemChunk.java index f685d67f3d0cd..13150370e1093 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedReadOnlyMemChunk.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedReadOnlyMemChunk.java @@ -200,8 +200,8 @@ && isPointDeleted(timestamp, timeColumnDeletion, timeDeleteCursor)) { // prepare column access info for current page int[][] columnAccessInfo = timeValuePairIterator.getColumnAccessInfo(); + time[pointsInPage] = timeValuePairIterator.getTime(); for (int i = 0; i < dataTypes.size(); i++) { - time[pointsInPage] = timeValuePairIterator.getTime(); pageColumnAccessInfo[i].add(columnAccessInfo[i]); } timeValuePairIterator.step(); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemAlignedChunkReader.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemAlignedChunkReader.java index 8ff72e1638ba3..0682b7133ba93 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemAlignedChunkReader.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemAlignedChunkReader.java @@ -290,8 +290,8 @@ && isPointDeleted(timestamp, timeColumnDeletion, timeDeleteCursor)) { // prepare column access info for current page int[][] accessInfo = timeValuePairIterator.getColumnAccessInfo(); + time[pointsInPage] = timeValuePairIterator.getTime(); for (int i = 0; i < tsDataTypes.size(); i++) { - time[pointsInPage] = timeValuePairIterator.getTime(); pageColumnAccessInfo[i].add(accessInfo[i]); } timeValuePairIterator.step(); From 1d2a84068e127beb56c7f2dc24354556afaf8c4f Mon Sep 17 00:00:00 2001 From: shizy Date: Sun, 5 Jan 2025 11:32:42 +0800 Subject: [PATCH 37/54] fix: delete column of aligned time series --- .../memtable/AlignedWritableMemChunk.java | 21 +++++++--- .../db/utils/datastructure/AlignedTVList.java | 39 +++++++++---------- 2 files changed, 34 insertions(+), 26 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedWritableMemChunk.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedWritableMemChunk.java index a65154b624ba6..ae9fd9c59d118 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedWritableMemChunk.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedWritableMemChunk.java @@ -500,14 +500,17 @@ public Pair deleteDataFromAColumn( } public void removeColumn(String measurementId) { - list.deleteColumn(measurementIndexMap.get(measurementId)); + int columnIndex = measurementIndexMap.get(measurementId); + list.deleteColumn(columnIndex); for (AlignedTVList alignedTvList : sortedList) { - alignedTvList.deleteColumn(measurementIndexMap.get(measurementId)); + alignedTvList.deleteColumn(columnIndex); } - IMeasurementSchema schemaToBeRemoved = schemaList.get(measurementIndexMap.get(measurementId)); - schemaList.remove(schemaToBeRemoved); + IMeasurementSchema schemaToBeRemoved = schemaList.get(columnIndex); measurementIndexMap.clear(); for (int i = 0; i < schemaList.size(); i++) { + if (schemaList.get(i).getMeasurementName().equals(schemaToBeRemoved.getMeasurementName())) { + continue; + } measurementIndexMap.put(schemaList.get(i).getMeasurementName(), i); } } @@ -957,7 +960,15 @@ public List getSchemaList() { } public boolean isAllDeleted() { - return list.isAllDeleted(); + if (!list.isAllDeleted()) { + return false; + } + for (AlignedTVList alignedTvList : sortedList) { + if (!alignedTvList.isAllDeleted()) { + return false; + } + } + return true; } public List getSortedList() { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/AlignedTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/AlignedTVList.java index 9749fb3afb90f..5f5822f9434ca 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/AlignedTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/AlignedTVList.java @@ -504,7 +504,7 @@ public boolean isNullValue(int unsortedRowIndex, int columnIndex) { return true; } - if (values.get(columnIndex) == null) { + if (columnIndex < 0 || columnIndex >= values.size() || values.get(columnIndex) == null) { return true; } if (bitMaps == null @@ -584,6 +584,10 @@ private void updateMinTime() { * @return Delete info pair. Left: deletedNumber int; right: ifDeleteColumn boolean */ public Pair delete(long lowerBound, long upperBound, int columnIndex) { + if (columnIndex >= values.size()) { + return new Pair<>(0, false); + } + int deletedNumber = 0; boolean deleteColumn = true; for (int i = 0; i < rowCount; i++) { @@ -592,12 +596,6 @@ public Pair delete(long lowerBound, long upperBound, int colum int originRowIndex = getValueIndex(i); int arrayIndex = originRowIndex / ARRAY_SIZE; int elementIndex = originRowIndex % ARRAY_SIZE; - if (dataTypes.get(columnIndex).isBinary()) { - Binary value = ((Binary[]) values.get(columnIndex).get(arrayIndex))[elementIndex]; - if (value != null) { - memoryBinaryChunkSize[columnIndex] -= getBinarySize(value); - } - } markNullValue(columnIndex, arrayIndex, elementIndex); deletedNumber++; } else { @@ -608,23 +606,22 @@ public Pair delete(long lowerBound, long upperBound, int colum } public void deleteColumn(int columnIndex) { - dataTypes.remove(columnIndex); - - long[] tmpValueChunkRawSize = memoryBinaryChunkSize; - memoryBinaryChunkSize = new long[dataTypes.size()]; - int copyIndex = 0; - for (int i = 0; i < tmpValueChunkRawSize.length; i++) { - if (i == columnIndex) { - continue; + if (bitMaps == null) { + bitMaps = new ArrayList<>(dataTypes.size()); + for (int j = 0; j < dataTypes.size(); j++) { + bitMaps.add(null); } - memoryBinaryChunkSize[copyIndex++] = tmpValueChunkRawSize[i]; } - - for (Object array : values.get(columnIndex)) { - PrimitiveArrayManager.release(array); + if (bitMaps.get(columnIndex) == null) { + List columnBitMaps = new ArrayList<>(); + for (int i = 0; i < values.get(columnIndex).size(); i++) { + columnBitMaps.add(new BitMap(ARRAY_SIZE)); + } + bitMaps.set(columnIndex, columnBitMaps); + } + for (int i = 0; i < bitMaps.get(columnIndex).size(); i++) { + bitMaps.get(columnIndex).get(i).markAll(); } - values.remove(columnIndex); - bitMaps.remove(columnIndex); } protected Object cloneValue(TSDataType type, Object value) { From 93b9eeb1a0f8db367b2412c49c9cb326ef01cda2 Mon Sep 17 00:00:00 2001 From: shizy Date: Sun, 5 Jan 2025 13:29:54 +0800 Subject: [PATCH 38/54] fix: aligned timeseries encode bug --- .../memtable/AlignedWritableMemChunk.java | 35 ++++++++++++------- 1 file changed, 22 insertions(+), 13 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedWritableMemChunk.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedWritableMemChunk.java index ae9fd9c59d118..4a75e3a8de8e6 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedWritableMemChunk.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedWritableMemChunk.java @@ -734,7 +734,8 @@ private void writePageValuesIntoWriter( int[] accessInfo = pageAccessInfo.get(index); TsPrimitiveType value = timeValuePairIterator.getPrimitiveObject(accessInfo, columnIndex); - valueChunkWriter.write(times[index], value.getBoolean(), value.getValue() == null); + valueChunkWriter.write( + times[index], value != null && value.getBoolean(), value == null); } break; case INT32: @@ -743,7 +744,7 @@ private void writePageValuesIntoWriter( int[] accessInfo = pageAccessInfo.get(index); TsPrimitiveType value = timeValuePairIterator.getPrimitiveObject(accessInfo, columnIndex); - valueChunkWriter.write(times[index], value.getInt(), value.getValue() == null); + valueChunkWriter.write(times[index], value == null ? 0 : value.getInt(), value == null); } break; case INT64: @@ -752,7 +753,8 @@ private void writePageValuesIntoWriter( int[] accessInfo = pageAccessInfo.get(index); TsPrimitiveType value = timeValuePairIterator.getPrimitiveObject(accessInfo, columnIndex); - valueChunkWriter.write(times[index], value.getLong(), value.getValue() == null); + valueChunkWriter.write( + times[index], value == null ? 0L : value.getLong(), value == null); } break; case FLOAT: @@ -760,7 +762,8 @@ private void writePageValuesIntoWriter( int[] accessInfo = pageAccessInfo.get(index); TsPrimitiveType value = timeValuePairIterator.getPrimitiveObject(accessInfo, columnIndex); - valueChunkWriter.write(times[index], value.getFloat(), value.getValue() == null); + valueChunkWriter.write( + times[index], value == null ? 0f : value.getFloat(), value == null); } break; case DOUBLE: @@ -768,7 +771,8 @@ private void writePageValuesIntoWriter( int[] accessInfo = pageAccessInfo.get(index); TsPrimitiveType value = timeValuePairIterator.getPrimitiveObject(accessInfo, columnIndex); - valueChunkWriter.write(times[index], value.getDouble(), value.getValue() == null); + valueChunkWriter.write( + times[index], value == null ? 0d : value.getDouble(), value == null); } break; case TEXT: @@ -778,7 +782,10 @@ private void writePageValuesIntoWriter( int[] accessInfo = pageAccessInfo.get(index); TsPrimitiveType value = timeValuePairIterator.getPrimitiveObject(accessInfo, columnIndex); - valueChunkWriter.write(times[index], value.getBinary(), value.getValue() == null); + valueChunkWriter.write( + times[index], + value == null ? Binary.EMPTY_VALUE : value.getBinary(), + value == null); } break; default: @@ -815,8 +822,8 @@ public synchronized void encode(BlockingQueue ioTaskQueue) { while (timeValuePairIterator.hasNextTimeValuePair()) { // prepare column access info for current page int[][] accessInfo = timeValuePairIterator.getColumnAccessInfo(); + times[pointNumInPage] = timeValuePairIterator.getTime(); for (int i = 0; i < dataTypes.size(); i++) { - times[pointNumInPage] = timeValuePairIterator.getTime(); pageColumnAccessInfo[i].add(accessInfo[i]); } timeValuePairIterator.step(); @@ -848,12 +855,14 @@ public synchronized void encode(BlockingQueue ioTaskQueue) { } // last batch of points - if (pointNumInPage > 0) { - writePageValuesIntoWriter( - alignedChunkWriter, times, pageColumnAccessInfo, timeValuePairIterator); - alignedChunkWriter.write(times, pointNumInPage, 0); - alignedChunkWriter.sealCurrentPage(); - alignedChunkWriter.clearPageWriter(); + if (pointNumInChunk > 0) { + if (pointNumInPage > 0) { + writePageValuesIntoWriter( + alignedChunkWriter, times, pageColumnAccessInfo, timeValuePairIterator); + alignedChunkWriter.write(times, pointNumInPage, 0); + alignedChunkWriter.sealCurrentPage(); + alignedChunkWriter.clearPageWriter(); + } try { ioTaskQueue.put(alignedChunkWriter); } catch (InterruptedException e) { From 46775f0fdb7424f7b08491a7a07b24f8f3eff5ba Mon Sep 17 00:00:00 2001 From: shizy Date: Sun, 5 Jan 2025 19:28:17 +0800 Subject: [PATCH 39/54] fix: IoTDBGroupByNaturalMonthIT --- .../utils/ResourceByPathUtils.java | 129 +++++++++--------- 1 file changed, 67 insertions(+), 62 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/utils/ResourceByPathUtils.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/utils/ResourceByPathUtils.java index 6956a6945bc41..c0ed5c73fa3df 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/utils/ResourceByPathUtils.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/utils/ResourceByPathUtils.java @@ -52,6 +52,7 @@ import org.apache.tsfile.file.metadata.statistics.Statistics; import org.apache.tsfile.read.common.TimeRange; import org.apache.tsfile.read.filter.basic.Filter; +import org.apache.tsfile.read.filter.operator.GroupByFilter; import org.apache.tsfile.utils.Pair; import org.apache.tsfile.write.schema.IMeasurementSchema; import org.apache.tsfile.write.schema.VectorMeasurementSchema; @@ -356,6 +357,7 @@ private Map prepareAlignedTvListMapForQuery( // immutable aligned TVList for (AlignedTVList alignedTvList : alignedMemChunk.getSortedList()) { if (globalTimeFilter != null + && !(globalTimeFilter instanceof GroupByFilter) && !globalTimeFilter.satisfyStartEndTime( alignedTvList.getMinTime(), alignedTvList.getMaxTime())) { continue; @@ -376,36 +378,37 @@ private Map prepareAlignedTvListMapForQuery( AlignedTVList cloneList = null; list.lockQueryList(); try { - if (!isWorkMemTable) { - if (globalTimeFilter == null - || globalTimeFilter.satisfyStartEndTime(list.getMinTime(), list.getMaxTime())) { + if (globalTimeFilter == null + || globalTimeFilter instanceof GroupByFilter + || globalTimeFilter.satisfyStartEndTime(list.getMinTime(), list.getMaxTime())) { + if (!isWorkMemTable) { LOGGER.debug( "Flushing MemTable - add current query context to mutable AlignedTVList's query list"); list.getQueryContextList().add(context); alignedTvListQueryMap.put(list, list.rowCount()); - } - } else { - if (list.isSorted() || list.getQueryContextList().isEmpty()) { - LOGGER.debug( - "Working MemTable - add current query context to mutable AlignedTVList's query list when it's sorted or no other query on it"); - list.getQueryContextList().add(context); - alignedTvListQueryMap.put(list, list.rowCount()); } else { - LOGGER.debug( - "Working MemTable - clone mutable AlignedTVList and replace old AlignedTVList in working MemTable"); - QueryContext firstQuery = list.getQueryContextList().get(0); - // reserve query memory - if (firstQuery instanceof FragmentInstanceContext) { - MemoryReservationManager memoryReservationManager = - ((FragmentInstanceContext) firstQuery).getMemoryReservationContext(); - memoryReservationManager.reserveMemoryCumulatively(list.calculateRamSize()); + if (list.isSorted() || list.getQueryContextList().isEmpty()) { + LOGGER.debug( + "Working MemTable - add current query context to mutable AlignedTVList's query list when it's sorted or no other query on it"); + list.getQueryContextList().add(context); + alignedTvListQueryMap.put(list, list.rowCount()); + } else { + LOGGER.debug( + "Working MemTable - clone mutable AlignedTVList and replace old AlignedTVList in working MemTable"); + QueryContext firstQuery = list.getQueryContextList().get(0); + // reserve query memory + if (firstQuery instanceof FragmentInstanceContext) { + MemoryReservationManager memoryReservationManager = + ((FragmentInstanceContext) firstQuery).getMemoryReservationContext(); + memoryReservationManager.reserveMemoryCumulatively(list.calculateRamSize()); + } + list.setOwnerQuery(firstQuery); + + // clone TVList + cloneList = list.clone(); + cloneList.getQueryContextList().add(context); + alignedTvListQueryMap.put(cloneList, cloneList.rowCount()); } - list.setOwnerQuery(firstQuery); - - // clone TVList - cloneList = list.clone(); - cloneList.getQueryContextList().add(context); - alignedTvListQueryMap.put(cloneList, cloneList.rowCount()); } } } finally { @@ -546,6 +549,7 @@ private Map prepareTvListMapForQuery( // immutable sorted lists for (TVList tvList : memChunk.getSortedList()) { if (globalTimeFilter != null + && !(globalTimeFilter instanceof GroupByFilter) && !globalTimeFilter.satisfyStartEndTime(tvList.getMinTime(), tvList.getMaxTime())) { continue; } @@ -565,50 +569,51 @@ private Map prepareTvListMapForQuery( TVList cloneList = null; list.lockQueryList(); try { - if (!isWorkMemTable) { - if (globalTimeFilter == null - || globalTimeFilter.satisfyStartEndTime(list.getMinTime(), list.getMaxTime())) { + if (globalTimeFilter == null + || globalTimeFilter instanceof GroupByFilter + || globalTimeFilter.satisfyStartEndTime(list.getMinTime(), list.getMaxTime())) { + if (!isWorkMemTable) { LOGGER.debug( "Flushing MemTable - add current query context to mutable TVList's query list"); list.getQueryContextList().add(context); tvListQueryMap.put(list, list.rowCount()); - } - } else { - if (list.isSorted() || list.getQueryContextList().isEmpty()) { - LOGGER.debug( - "Working MemTable - add current query context to mutable TVList's query list when it's sorted or no other query on it"); - list.getQueryContextList().add(context); - tvListQueryMap.put(list, list.rowCount()); } else { - /* - * +----------------------+ - * | MemTable | - * | | - * | +------------+ | +-----------------+ - * | | TVList |<---+--+ +---+ Previous Query | - * | +-----^------+ | | | +-----------------+ - * | | | | | - * +----------+-----------+ | | +----------------+ - * | Clone +---+---+ Current Query | - * +-----+------+ | +----------------+ - * | TVList | <---------+ - * +------------+ - */ - LOGGER.debug( - "Working MemTable - clone mutable TVList and replace old TVList in working MemTable"); - QueryContext firstQuery = list.getQueryContextList().get(0); - // reserve query memory - if (firstQuery instanceof FragmentInstanceContext) { - MemoryReservationManager memoryReservationManager = - ((FragmentInstanceContext) firstQuery).getMemoryReservationContext(); - memoryReservationManager.reserveMemoryCumulatively(list.calculateRamSize()); + if (list.isSorted() || list.getQueryContextList().isEmpty()) { + LOGGER.debug( + "Working MemTable - add current query context to mutable TVList's query list when it's sorted or no other query on it"); + list.getQueryContextList().add(context); + tvListQueryMap.put(list, list.rowCount()); + } else { + /* + * +----------------------+ + * | MemTable | + * | | + * | +------------+ | +-----------------+ + * | | TVList |<---+--+ +---+ Previous Query | + * | +-----^------+ | | | +-----------------+ + * | | | | | + * +----------+-----------+ | | +----------------+ + * | Clone +---+---+ Current Query | + * +-----+------+ | +----------------+ + * | TVList | <---------+ + * +------------+ + */ + LOGGER.debug( + "Working MemTable - clone mutable TVList and replace old TVList in working MemTable"); + QueryContext firstQuery = list.getQueryContextList().get(0); + // reserve query memory + if (firstQuery instanceof FragmentInstanceContext) { + MemoryReservationManager memoryReservationManager = + ((FragmentInstanceContext) firstQuery).getMemoryReservationContext(); + memoryReservationManager.reserveMemoryCumulatively(list.calculateRamSize()); + } + list.setOwnerQuery(firstQuery); + + // clone TVList + cloneList = list.clone(); + cloneList.getQueryContextList().add(context); + tvListQueryMap.put(cloneList, cloneList.rowCount()); } - list.setOwnerQuery(firstQuery); - - // clone TVList - cloneList = list.clone(); - cloneList.getQueryContextList().add(context); - tvListQueryMap.put(cloneList, cloneList.rowCount()); } } } finally { From 3501b8f54a0b5546f0e4bc866d597eb00490c6ad Mon Sep 17 00:00:00 2001 From: shizy Date: Sun, 5 Jan 2025 19:32:40 +0800 Subject: [PATCH 40/54] remove avgSeriesPointNumberThreshold setting --- .../src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java | 3 --- .../org/apache/iotdb/db/utils/datastructure/AlignedTVList.java | 2 +- 2 files changed, 1 insertion(+), 4 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java index 012176820c00f..623081d3b1a9a 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java @@ -439,9 +439,6 @@ public class IoTDBConfig { */ private int tvListSortThreshold = 0; - /** When average series point number reaches this, flush the memtable to disk */ - private int avgSeriesPointNumberThreshold = 100000; - /** Enable inner space compaction for sequence files */ private volatile boolean enableSeqSpaceCompaction = true; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/AlignedTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/AlignedTVList.java index 5f5822f9434ca..2f29750e9752b 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/AlignedTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/AlignedTVList.java @@ -1534,7 +1534,7 @@ public class AlignedTVListIterator extends TVListIterator { private TSEncoding[] encodingArray; // remember the selected index of last not-null value for each column during prepareNext phase. - // It is already transferred by getValueIndex method, so it can be directly used in + // It is already converted by getValueIndex method, so it can be directly used in // getPrimitiveObject method. private int[] selectedIndex; From 062adc7f950844fd8386c51e25fe7262ba3659d7 Mon Sep 17 00:00:00 2001 From: shizy Date: Mon, 6 Jan 2025 08:25:55 +0800 Subject: [PATCH 41/54] fix: IoTDBDeleteAlignedTimeseriesIT & AlignedTVListTest --- .../memtable/AlignedWritableMemChunk.java | 17 ++++++++-------- .../db/utils/datastructure/AlignedTVList.java | 4 ---- .../datastructure/AlignedTVListTest.java | 20 +++++++++---------- 3 files changed, 19 insertions(+), 22 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedWritableMemChunk.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedWritableMemChunk.java index 4a75e3a8de8e6..19a30c059b6ea 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedWritableMemChunk.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedWritableMemChunk.java @@ -271,7 +271,14 @@ private Pair checkAndReorderColumnValuesInInsertPlan( // We need to extend a new column in AlignedMemChunk and AlignedTVList. // And the reorderedColumnValues should extend one more column for the new measurement if (index == null) { - index = measurementIndexMap.size(); + index = + measurementIndexMap.isEmpty() + ? 0 + : measurementIndexMap.values().stream() + .mapToInt(Integer::intValue) + .max() + .getAsInt() + + 1; this.measurementIndexMap.put(schemaListInInsertPlan.get(i).getMeasurementName(), index); this.schemaList.add(schemaListInInsertPlan.get(i)); this.list.extendColumn(schemaListInInsertPlan.get(i).getType()); @@ -506,13 +513,7 @@ public void removeColumn(String measurementId) { alignedTvList.deleteColumn(columnIndex); } IMeasurementSchema schemaToBeRemoved = schemaList.get(columnIndex); - measurementIndexMap.clear(); - for (int i = 0; i < schemaList.size(); i++) { - if (schemaList.get(i).getMeasurementName().equals(schemaToBeRemoved.getMeasurementName())) { - continue; - } - measurementIndexMap.put(schemaList.get(i).getMeasurementName(), i); - } + measurementIndexMap.remove(schemaToBeRemoved.getMeasurementName()); } @Override diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/AlignedTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/AlignedTVList.java index 2f29750e9752b..d7558e66b5ac8 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/AlignedTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/AlignedTVList.java @@ -759,10 +759,6 @@ protected void releaseLastValueArray() { } } - public long[] memoryBinaryChunkSize() { - return memoryBinaryChunkSize; - } - @SuppressWarnings("squid:S3776") // Suppress high Cognitive Complexity warning @Override public synchronized void putAlignedValues( diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/utils/datastructure/AlignedTVListTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/utils/datastructure/AlignedTVListTest.java index 25e45af4ea797..c7ae72c6ca12f 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/utils/datastructure/AlignedTVListTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/utils/datastructure/AlignedTVListTest.java @@ -224,28 +224,28 @@ public void testCalculateChunkSize() { Assert.assertEquals(tvList.memoryBinaryChunkSize[1], 720); tvList.delete(5, 15); - Assert.assertEquals(tvList.memoryBinaryChunkSize[1], 324); + Assert.assertEquals(tvList.memoryBinaryChunkSize[1], 720); tvList.deleteColumn(0); - Assert.assertEquals(tvList.memoryBinaryChunkSize.length, 1); - Assert.assertEquals(tvList.memoryBinaryChunkSize[0], 324); + Assert.assertEquals(tvList.memoryBinaryChunkSize.length, 2); + Assert.assertEquals(tvList.memoryBinaryChunkSize[1], 720); tvList.extendColumn(TSDataType.INT32); - Assert.assertEquals(tvList.memoryBinaryChunkSize.length, 2); - Assert.assertEquals(tvList.memoryBinaryChunkSize[0], 324); + Assert.assertEquals(tvList.memoryBinaryChunkSize.length, 3); + Assert.assertEquals(tvList.memoryBinaryChunkSize[1], 720); tvList.extendColumn(TSDataType.TEXT); - Assert.assertEquals(tvList.memoryBinaryChunkSize.length, 3); - Assert.assertEquals(tvList.memoryBinaryChunkSize[0], 324); + Assert.assertEquals(tvList.memoryBinaryChunkSize.length, 4); + Assert.assertEquals(tvList.memoryBinaryChunkSize[1], 720); Assert.assertEquals(tvList.memoryBinaryChunkSize[2], 0); tvList.delete(4, 6); - Assert.assertEquals(tvList.memoryBinaryChunkSize.length, 3); - Assert.assertEquals(tvList.memoryBinaryChunkSize[0], 216); + Assert.assertEquals(tvList.memoryBinaryChunkSize.length, 4); + Assert.assertEquals(tvList.memoryBinaryChunkSize[1], 720); Assert.assertEquals(tvList.memoryBinaryChunkSize[2], 0); tvList.clear(); - Assert.assertEquals(tvList.memoryBinaryChunkSize[0], 0); + Assert.assertEquals(tvList.memoryBinaryChunkSize[1], 0); Assert.assertEquals(tvList.memoryBinaryChunkSize[2], 0); } } From 7c2c11d141b2bf52f61dad0f36b111d2d3b9843d Mon Sep 17 00:00:00 2001 From: shizy Date: Mon, 6 Jan 2025 15:28:51 +0800 Subject: [PATCH 42/54] fix: Copy globalTimeFilter due to GroupByMonthFilter --- .../utils/ResourceByPathUtils.java | 156 ++++++++++-------- 1 file changed, 83 insertions(+), 73 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/utils/ResourceByPathUtils.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/utils/ResourceByPathUtils.java index c0ed5c73fa3df..ae7f2eba05f7e 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/utils/ResourceByPathUtils.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/utils/ResourceByPathUtils.java @@ -52,7 +52,6 @@ import org.apache.tsfile.file.metadata.statistics.Statistics; import org.apache.tsfile.read.common.TimeRange; import org.apache.tsfile.read.filter.basic.Filter; -import org.apache.tsfile.read.filter.operator.GroupByFilter; import org.apache.tsfile.utils.Pair; import org.apache.tsfile.write.schema.IMeasurementSchema; import org.apache.tsfile.write.schema.VectorMeasurementSchema; @@ -353,12 +352,17 @@ private Map prepareAlignedTvListMapForQuery( AlignedWritableMemChunk alignedMemChunk, boolean isWorkMemTable, Filter globalTimeFilter) { + // should copy globalTimeFilter because GroupByMonthFilter is stateful + Filter copyTimeFilter = null; + if (globalTimeFilter != null) { + copyTimeFilter = globalTimeFilter.copy(); + } + Map alignedTvListQueryMap = new LinkedHashMap<>(); // immutable aligned TVList for (AlignedTVList alignedTvList : alignedMemChunk.getSortedList()) { - if (globalTimeFilter != null - && !(globalTimeFilter instanceof GroupByFilter) - && !globalTimeFilter.satisfyStartEndTime( + if (copyTimeFilter != null + && !copyTimeFilter.satisfyStartEndTime( alignedTvList.getMinTime(), alignedTvList.getMaxTime())) { continue; } @@ -378,37 +382,37 @@ private Map prepareAlignedTvListMapForQuery( AlignedTVList cloneList = null; list.lockQueryList(); try { - if (globalTimeFilter == null - || globalTimeFilter instanceof GroupByFilter - || globalTimeFilter.satisfyStartEndTime(list.getMinTime(), list.getMaxTime())) { - if (!isWorkMemTable) { + if (copyTimeFilter != null + && !copyTimeFilter.satisfyStartEndTime(list.getMinTime(), list.getMaxTime())) { + return alignedTvListQueryMap; + } + if (!isWorkMemTable) { + LOGGER.debug( + "Flushing MemTable - add current query context to mutable AlignedTVList's query list"); + list.getQueryContextList().add(context); + alignedTvListQueryMap.put(list, list.rowCount()); + } else { + if (list.isSorted() || list.getQueryContextList().isEmpty()) { LOGGER.debug( - "Flushing MemTable - add current query context to mutable AlignedTVList's query list"); + "Working MemTable - add current query context to mutable AlignedTVList's query list when it's sorted or no other query on it"); list.getQueryContextList().add(context); alignedTvListQueryMap.put(list, list.rowCount()); } else { - if (list.isSorted() || list.getQueryContextList().isEmpty()) { - LOGGER.debug( - "Working MemTable - add current query context to mutable AlignedTVList's query list when it's sorted or no other query on it"); - list.getQueryContextList().add(context); - alignedTvListQueryMap.put(list, list.rowCount()); - } else { - LOGGER.debug( - "Working MemTable - clone mutable AlignedTVList and replace old AlignedTVList in working MemTable"); - QueryContext firstQuery = list.getQueryContextList().get(0); - // reserve query memory - if (firstQuery instanceof FragmentInstanceContext) { - MemoryReservationManager memoryReservationManager = - ((FragmentInstanceContext) firstQuery).getMemoryReservationContext(); - memoryReservationManager.reserveMemoryCumulatively(list.calculateRamSize()); - } - list.setOwnerQuery(firstQuery); - - // clone TVList - cloneList = list.clone(); - cloneList.getQueryContextList().add(context); - alignedTvListQueryMap.put(cloneList, cloneList.rowCount()); + LOGGER.debug( + "Working MemTable - clone mutable AlignedTVList and replace old AlignedTVList in working MemTable"); + QueryContext firstQuery = list.getQueryContextList().get(0); + // reserve query memory + if (firstQuery instanceof FragmentInstanceContext) { + MemoryReservationManager memoryReservationManager = + ((FragmentInstanceContext) firstQuery).getMemoryReservationContext(); + memoryReservationManager.reserveMemoryCumulatively(list.calculateRamSize()); } + list.setOwnerQuery(firstQuery); + + // clone TVList + cloneList = list.clone(); + cloneList.getQueryContextList().add(context); + alignedTvListQueryMap.put(cloneList, cloneList.rowCount()); } } } finally { @@ -545,12 +549,17 @@ private Map prepareTvListMapForQuery( WritableMemChunk memChunk, boolean isWorkMemTable, Filter globalTimeFilter) { + // should copy globalTimeFilter because GroupByMonthFilter is stateful + Filter copyTimeFilter = null; + if (globalTimeFilter != null) { + copyTimeFilter = globalTimeFilter.copy(); + } + Map tvListQueryMap = new LinkedHashMap<>(); // immutable sorted lists for (TVList tvList : memChunk.getSortedList()) { - if (globalTimeFilter != null - && !(globalTimeFilter instanceof GroupByFilter) - && !globalTimeFilter.satisfyStartEndTime(tvList.getMinTime(), tvList.getMaxTime())) { + if (copyTimeFilter != null + && !copyTimeFilter.satisfyStartEndTime(tvList.getMinTime(), tvList.getMaxTime())) { continue; } tvList.lockQueryList(); @@ -569,51 +578,52 @@ private Map prepareTvListMapForQuery( TVList cloneList = null; list.lockQueryList(); try { - if (globalTimeFilter == null - || globalTimeFilter instanceof GroupByFilter - || globalTimeFilter.satisfyStartEndTime(list.getMinTime(), list.getMaxTime())) { - if (!isWorkMemTable) { + if (copyTimeFilter != null + && !copyTimeFilter.satisfyStartEndTime(list.getMinTime(), list.getMaxTime())) { + return tvListQueryMap; + } + + if (!isWorkMemTable) { + LOGGER.debug( + "Flushing MemTable - add current query context to mutable TVList's query list"); + list.getQueryContextList().add(context); + tvListQueryMap.put(list, list.rowCount()); + } else { + if (list.isSorted() || list.getQueryContextList().isEmpty()) { LOGGER.debug( - "Flushing MemTable - add current query context to mutable TVList's query list"); + "Working MemTable - add current query context to mutable TVList's query list when it's sorted or no other query on it"); list.getQueryContextList().add(context); tvListQueryMap.put(list, list.rowCount()); } else { - if (list.isSorted() || list.getQueryContextList().isEmpty()) { - LOGGER.debug( - "Working MemTable - add current query context to mutable TVList's query list when it's sorted or no other query on it"); - list.getQueryContextList().add(context); - tvListQueryMap.put(list, list.rowCount()); - } else { - /* - * +----------------------+ - * | MemTable | - * | | - * | +------------+ | +-----------------+ - * | | TVList |<---+--+ +---+ Previous Query | - * | +-----^------+ | | | +-----------------+ - * | | | | | - * +----------+-----------+ | | +----------------+ - * | Clone +---+---+ Current Query | - * +-----+------+ | +----------------+ - * | TVList | <---------+ - * +------------+ - */ - LOGGER.debug( - "Working MemTable - clone mutable TVList and replace old TVList in working MemTable"); - QueryContext firstQuery = list.getQueryContextList().get(0); - // reserve query memory - if (firstQuery instanceof FragmentInstanceContext) { - MemoryReservationManager memoryReservationManager = - ((FragmentInstanceContext) firstQuery).getMemoryReservationContext(); - memoryReservationManager.reserveMemoryCumulatively(list.calculateRamSize()); - } - list.setOwnerQuery(firstQuery); - - // clone TVList - cloneList = list.clone(); - cloneList.getQueryContextList().add(context); - tvListQueryMap.put(cloneList, cloneList.rowCount()); + /* + * +----------------------+ + * | MemTable | + * | | + * | +------------+ | +-----------------+ + * | | TVList |<---+--+ +---+ Previous Query | + * | +-----^------+ | | | +-----------------+ + * | | | | | + * +----------+-----------+ | | +----------------+ + * | Clone +---+---+ Current Query | + * +-----+------+ | +----------------+ + * | TVList | <---------+ + * +------------+ + */ + LOGGER.debug( + "Working MemTable - clone mutable TVList and replace old TVList in working MemTable"); + QueryContext firstQuery = list.getQueryContextList().get(0); + // reserve query memory + if (firstQuery instanceof FragmentInstanceContext) { + MemoryReservationManager memoryReservationManager = + ((FragmentInstanceContext) firstQuery).getMemoryReservationContext(); + memoryReservationManager.reserveMemoryCumulatively(list.calculateRamSize()); } + list.setOwnerQuery(firstQuery); + + // clone TVList + cloneList = list.clone(); + cloneList.getQueryContextList().add(context); + tvListQueryMap.put(cloneList, cloneList.rowCount()); } } } finally { From edf79528dcba5ffb46930a3cfc2a0bc956feee16 Mon Sep 17 00:00:00 2001 From: shizy Date: Mon, 6 Jan 2025 22:41:52 +0800 Subject: [PATCH 43/54] reset tmpLength for backward sort --- .../org/apache/iotdb/db/utils/datastructure/BackwardSort.java | 1 + 1 file changed, 1 insertion(+) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BackwardSort.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BackwardSort.java index 4800032c7a06a..d5916ced12c10 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BackwardSort.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BackwardSort.java @@ -78,6 +78,7 @@ public void clearTmp() { PrimitiveArrayManager.release(dataArray); } tmpIndices.clear(); + tmpLength = 0; } public void backwardSort(List timestamps, int rowCount) { From d6bf7018c12e9f10edc81fdcc95533fedf9e22db Mon Sep 17 00:00:00 2001 From: shizy Date: Fri, 10 Jan 2025 11:24:58 +0800 Subject: [PATCH 44/54] * fix TVList clear * bitmap mark * sequence row count --- .../execution/fragment/FragmentInstanceContext.java | 2 +- .../dataregion/memtable/AlignedWritableMemChunk.java | 5 ++--- .../apache/iotdb/db/utils/datastructure/BinaryTVList.java | 2 +- .../apache/iotdb/db/utils/datastructure/BooleanTVList.java | 2 +- .../apache/iotdb/db/utils/datastructure/DoubleTVList.java | 2 +- .../org/apache/iotdb/db/utils/datastructure/FloatTVList.java | 2 +- .../org/apache/iotdb/db/utils/datastructure/IntTVList.java | 2 +- .../org/apache/iotdb/db/utils/datastructure/LongTVList.java | 2 +- .../java/org/apache/iotdb/db/utils/datastructure/TVList.java | 2 +- 9 files changed, 10 insertions(+), 11 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceContext.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceContext.java index 2b12262aaaecd..15637998825e1 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceContext.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceContext.java @@ -668,8 +668,8 @@ private void releaseTVListOwnedByQuery() { "TVList {} is released by the query, FragmentInstance Id is {}", tvList, this.getId()); - tvList.clear(); memoryReservationManager.releaseMemoryCumulatively(tvList.calculateRamSize()); + tvList.clear(); } else { LOGGER.debug( "TVList {} is now owned by another query, FragmentInstance Id is {}", diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedWritableMemChunk.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedWritableMemChunk.java index 19a30c059b6ea..af3b9d0abfd22 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedWritableMemChunk.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedWritableMemChunk.java @@ -318,10 +318,9 @@ private void filterDeletedTimeStamp( long timestamp = alignedTVList.getTime(row); BitMap bitMap = new BitMap(schemaList.size()); - bitMap.markAll(); for (int column = 0; column < schemaList.size(); column++) { - if (!alignedTVList.isNullValue(alignedTVList.getValueIndex(row), column)) { - bitMap.unmark(column); + if (alignedTVList.isNullValue(alignedTVList.getValueIndex(row), column)) { + bitMap.mark(column); } // skip deleted row diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BinaryTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BinaryTVList.java index c7453f9f76815..bf3d51f9752a5 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BinaryTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BinaryTVList.java @@ -249,7 +249,7 @@ int dropNullValThenUpdateMinMaxTimeAndSorted( if (sorted && (rowCount == 0 - || (end - start > nullCnt) && time[start - tIdxOffset] > getTime(rowCount - 1))) { + || (end - start > nullCnt) && time[start - tIdxOffset] >= getTime(rowCount - 1))) { seqRowCount += inputSeqRowCount; } sorted = sorted && inputSorted && (rowCount == 0 || inPutMinTime >= getTime(rowCount - 1)); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BooleanTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BooleanTVList.java index 16caedc83ce89..c32df45b849a4 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BooleanTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BooleanTVList.java @@ -249,7 +249,7 @@ int dropNullValThenUpdateMinMaxTimeAndSorted( if (sorted && (rowCount == 0 - || (end - start > nullCnt) && time[start - tIdxOffset] > getTime(rowCount - 1))) { + || (end - start > nullCnt) && time[start - tIdxOffset] >= getTime(rowCount - 1))) { seqRowCount += inputSeqRowCount; } sorted = sorted && inputSorted && (rowCount == 0 || inPutMinTime >= getTime(rowCount - 1)); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/DoubleTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/DoubleTVList.java index 6682435e04722..a3acc7495dfed 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/DoubleTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/DoubleTVList.java @@ -255,7 +255,7 @@ int dropNullValThenUpdateMinMaxTimeAndSorted( if (sorted && (rowCount == 0 - || (end - start > nullCnt) && time[start - tIdxOffset] > getTime(rowCount - 1))) { + || (end - start > nullCnt) && time[start - tIdxOffset] >= getTime(rowCount - 1))) { seqRowCount += inputSeqRowCount; } sorted = sorted && inputSorted && (rowCount == 0 || inPutMinTime >= getTime(rowCount - 1)); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/FloatTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/FloatTVList.java index 986a662bb350d..714f611e0ae88 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/FloatTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/FloatTVList.java @@ -255,7 +255,7 @@ int dropNullValThenUpdateMinMaxTimeAndSorted( if (sorted && (rowCount == 0 - || (end - start > nullCnt) && time[start - tIdxOffset] > getTime(rowCount - 1))) { + || (end - start > nullCnt) && time[start - tIdxOffset] >= getTime(rowCount - 1))) { seqRowCount += inputSeqRowCount; } sorted = sorted && inputSorted && (rowCount == 0 || inPutMinTime >= getTime(rowCount - 1)); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/IntTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/IntTVList.java index 106b3bb740243..43d2977708d55 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/IntTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/IntTVList.java @@ -247,7 +247,7 @@ int dropNullValThenUpdateMinMaxTimeAndSorted( if (sorted && (rowCount == 0 - || (end - start > nullCnt) && time[start - tIdxOffset] > getTime(rowCount - 1))) { + || (end - start > nullCnt) && time[start - tIdxOffset] >= getTime(rowCount - 1))) { seqRowCount += inputSeqRowCount; } sorted = sorted && inputSorted && (rowCount == 0 || inPutMinTime >= getTime(rowCount - 1)); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/LongTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/LongTVList.java index 37b21699de584..aa73cb32dd40a 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/LongTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/LongTVList.java @@ -247,7 +247,7 @@ int dropNullValThenUpdateMinMaxTimeAndSorted( if (sorted && (rowCount == 0 - || (end - start > nullCnt) && time[start - tIdxOffset] > getTime(rowCount - 1))) { + || (end - start > nullCnt) && time[start - tIdxOffset] >= getTime(rowCount - 1))) { seqRowCount += inputSeqRowCount; } sorted = sorted && inputSorted && (rowCount == 0 || inPutMinTime >= getTime(rowCount - 1)); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java index 1a446c56fc325..d4552cf98ac9f 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java @@ -493,7 +493,7 @@ void updateMinMaxTimeAndSorted(long[] time, int start, int end) { } } } - if (sorted && (rowCount == 0 || time[start] > getTime(rowCount - 1))) { + if (sorted && (rowCount == 0 || time[start] >= getTime(rowCount - 1))) { seqRowCount += inputSeqRowCount; } sorted = sorted && inputSorted && (rowCount == 0 || inPutMinTime >= getTime(rowCount - 1)); From 89ad4d5a801f02fa8415b5188898334dd06f5dd9 Mon Sep 17 00:00:00 2001 From: shizy Date: Fri, 10 Jan 2025 14:18:48 +0800 Subject: [PATCH 45/54] hot-load TVLIST_SORT_THRESHOLD --- .../java/org/apache/iotdb/db/conf/IoTDBDescriptor.java | 9 +++++++++ .../dataregion/memtable/AlignedReadOnlyMemChunk.java | 3 +-- .../dataregion/memtable/AlignedWritableMemChunk.java | 6 +++++- .../dataregion/memtable/IWritableMemChunk.java | 6 ------ .../dataregion/memtable/ReadOnlyMemChunk.java | 8 +++++++- .../dataregion/memtable/WritableMemChunk.java | 4 ++++ .../read/reader/chunk/MemAlignedChunkReader.java | 6 +++--- .../dataregion/read/reader/chunk/MemChunkReader.java | 5 ++--- .../iotdb/db/utils/datastructure/AlignedTVList.java | 2 +- .../db/utils/datastructure/PageColumnAccessInfo.java | 8 +++----- .../resources/conf/iotdb-system.properties.template | 2 +- 11 files changed, 36 insertions(+), 23 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java index deec6d10bbe2a..841d854ec2b84 100755 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java @@ -2065,6 +2065,15 @@ public synchronized void loadHotModifiedProps(TrimProperties properties) loadQuerySampleThroughput(properties); // update trusted_uri_pattern loadTrustedUriPattern(properties); + + // tvlist_sort_threshold + conf.setTVListSortThreshold( + Integer.parseInt( + Optional.ofNullable( + properties.getProperty( + "tvlist_sort_threshold", String.valueOf(conf.getTvListSortThreshold()))) + .map(String::trim) + .orElse(String.valueOf(conf.getTvListSortThreshold())))); } catch (Exception e) { if (e instanceof InterruptedException) { Thread.currentThread().interrupt(); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedReadOnlyMemChunk.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedReadOnlyMemChunk.java index 13150370e1093..d898db066bb97 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedReadOnlyMemChunk.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedReadOnlyMemChunk.java @@ -51,7 +51,6 @@ import java.util.List; import java.util.Map; -import static org.apache.iotdb.db.storageengine.dataregion.memtable.IWritableMemChunk.MAX_NUMBER_OF_POINTS_IN_PAGE; import static org.apache.iotdb.db.utils.ModificationUtils.isPointDeleted; public class AlignedReadOnlyMemChunk extends ReadOnlyMemChunk { @@ -151,7 +150,7 @@ public void initChunkMetaFromTvLists() { long[] time = new long[MAX_NUMBER_OF_POINTS_IN_PAGE]; PageColumnAccessInfo[] pageColumnAccessInfo = new PageColumnAccessInfo[dataTypes.size()]; for (int i = 0; i < pageColumnAccessInfo.length; i++) { - pageColumnAccessInfo[i] = new PageColumnAccessInfo(); + pageColumnAccessInfo[i] = new PageColumnAccessInfo(MAX_NUMBER_OF_POINTS_IN_PAGE); } int[] timeDeleteCursor = new int[] {0}; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedWritableMemChunk.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedWritableMemChunk.java index af3b9d0abfd22..d903a0ae2cf7e 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedWritableMemChunk.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedWritableMemChunk.java @@ -32,6 +32,7 @@ import org.apache.iotdb.db.utils.datastructure.PageColumnAccessInfo; import org.apache.iotdb.db.utils.datastructure.TVList; +import org.apache.tsfile.common.conf.TSFileDescriptor; import org.apache.tsfile.enums.TSDataType; import org.apache.tsfile.read.common.TimeRange; import org.apache.tsfile.utils.Binary; @@ -72,6 +73,9 @@ public class AlignedWritableMemChunk implements IWritableMemChunk { private static final IoTDBConfig CONFIG = IoTDBDescriptor.getInstance().getConfig(); private final long TARGET_CHUNK_SIZE = CONFIG.getTargetChunkSize(); private long maxNumberOfPointsInChunk = CONFIG.getTargetChunkPointNum(); + private final int TVLIST_SORT_THRESHOLD = CONFIG.getTvListSortThreshold(); + private final int MAX_NUMBER_OF_POINTS_IN_PAGE = + TSFileDescriptor.getInstance().getConfig().getMaxNumberOfPointsInPage(); private static final String UNSUPPORTED_TYPE = "Unsupported data type:"; @@ -816,7 +820,7 @@ public synchronized void encode(BlockingQueue ioTaskQueue) { PageColumnAccessInfo[] pageColumnAccessInfo = new PageColumnAccessInfo[dataTypes.size()]; for (int i = 0; i < pageColumnAccessInfo.length; i++) { - pageColumnAccessInfo[i] = new PageColumnAccessInfo(); + pageColumnAccessInfo[i] = new PageColumnAccessInfo(MAX_NUMBER_OF_POINTS_IN_PAGE); } while (timeValuePairIterator.hasNextTimeValuePair()) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/IWritableMemChunk.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/IWritableMemChunk.java index cee4e2e302460..576b82bff4dc7 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/IWritableMemChunk.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/IWritableMemChunk.java @@ -19,11 +19,9 @@ package org.apache.iotdb.db.storageengine.dataregion.memtable; import org.apache.iotdb.common.rpc.thrift.TSStatus; -import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.storageengine.dataregion.wal.buffer.WALEntryValue; import org.apache.iotdb.db.utils.datastructure.TVList; -import org.apache.tsfile.common.conf.TSFileDescriptor; import org.apache.tsfile.enums.TSDataType; import org.apache.tsfile.utils.Binary; import org.apache.tsfile.utils.BitMap; @@ -34,10 +32,6 @@ import java.util.concurrent.BlockingQueue; public interface IWritableMemChunk extends WALEntryValue { - int TVLIST_SORT_THRESHOLD = IoTDBDescriptor.getInstance().getConfig().getTvListSortThreshold(); - int MAX_NUMBER_OF_POINTS_IN_PAGE = - TSFileDescriptor.getInstance().getConfig().getMaxNumberOfPointsInPage(); - void putLong(long t, long v); void putInt(long t, int v); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/ReadOnlyMemChunk.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/ReadOnlyMemChunk.java index 7cb73cf54e88c..1f1e6582c53a9 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/ReadOnlyMemChunk.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/ReadOnlyMemChunk.java @@ -51,7 +51,6 @@ import java.util.List; import java.util.Map; -import static org.apache.iotdb.db.storageengine.dataregion.memtable.IWritableMemChunk.MAX_NUMBER_OF_POINTS_IN_PAGE; import static org.apache.iotdb.db.utils.ModificationUtils.isPointDeleted; /** @@ -88,6 +87,9 @@ public class ReadOnlyMemChunk { private MergeSortTvListIterator timeValuePairIterator; + protected final int MAX_NUMBER_OF_POINTS_IN_PAGE = + TSFileDescriptor.getInstance().getConfig().getMaxNumberOfPointsInPage(); + protected ReadOnlyMemChunk(QueryContext context) { this.context = context; } @@ -335,4 +337,8 @@ public TsBlock getTsBlock() { public MergeSortTvListIterator getMergeSortTVListIterator() { return timeValuePairIterator; } + + public int getMaxNumberOfPointsInPage() { + return MAX_NUMBER_OF_POINTS_IN_PAGE; + } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/WritableMemChunk.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/WritableMemChunk.java index 303b9ba76e728..a23c371c3e160 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/WritableMemChunk.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/WritableMemChunk.java @@ -30,6 +30,7 @@ import org.apache.iotdb.db.utils.datastructure.MergeSortTvListIterator; import org.apache.iotdb.db.utils.datastructure.TVList; +import org.apache.tsfile.common.conf.TSFileDescriptor; import org.apache.tsfile.enums.TSDataType; import org.apache.tsfile.read.TimeValuePair; import org.apache.tsfile.read.common.TimeRange; @@ -66,6 +67,9 @@ public class WritableMemChunk implements IWritableMemChunk { private static final IoTDBConfig CONFIG = IoTDBDescriptor.getInstance().getConfig(); private final long TARGET_CHUNK_SIZE = CONFIG.getTargetChunkSize(); private final long MAX_NUMBER_OF_POINTS_IN_CHUNK = CONFIG.getTargetChunkPointNum(); + private final int TVLIST_SORT_THRESHOLD = CONFIG.getTvListSortThreshold(); + private int MAX_NUMBER_OF_POINTS_IN_PAGE = + TSFileDescriptor.getInstance().getConfig().getMaxNumberOfPointsInPage(); public WritableMemChunk(IMeasurementSchema schema) { this.schema = schema; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemAlignedChunkReader.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemAlignedChunkReader.java index 0682b7133ba93..3b46b3ab2957a 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemAlignedChunkReader.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemAlignedChunkReader.java @@ -42,7 +42,6 @@ import java.util.List; import java.util.function.Supplier; -import static org.apache.iotdb.db.storageengine.dataregion.memtable.IWritableMemChunk.MAX_NUMBER_OF_POINTS_IN_PAGE; import static org.apache.iotdb.db.utils.ModificationUtils.isPointDeleted; /** To read aligned chunk data in memory. */ @@ -247,10 +246,11 @@ private synchronized void writeValidValuesIntoTsBlock(TsBlockBuilder builder) { List> valueColumnsDeletionList = readableChunk.getValueColumnsDeletionList(); int pointsInPage = 0; - long[] time = new long[MAX_NUMBER_OF_POINTS_IN_PAGE]; + long[] time = new long[readableChunk.getMaxNumberOfPointsInPage()]; PageColumnAccessInfo[] pageColumnAccessInfo = new PageColumnAccessInfo[tsDataTypes.size()]; for (int i = 0; i < pageColumnAccessInfo.length; i++) { - pageColumnAccessInfo[i] = new PageColumnAccessInfo(); + pageColumnAccessInfo[i] = + new PageColumnAccessInfo(readableChunk.getMaxNumberOfPointsInPage()); } int[] timeDeleteCursor = new int[] {0}; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemChunkReader.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemChunkReader.java index 0f80b6a45ebfc..96516415055c6 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemChunkReader.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemChunkReader.java @@ -41,7 +41,6 @@ import java.util.List; import java.util.function.Supplier; -import static org.apache.iotdb.db.storageengine.dataregion.memtable.IWritableMemChunk.MAX_NUMBER_OF_POINTS_IN_PAGE; import static org.apache.iotdb.db.utils.ModificationUtils.isPointDeleted; /** To read chunk data in memory. */ @@ -232,11 +231,11 @@ private synchronized void writeValidValuesIntoTsBlock(TsBlockBuilder builder) builder.declarePosition(); } } - if (builder.getPositionCount() > MAX_NUMBER_OF_POINTS_IN_PAGE) { + if (builder.getPositionCount() > readableChunk.getMaxNumberOfPointsInPage()) { throw new RuntimeException( String.format( "Points in current page %d is larger than %d", - builder.getPositionCount(), MAX_NUMBER_OF_POINTS_IN_PAGE)); + builder.getPositionCount(), readableChunk.getMaxNumberOfPointsInPage())); } } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/AlignedTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/AlignedTVList.java index d7558e66b5ac8..70381f3660ee2 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/AlignedTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/AlignedTVList.java @@ -1581,7 +1581,7 @@ private void prepareNext() { // handle duplicated timestamp while (index + 1 < rows && getTime(index + 1) == currentTime) { index++; - // skip all-Null rows if allValueColDeletedMap exits + // skip all-Null rows if allValueColDeletedMap exists int rowIndex = getValueIndex(index); if (allValueColDeletedMap == null || !allValueColDeletedMap.isMarked(rowIndex)) { for (int columnIndex = 0; columnIndex < dataTypeArray.length; columnIndex++) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/PageColumnAccessInfo.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/PageColumnAccessInfo.java index 518bfdc7d010e..39d8976932b84 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/PageColumnAccessInfo.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/PageColumnAccessInfo.java @@ -19,16 +19,14 @@ package org.apache.iotdb.db.utils.datastructure; -import static org.apache.iotdb.db.storageengine.dataregion.memtable.IWritableMemChunk.MAX_NUMBER_OF_POINTS_IN_PAGE; - public class PageColumnAccessInfo { // time -> (selectedTVList, selectedIndex) private final int[][] indices; private int count; - public PageColumnAccessInfo() { - this.indices = new int[MAX_NUMBER_OF_POINTS_IN_PAGE][]; - for (int i = 0; i < MAX_NUMBER_OF_POINTS_IN_PAGE; i++) { + public PageColumnAccessInfo(int maxNumberOfPointsInPage) { + this.indices = new int[maxNumberOfPointsInPage][]; + for (int i = 0; i < maxNumberOfPointsInPage; i++) { indices[i] = new int[2]; } this.count = 0; diff --git a/iotdb-core/node-commons/src/assembly/resources/conf/iotdb-system.properties.template b/iotdb-core/node-commons/src/assembly/resources/conf/iotdb-system.properties.template index 51861ff9412e1..9affa68623c73 100644 --- a/iotdb-core/node-commons/src/assembly/resources/conf/iotdb-system.properties.template +++ b/iotdb-core/node-commons/src/assembly/resources/conf/iotdb-system.properties.template @@ -1144,7 +1144,7 @@ tvlist_sort_algorithm=TIM # When point number in the working TVList exceeds this, it is sorted and handover in writable memtable # default 0 means it does not handover working tvlist -# effectiveMode: restart +# effectiveMode: hot_reload # Datatype: int tvlist_sort_threshold=0 From a3feeeea35f5eae3788cfd1338b19c5f44b992fb Mon Sep 17 00:00:00 2001 From: shizy Date: Fri, 10 Jan 2025 19:49:45 +0800 Subject: [PATCH 46/54] fix: isNullValue caller --- .../dataregion/memtable/WritableMemChunk.java | 4 +-- .../db/utils/datastructure/BinaryTVList.java | 4 +-- .../db/utils/datastructure/BooleanTVList.java | 4 +-- .../db/utils/datastructure/DoubleTVList.java | 4 +-- .../db/utils/datastructure/FloatTVList.java | 4 +-- .../db/utils/datastructure/IntTVList.java | 4 +-- .../db/utils/datastructure/LongTVList.java | 4 +-- .../iotdb/db/utils/datastructure/TVList.java | 30 +++++++++---------- 8 files changed, 28 insertions(+), 30 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/WritableMemChunk.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/WritableMemChunk.java index a23c371c3e160..79d9cffa3cb7a 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/WritableMemChunk.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/WritableMemChunk.java @@ -350,7 +350,7 @@ private void filterDeletedTimestamp( int[] deletionCursor = {0}; int rowCount = tvlist.rowCount(); for (int i = 0; i < rowCount; i++) { - if (tvlist.getBitMap() != null && tvlist.isNullValue(i)) { + if (tvlist.getBitMap() != null && tvlist.isNullValue(tvlist.getValueIndex(i))) { continue; } long curTime = tvlist.getTime(i); @@ -478,7 +478,7 @@ public String toString() { tvLists.add(list); for (TVList tvList : tvLists) { for (int i = 0; i < tvList.rowCount(); i++) { - if (tvList.isNullValue(i)) { + if (tvList.isNullValue(tvList.getValueIndex(i))) { continue; } size++; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BinaryTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BinaryTVList.java index bf3d51f9752a5..f6f2b53cd44d6 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BinaryTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BinaryTVList.java @@ -148,7 +148,7 @@ protected void writeValidValuesIntoTsBlock( List deletionList) { int[] deleteCursor = {0}; for (int i = 0; i < rowCount; i++) { - if (!isNullValue(i) + if (!isNullValue(getValueIndex(i)) && !isPointDeleted(getTime(i), deletionList, deleteCursor) && (i == rowCount - 1 || getTime(i) != getTime(i + 1))) { builder.getTimeColumnBuilder().writeLong(getTime(i)); @@ -282,7 +282,7 @@ public void serializeToWAL(IWALByteBufferView buffer) { } else { WALWriteUtils.write(new Binary(new byte[0]), buffer); } - WALWriteUtils.write(isNullValue(rowIdx), buffer); + WALWriteUtils.write(isNullValue(getValueIndex(rowIdx)), buffer); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BooleanTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BooleanTVList.java index c32df45b849a4..317d8b62398ba 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BooleanTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BooleanTVList.java @@ -148,7 +148,7 @@ protected void writeValidValuesIntoTsBlock( List deletionList) { int[] deleteCursor = {0}; for (int i = 0; i < rowCount; i++) { - if (!isNullValue(i) + if (!isNullValue(getValueIndex(i)) && !isPointDeleted(getTime(i), deletionList, deleteCursor) && (i == rowCount - 1 || getTime(i) != getTime(i + 1))) { builder.getTimeColumnBuilder().writeLong(getTime(i)); @@ -273,7 +273,7 @@ public void serializeToWAL(IWALByteBufferView buffer) { for (int rowIdx = 0; rowIdx < rowCount; ++rowIdx) { buffer.putLong(getTime(rowIdx)); WALWriteUtils.write(getBoolean(rowIdx), buffer); - WALWriteUtils.write(isNullValue(rowIdx), buffer); + WALWriteUtils.write(isNullValue(getValueIndex(rowIdx)), buffer); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/DoubleTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/DoubleTVList.java index a3acc7495dfed..fc918c2d81472 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/DoubleTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/DoubleTVList.java @@ -152,7 +152,7 @@ protected void writeValidValuesIntoTsBlock( List deletionList) { int[] deleteCursor = {0}; for (int i = 0; i < rowCount; i++) { - if (!isNullValue(i) + if (!isNullValue(getValueIndex(i)) && !isPointDeleted(getTime(i), deletionList, deleteCursor) && (i == rowCount - 1 || getTime(i) != getTime(i + 1))) { builder.getTimeColumnBuilder().writeLong(getTime(i)); @@ -279,7 +279,7 @@ public void serializeToWAL(IWALByteBufferView buffer) { for (int rowIdx = 0; rowIdx < rowCount; ++rowIdx) { buffer.putLong(getTime(rowIdx)); buffer.putDouble(getDouble(rowIdx)); - WALWriteUtils.write(isNullValue(rowIdx), buffer); + WALWriteUtils.write(isNullValue(getValueIndex(rowIdx)), buffer); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/FloatTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/FloatTVList.java index 714f611e0ae88..c39a11565c8a3 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/FloatTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/FloatTVList.java @@ -152,7 +152,7 @@ protected void writeValidValuesIntoTsBlock( List deletionList) { int[] deleteCursor = {0}; for (int i = 0; i < rowCount; i++) { - if (!isNullValue(i) + if (!isNullValue(getValueIndex(i)) && !isPointDeleted(getTime(i), deletionList, deleteCursor) && (i == rowCount - 1 || getTime(i) != getTime(i + 1))) { builder.getTimeColumnBuilder().writeLong(getTime(i)); @@ -279,7 +279,7 @@ public void serializeToWAL(IWALByteBufferView buffer) { for (int rowIdx = 0; rowIdx < rowCount; ++rowIdx) { buffer.putLong(getTime(rowIdx)); buffer.putFloat(getFloat(rowIdx)); - WALWriteUtils.write(isNullValue(rowIdx), buffer); + WALWriteUtils.write(isNullValue(getValueIndex(rowIdx)), buffer); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/IntTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/IntTVList.java index 43d2977708d55..4f7c0c86f4f44 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/IntTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/IntTVList.java @@ -147,7 +147,7 @@ protected void writeValidValuesIntoTsBlock( List deletionList) { int[] deleteCursor = {0}; for (int i = 0; i < rowCount; i++) { - if (!isNullValue(i) + if (!isNullValue(getValueIndex(i)) && !isPointDeleted(getTime(i), deletionList, deleteCursor) && (i == rowCount - 1 || getTime(i) != getTime(i + 1))) { builder.getTimeColumnBuilder().writeLong(getTime(i)); @@ -271,7 +271,7 @@ public void serializeToWAL(IWALByteBufferView buffer) { for (int rowIdx = 0; rowIdx < rowCount; ++rowIdx) { buffer.putLong(getTime(rowIdx)); buffer.putInt(getInt(rowIdx)); - WALWriteUtils.write(isNullValue(rowIdx), buffer); + WALWriteUtils.write(isNullValue(getValueIndex(rowIdx)), buffer); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/LongTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/LongTVList.java index aa73cb32dd40a..39def120d2319 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/LongTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/LongTVList.java @@ -147,7 +147,7 @@ protected void writeValidValuesIntoTsBlock( List deletionList) { int[] deleteCursor = {0}; for (int i = 0; i < rowCount; i++) { - if (!isNullValue(i) + if (!isNullValue(getValueIndex(i)) && !isPointDeleted(getTime(i), deletionList, deleteCursor) && (i == rowCount - 1 || getTime(i) != getTime(i + 1))) { builder.getTimeColumnBuilder().writeLong(getTime(i)); @@ -271,7 +271,7 @@ public void serializeToWAL(IWALByteBufferView buffer) { for (int rowIdx = 0; rowIdx < rowCount; ++rowIdx) { buffer.putLong(getTime(rowIdx)); buffer.putLong(getLong(rowIdx)); - WALWriteUtils.write(isNullValue(rowIdx), buffer); + WALWriteUtils.write(isNullValue(getValueIndex(rowIdx)), buffer); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java index d4552cf98ac9f..fe080ef38b3a9 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java @@ -159,17 +159,13 @@ public int seqRowCount() { return seqRowCount; } - public long chunkSize() { - return 0; - } - public int count() { if (bitMap == null) { return rowCount; } int count = 0; - for (int row = 0; row < rowCount; row++) { - if (!isNullValue(row)) { + for (int rowIdx = 0; rowIdx < rowCount; rowIdx++) { + if (!isNullValue(rowIdx)) { count++; } } @@ -239,18 +235,18 @@ protected void markNullValue(int arrayIndex, int elementIndex) { /** * Get whether value is null at the given position in TvList. * - * @param rowIndex value index + * @param unsortedRowIndex value index * @return boolean */ - public boolean isNullValue(int rowIndex) { - if (rowIndex >= rowCount) { + public boolean isNullValue(int unsortedRowIndex) { + if (unsortedRowIndex >= rowCount) { throw new IndexOutOfBoundsException("Index out of bound error!"); } - if (bitMap == null || bitMap.get(rowIndex / ARRAY_SIZE) == null) { + if (bitMap == null || bitMap.get(unsortedRowIndex / ARRAY_SIZE) == null) { return false; } - int arrayIndex = rowIndex / ARRAY_SIZE; - int elementIndex = rowIndex % ARRAY_SIZE; + int arrayIndex = unsortedRowIndex / ARRAY_SIZE; + int elementIndex = unsortedRowIndex % ARRAY_SIZE; return bitMap.get(arrayIndex).isMarked(elementIndex); } @@ -413,10 +409,12 @@ public int delete(long lowerBound, long upperBound) { long time = getTime(i); if (time >= lowerBound && time <= upperBound) { int originRowIndex = getValueIndex(i); - int arrayIndex = originRowIndex / ARRAY_SIZE; - int elementIndex = originRowIndex % ARRAY_SIZE; - markNullValue(arrayIndex, elementIndex); - deletedNumber++; + if (!isNullValue(originRowIndex)) { + int arrayIndex = originRowIndex / ARRAY_SIZE; + int elementIndex = originRowIndex % ARRAY_SIZE; + markNullValue(arrayIndex, elementIndex); + deletedNumber++; + } } else { maxTime = Math.max(time, maxTime); minTime = Math.min(time, minTime); From cb7e13a8ec0fa752e7a894c8800bd7ac78c1ea3d Mon Sep 17 00:00:00 2001 From: shizy Date: Sat, 11 Jan 2025 21:53:05 +0800 Subject: [PATCH 47/54] fix unit test --- .../read/reader/chunk/MemAlignedChunkLoaderTest.java | 1 + .../dataregion/read/reader/chunk/MemChunkLoaderTest.java | 6 ++++++ 2 files changed, 7 insertions(+) diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemAlignedChunkLoaderTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemAlignedChunkLoaderTest.java index 75a70a4e41956..bae8a0a779fae 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemAlignedChunkLoaderTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemAlignedChunkLoaderTest.java @@ -75,6 +75,7 @@ public void testMemAlignedChunkLoader() throws IOException { Mockito.when(timeStatistics.getCount()).thenReturn(2L); timeStatitsticsList.add(timeStatistics); Mockito.when(chunk.getTimeStatisticsList()).thenReturn(timeStatitsticsList); + Mockito.when(chunk.getMaxNumberOfPointsInPage()).thenReturn(1000); List[]> valuesStatitsticsList = new ArrayList<>(); Statistics[] valuesStatistics = new Statistics[6]; diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemChunkLoaderTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemChunkLoaderTest.java index 003f88ffef918..8351f0dcf1f61 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemChunkLoaderTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemChunkLoaderTest.java @@ -64,6 +64,7 @@ public class MemChunkLoaderTest { public void testBooleanMemChunkLoader() throws IOException { ReadOnlyMemChunk chunk = Mockito.mock(ReadOnlyMemChunk.class); Mockito.when(chunk.getDataType()).thenReturn(TSDataType.BOOLEAN); + Mockito.when(chunk.getMaxNumberOfPointsInPage()).thenReturn(1000); Map booleanTvListMap = buildBooleanTvListMap(); Mockito.when(chunk.getTvListQueryMap()).thenReturn(booleanTvListMap); List booleanTvLists = new ArrayList<>(booleanTvListMap.keySet()); @@ -144,6 +145,7 @@ private Map buildBooleanTvListMap() { public void testInt32MemChunkLoader() throws IOException { ReadOnlyMemChunk chunk = Mockito.mock(ReadOnlyMemChunk.class); Mockito.when(chunk.getDataType()).thenReturn(TSDataType.INT32); + Mockito.when(chunk.getMaxNumberOfPointsInPage()).thenReturn(1000); Map int32TvListMap = buildInt32TvListMap(); Mockito.when(chunk.getTvListQueryMap()).thenReturn(int32TvListMap); List int32TvLists = new ArrayList<>(int32TvListMap.keySet()); @@ -224,6 +226,7 @@ private Map buildInt32TvListMap() { public void testInt64MemChunkLoader() throws IOException { ReadOnlyMemChunk chunk = Mockito.mock(ReadOnlyMemChunk.class); Mockito.when(chunk.getDataType()).thenReturn(TSDataType.INT64); + Mockito.when(chunk.getMaxNumberOfPointsInPage()).thenReturn(1000); Map int64TvListMap = buildInt64TvListMap(); Mockito.when(chunk.getTvListQueryMap()).thenReturn(int64TvListMap); List int64TvLists = new ArrayList<>(int64TvListMap.keySet()); @@ -304,6 +307,7 @@ private Map buildInt64TvListMap() { public void testFloatMemChunkLoader() throws IOException { ReadOnlyMemChunk chunk = Mockito.mock(ReadOnlyMemChunk.class); Mockito.when(chunk.getDataType()).thenReturn(TSDataType.FLOAT); + Mockito.when(chunk.getMaxNumberOfPointsInPage()).thenReturn(1000); Map floatTvListMap = buildFloatTvListMap(); Mockito.when(chunk.getTvListQueryMap()).thenReturn(floatTvListMap); List floatTvLists = new ArrayList<>(floatTvListMap.keySet()); @@ -384,6 +388,7 @@ private Map buildFloatTvListMap() { public void testDoubleMemChunkLoader() throws IOException { ReadOnlyMemChunk chunk = Mockito.mock(ReadOnlyMemChunk.class); Mockito.when(chunk.getDataType()).thenReturn(TSDataType.DOUBLE); + Mockito.when(chunk.getMaxNumberOfPointsInPage()).thenReturn(1000); Map doubleTvListMap = buildDoubleTvListMap(); Mockito.when(chunk.getTvListQueryMap()).thenReturn(doubleTvListMap); List doubleTvLists = new ArrayList<>(doubleTvListMap.keySet()); @@ -464,6 +469,7 @@ private Map buildDoubleTvListMap() { public void testTextMemChunkLoader() throws IOException { ReadOnlyMemChunk chunk = Mockito.mock(ReadOnlyMemChunk.class); Mockito.when(chunk.getDataType()).thenReturn(TSDataType.TEXT); + Mockito.when(chunk.getMaxNumberOfPointsInPage()).thenReturn(1000); Map textTvListMap = buildTextTvListMap(); Mockito.when(chunk.getTvListQueryMap()).thenReturn(textTvListMap); List textTvLists = new ArrayList<>(textTvListMap.keySet()); From 1dc41929af2ca67396302f709136a7f76e640572 Mon Sep 17 00:00:00 2001 From: shizy Date: Sat, 11 Jan 2025 21:57:20 +0800 Subject: [PATCH 48/54] refactor: abstract prepareTvListMapForQuery method --- .../execution/fragment/QueryContext.java | 5 - .../utils/ResourceByPathUtils.java | 309 ++++++------------ .../memtable/AlignedReadOnlyMemChunk.java | 28 +- .../memtable/AlignedWritableMemChunk.java | 6 +- .../memtable/IWritableMemChunk.java | 10 +- .../dataregion/memtable/WritableMemChunk.java | 5 +- .../chunk/MemAlignedChunkLoaderTest.java | 11 +- 7 files changed, 146 insertions(+), 228 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/QueryContext.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/QueryContext.java index fed6e7e9f61e6..2aa6f9779efd0 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/QueryContext.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/QueryContext.java @@ -26,7 +26,6 @@ import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileID; import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; import org.apache.iotdb.db.utils.ModificationUtils; -import org.apache.iotdb.db.utils.datastructure.AlignedTVList; import org.apache.iotdb.db.utils.datastructure.PatternTreeMapFactory; import org.apache.iotdb.db.utils.datastructure.PatternTreeMapFactory.ModsSerializer; import org.apache.iotdb.db.utils.datastructure.TVList; @@ -216,8 +215,4 @@ public void setIgnoreAllNullRows(boolean ignoreAllNullRows) { public void addTVListToSet(Map tvListMap) { tvListSet.addAll(tvListMap.keySet()); } - - public void addAlignedTVListToSet(Map alignedTvListMap) { - tvListSet.addAll(alignedTvListMap.keySet()); - } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/utils/ResourceByPathUtils.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/utils/ResourceByPathUtils.java index ae7f2eba05f7e..1de01eed345b0 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/utils/ResourceByPathUtils.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/utils/ResourceByPathUtils.java @@ -30,13 +30,12 @@ import org.apache.iotdb.db.storageengine.dataregion.memtable.AlignedWritableMemChunk; import org.apache.iotdb.db.storageengine.dataregion.memtable.AlignedWritableMemChunkGroup; import org.apache.iotdb.db.storageengine.dataregion.memtable.IMemTable; +import org.apache.iotdb.db.storageengine.dataregion.memtable.IWritableMemChunk; import org.apache.iotdb.db.storageengine.dataregion.memtable.IWritableMemChunkGroup; import org.apache.iotdb.db.storageengine.dataregion.memtable.ReadOnlyMemChunk; -import org.apache.iotdb.db.storageengine.dataregion.memtable.WritableMemChunk; import org.apache.iotdb.db.storageengine.dataregion.modification.ModEntry; import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; import org.apache.iotdb.db.utils.ModificationUtils; -import org.apache.iotdb.db.utils.datastructure.AlignedTVList; import org.apache.iotdb.db.utils.datastructure.TVList; import org.apache.tsfile.enums.TSDataType; @@ -74,6 +73,7 @@ * MeasurementPath have different implementations, and the default PartialPath should not use it. */ public abstract class ResourceByPathUtils { + private static final Logger LOGGER = LoggerFactory.getLogger(ResourceByPathUtils.class); public static ResourceByPathUtils getResourceInstance(IFullPath path) { if (path instanceof AlignedFullPath) { @@ -101,10 +101,112 @@ public abstract List getVisibleMetadataListFromWriter( TsFileResource tsFileResource, QueryContext context, long timeLowerBound); + + /** + * Prepare the TVList references for the query. We remember TVLists' row count here and determine + * whether the TVLists needs sorting later during operator execution based on it. It need not + * protect sorted list. Sorted list is changed in the handover process of inserting, which holds + * the data region write lock. At this moment, query thread holds the data region read lock. + * + * @param context query context + * @param memChunk writable memchunk + * @param isWorkMemTable in working or flushing memtable + * @param globalTimeFilter global time filter + * @return Map + */ + protected Map prepareTvListMapForQuery( + QueryContext context, + IWritableMemChunk memChunk, + boolean isWorkMemTable, + Filter globalTimeFilter) { + // should copy globalTimeFilter because GroupByMonthFilter is stateful + Filter copyTimeFilter = null; + if (globalTimeFilter != null) { + copyTimeFilter = globalTimeFilter.copy(); + } + + Map tvListQueryMap = new LinkedHashMap<>(); + // immutable sorted lists + for (TVList tvList : memChunk.getSortedList()) { + if (copyTimeFilter != null + && !copyTimeFilter.satisfyStartEndTime(tvList.getMinTime(), tvList.getMaxTime())) { + continue; + } + tvList.lockQueryList(); + try { + LOGGER.debug( + "Flushing/Working MemTable - add current query context to immutable TVList's query list"); + tvList.getQueryContextList().add(context); + tvListQueryMap.put(tvList, tvList.rowCount()); + } finally { + tvList.unlockQueryList(); + } + } + + // mutable tvlist + TVList list = memChunk.getWorkingTVList(); + TVList cloneList = null; + list.lockQueryList(); + try { + if (copyTimeFilter != null + && !copyTimeFilter.satisfyStartEndTime(list.getMinTime(), list.getMaxTime())) { + return tvListQueryMap; + } + + if (!isWorkMemTable) { + LOGGER.debug( + "Flushing MemTable - add current query context to mutable TVList's query list"); + list.getQueryContextList().add(context); + tvListQueryMap.put(list, list.rowCount()); + } else { + if (list.isSorted() || list.getQueryContextList().isEmpty()) { + LOGGER.debug( + "Working MemTable - add current query context to mutable TVList's query list when it's sorted or no other query on it"); + list.getQueryContextList().add(context); + tvListQueryMap.put(list, list.rowCount()); + } else { + /* + * +----------------------+ + * | MemTable | + * | | + * | +------------+ | +-----------------+ + * | | TVList |<---+--+ +---+ Previous Query | + * | +-----^------+ | | | +-----------------+ + * | | | | | + * +----------+-----------+ | | +----------------+ + * | Clone +---+---+ Current Query | + * +-----+------+ | +----------------+ + * | TVList | <---------+ + * +------------+ + */ + LOGGER.debug( + "Working MemTable - clone mutable TVList and replace old TVList in working MemTable"); + QueryContext firstQuery = list.getQueryContextList().get(0); + // reserve query memory + if (firstQuery instanceof FragmentInstanceContext) { + MemoryReservationManager memoryReservationManager = + ((FragmentInstanceContext) firstQuery).getMemoryReservationContext(); + memoryReservationManager.reserveMemoryCumulatively(list.calculateRamSize()); + } + list.setOwnerQuery(firstQuery); + + // clone TVList + cloneList = list.clone(); + cloneList.getQueryContextList().add(context); + tvListQueryMap.put(cloneList, cloneList.rowCount()); + } + } + } finally { + list.unlockQueryList(); + } + if (cloneList != null) { + memChunk.setWorkingTVList(cloneList); + } + return tvListQueryMap; + } } class AlignedResourceByPathUtils extends ResourceByPathUtils { - private static final Logger LOGGER = LoggerFactory.getLogger(AlignedResourceByPathUtils.class); AlignedFullPath alignedFullPath; @@ -220,8 +322,8 @@ public ReadOnlyMemChunk getReadOnlyMemChunkFromMemTable( } // prepare AlignedTVList for query. It should clone TVList if necessary. - Map alignedTvListQueryMap = - prepareAlignedTvListMapForQuery( + Map alignedTvListQueryMap = + prepareTvListMapForQuery( context, alignedMemChunk, modsToMemtable == null, globalTimeFilter); // column index list for the query @@ -337,99 +439,10 @@ public List getVisibleMetadataListFromWriter( chunkMetadataList.removeIf(x -> x.getEndTime() < timeLowerBound); return new ArrayList<>(chunkMetadataList); } - - /** - * Prepare the AlignedTVList references for the query. Same logic as prepareTvListMapForQuery - * - * @param context query context - * @param alignedMemChunk aligned writable memchunk - * @param isWorkMemTable in working or flushing memtable - * @param globalTimeFilter global time filter - * @return Map - */ - private Map prepareAlignedTvListMapForQuery( - QueryContext context, - AlignedWritableMemChunk alignedMemChunk, - boolean isWorkMemTable, - Filter globalTimeFilter) { - // should copy globalTimeFilter because GroupByMonthFilter is stateful - Filter copyTimeFilter = null; - if (globalTimeFilter != null) { - copyTimeFilter = globalTimeFilter.copy(); - } - - Map alignedTvListQueryMap = new LinkedHashMap<>(); - // immutable aligned TVList - for (AlignedTVList alignedTvList : alignedMemChunk.getSortedList()) { - if (copyTimeFilter != null - && !copyTimeFilter.satisfyStartEndTime( - alignedTvList.getMinTime(), alignedTvList.getMaxTime())) { - continue; - } - alignedTvList.lockQueryList(); - try { - LOGGER.debug( - "Flushing/Working MemTable - add current query context to immutable AlignedTVList's query list"); - alignedTvList.getQueryContextList().add(context); - alignedTvListQueryMap.put(alignedTvList, alignedTvList.rowCount()); - } finally { - alignedTvList.unlockQueryList(); - } - } - - // mutable aligned TVList - AlignedTVList list = alignedMemChunk.getWorkingTVList(); - AlignedTVList cloneList = null; - list.lockQueryList(); - try { - if (copyTimeFilter != null - && !copyTimeFilter.satisfyStartEndTime(list.getMinTime(), list.getMaxTime())) { - return alignedTvListQueryMap; - } - if (!isWorkMemTable) { - LOGGER.debug( - "Flushing MemTable - add current query context to mutable AlignedTVList's query list"); - list.getQueryContextList().add(context); - alignedTvListQueryMap.put(list, list.rowCount()); - } else { - if (list.isSorted() || list.getQueryContextList().isEmpty()) { - LOGGER.debug( - "Working MemTable - add current query context to mutable AlignedTVList's query list when it's sorted or no other query on it"); - list.getQueryContextList().add(context); - alignedTvListQueryMap.put(list, list.rowCount()); - } else { - LOGGER.debug( - "Working MemTable - clone mutable AlignedTVList and replace old AlignedTVList in working MemTable"); - QueryContext firstQuery = list.getQueryContextList().get(0); - // reserve query memory - if (firstQuery instanceof FragmentInstanceContext) { - MemoryReservationManager memoryReservationManager = - ((FragmentInstanceContext) firstQuery).getMemoryReservationContext(); - memoryReservationManager.reserveMemoryCumulatively(list.calculateRamSize()); - } - list.setOwnerQuery(firstQuery); - - // clone TVList - cloneList = list.clone(); - cloneList.getQueryContextList().add(context); - alignedTvListQueryMap.put(cloneList, cloneList.rowCount()); - } - } - } finally { - list.unlockQueryList(); - } - if (cloneList != null) { - alignedMemChunk.setWorkingTVList(cloneList); - } - return alignedTvListQueryMap; - } } class MeasurementResourceByPathUtils extends ResourceByPathUtils { - private static final Logger LOGGER = - LoggerFactory.getLogger(MeasurementResourceByPathUtils.class); - NonAlignedFullPath fullPath; protected MeasurementResourceByPathUtils(IFullPath fullPath) { @@ -484,9 +497,8 @@ public ReadOnlyMemChunk getReadOnlyMemChunkFromMemTable( || !memTableMap.get(deviceID).contains(fullPath.getMeasurement())) { return null; } - WritableMemChunk memChunk = - (WritableMemChunk) - memTableMap.get(deviceID).getMemChunkMap().get(fullPath.getMeasurement()); + IWritableMemChunk memChunk = + memTableMap.get(deviceID).getMemChunkMap().get(fullPath.getMeasurement()); // prepare TVList for query. It should clone TVList if necessary. Map tvListQueryMap = prepareTvListMapForQuery(context, memChunk, modsToMemtable == null, globalTimeFilter); @@ -531,107 +543,4 @@ public List getVisibleMetadataListFromWriter( chunkMetadataList.removeIf(x -> x.getEndTime() < timeLowerBound); return chunkMetadataList; } - - /** - * Prepare the TVList references for the query. We remember TVLists' row count here and determine - * whether the TVLists needs sorting later during operator execution based on it. It need not - * protect sorted list. Sorted list is changed in the handover process of inserting, which holds - * the data region write lock. At this moment, query thread holds the data region read lock. - * - * @param context query context - * @param memChunk writable memchunk - * @param isWorkMemTable in working or flushing memtable - * @param globalTimeFilter global time filter - * @return Map - */ - private Map prepareTvListMapForQuery( - QueryContext context, - WritableMemChunk memChunk, - boolean isWorkMemTable, - Filter globalTimeFilter) { - // should copy globalTimeFilter because GroupByMonthFilter is stateful - Filter copyTimeFilter = null; - if (globalTimeFilter != null) { - copyTimeFilter = globalTimeFilter.copy(); - } - - Map tvListQueryMap = new LinkedHashMap<>(); - // immutable sorted lists - for (TVList tvList : memChunk.getSortedList()) { - if (copyTimeFilter != null - && !copyTimeFilter.satisfyStartEndTime(tvList.getMinTime(), tvList.getMaxTime())) { - continue; - } - tvList.lockQueryList(); - try { - LOGGER.debug( - "Flushing/Working MemTable - add current query context to immutable TVList's query list"); - tvList.getQueryContextList().add(context); - tvListQueryMap.put(tvList, tvList.rowCount()); - } finally { - tvList.unlockQueryList(); - } - } - - // mutable tvlist - TVList list = memChunk.getWorkingTVList(); - TVList cloneList = null; - list.lockQueryList(); - try { - if (copyTimeFilter != null - && !copyTimeFilter.satisfyStartEndTime(list.getMinTime(), list.getMaxTime())) { - return tvListQueryMap; - } - - if (!isWorkMemTable) { - LOGGER.debug( - "Flushing MemTable - add current query context to mutable TVList's query list"); - list.getQueryContextList().add(context); - tvListQueryMap.put(list, list.rowCount()); - } else { - if (list.isSorted() || list.getQueryContextList().isEmpty()) { - LOGGER.debug( - "Working MemTable - add current query context to mutable TVList's query list when it's sorted or no other query on it"); - list.getQueryContextList().add(context); - tvListQueryMap.put(list, list.rowCount()); - } else { - /* - * +----------------------+ - * | MemTable | - * | | - * | +------------+ | +-----------------+ - * | | TVList |<---+--+ +---+ Previous Query | - * | +-----^------+ | | | +-----------------+ - * | | | | | - * +----------+-----------+ | | +----------------+ - * | Clone +---+---+ Current Query | - * +-----+------+ | +----------------+ - * | TVList | <---------+ - * +------------+ - */ - LOGGER.debug( - "Working MemTable - clone mutable TVList and replace old TVList in working MemTable"); - QueryContext firstQuery = list.getQueryContextList().get(0); - // reserve query memory - if (firstQuery instanceof FragmentInstanceContext) { - MemoryReservationManager memoryReservationManager = - ((FragmentInstanceContext) firstQuery).getMemoryReservationContext(); - memoryReservationManager.reserveMemoryCumulatively(list.calculateRamSize()); - } - list.setOwnerQuery(firstQuery); - - // clone TVList - cloneList = list.clone(); - cloneList.getQueryContextList().add(context); - tvListQueryMap.put(cloneList, cloneList.rowCount()); - } - } - } finally { - list.unlockQueryList(); - } - if (cloneList != null) { - memChunk.setWorkingTVList(cloneList); - } - return tvListQueryMap; - } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedReadOnlyMemChunk.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedReadOnlyMemChunk.java index d898db066bb97..09bb9d8bc0130 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedReadOnlyMemChunk.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedReadOnlyMemChunk.java @@ -50,6 +50,7 @@ import java.util.Arrays; import java.util.List; import java.util.Map; +import java.util.stream.Collectors; import static org.apache.iotdb.db.utils.ModificationUtils.isPointDeleted; @@ -71,7 +72,7 @@ public class AlignedReadOnlyMemChunk extends ReadOnlyMemChunk { private final List[]> valueStatisticsList; // AlignedTVList rowCount during query - protected Map alignedTvListQueryMap; + protected Map alignedTvListQueryMap; // For example, it stores time series [s1, s2, s3] in AlignedWritableMemChunk. // When we select two of time series [s1, s3], the column index list should be [0, 2] @@ -93,7 +94,7 @@ public AlignedReadOnlyMemChunk( QueryContext context, List columnIndexList, IMeasurementSchema schema, - Map alignedTvListQueryMap, + Map alignedTvListQueryMap, List timeColumnDeletion, List> valueColumnsDeletionList) { super(context); @@ -109,13 +110,13 @@ public AlignedReadOnlyMemChunk( this.valueStatisticsList = new ArrayList<>(); this.alignedTvListQueryMap = alignedTvListQueryMap; this.columnIndexList = columnIndexList; - this.context.addAlignedTVListToSet(alignedTvListQueryMap); + this.context.addTVListToSet(alignedTvListQueryMap); } @Override public void sortTvLists() { - for (Map.Entry entry : getAligendTvListQueryMap().entrySet()) { - AlignedTVList alignedTvList = entry.getKey(); + for (Map.Entry entry : getAligendTvListQueryMap().entrySet()) { + AlignedTVList alignedTvList = (AlignedTVList) entry.getKey(); int queryRowCount = entry.getValue(); if (!alignedTvList.isSorted() && queryRowCount > alignedTvList.seqRowCount()) { alignedTvList.sort(); @@ -134,7 +135,11 @@ public void initChunkMetaFromTvLists() { new Statistics[valueChunkNames.size()]; // create MergeSortAlignedTVListIterator - List alignedTvLists = new ArrayList<>(alignedTvListQueryMap.keySet()); + List alignedTvLists = + alignedTvListQueryMap.keySet().stream() + .map(x -> (AlignedTVList) x) + .collect(Collectors.toList()); + timeValuePairIterator = new MergeSortAlignedTVListIterator( alignedTvLists, @@ -287,8 +292,8 @@ public boolean isEmpty() { @Override public IPointReader getPointReader() { - for (Map.Entry entry : alignedTvListQueryMap.entrySet()) { - AlignedTVList tvList = entry.getKey(); + for (Map.Entry entry : alignedTvListQueryMap.entrySet()) { + AlignedTVList tvList = (AlignedTVList) entry.getKey(); int queryLength = entry.getValue(); if (!tvList.isSorted() && queryLength > tvList.seqRowCount()) { tvList.sort(); @@ -318,7 +323,10 @@ private boolean isAllColumnNull(TsPrimitiveType[] primitiveValues) { } private void writeValidValuesIntoTsBlock(TsBlockBuilder builder) throws IOException { - List alignedTvLists = new ArrayList<>(alignedTvListQueryMap.keySet()); + List alignedTvLists = + alignedTvListQueryMap.keySet().stream() + .map(x -> (AlignedTVList) x) + .collect(Collectors.toList()); MergeSortAlignedTVListIterator timeValuePairIterator = new MergeSortAlignedTVListIterator( alignedTvLists, @@ -399,7 +407,7 @@ && isPointDeleted(tvPair.getTimestamp(), timeColumnDeletion, timeDeleteCursor)) } } - public Map getAligendTvListQueryMap() { + public Map getAligendTvListQueryMap() { return alignedTvListQueryMap; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedWritableMemChunk.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedWritableMemChunk.java index d903a0ae2cf7e..1ba99166137e2 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedWritableMemChunk.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedWritableMemChunk.java @@ -368,8 +368,9 @@ public AlignedTVList getWorkingTVList() { return list; } - public void setWorkingTVList(AlignedTVList list) { - this.list = list; + @Override + public void setWorkingTVList(TVList list) { + this.list = (AlignedTVList) list; } @Override @@ -984,6 +985,7 @@ public boolean isAllDeleted() { return true; } + @Override public List getSortedList() { return sortedList; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/IWritableMemChunk.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/IWritableMemChunk.java index 576b82bff4dc7..3e5cd3a8b1600 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/IWritableMemChunk.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/IWritableMemChunk.java @@ -127,10 +127,6 @@ void writeAlignedTablet( */ void sortTvListForFlush(); - default TVList getWorkingTVList() { - return null; - } - default long getMaxTime() { return Long.MAX_VALUE; } @@ -155,4 +151,10 @@ default long getMinTime() { long getLastPoint(); boolean isEmpty(); + + List getSortedList(); + + TVList getWorkingTVList(); + + void setWorkingTVList(TVList list); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/WritableMemChunk.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/WritableMemChunk.java index 79d9cffa3cb7a..c615bd3489a03 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/WritableMemChunk.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/WritableMemChunk.java @@ -30,7 +30,6 @@ import org.apache.iotdb.db.utils.datastructure.MergeSortTvListIterator; import org.apache.iotdb.db.utils.datastructure.TVList; -import org.apache.tsfile.common.conf.TSFileDescriptor; import org.apache.tsfile.enums.TSDataType; import org.apache.tsfile.read.TimeValuePair; import org.apache.tsfile.read.common.TimeRange; @@ -68,8 +67,6 @@ public class WritableMemChunk implements IWritableMemChunk { private final long TARGET_CHUNK_SIZE = CONFIG.getTargetChunkSize(); private final long MAX_NUMBER_OF_POINTS_IN_CHUNK = CONFIG.getTargetChunkPointNum(); private final int TVLIST_SORT_THRESHOLD = CONFIG.getTvListSortThreshold(); - private int MAX_NUMBER_OF_POINTS_IN_PAGE = - TSFileDescriptor.getInstance().getConfig().getMaxNumberOfPointsInPage(); public WritableMemChunk(IMeasurementSchema schema) { this.schema = schema; @@ -386,6 +383,7 @@ public TVList getWorkingTVList() { return list; } + @Override public void setWorkingTVList(TVList list) { this.list = list; } @@ -754,6 +752,7 @@ public static WritableMemChunk deserialize(DataInputStream stream) throws IOExce return memChunk; } + @Override public List getSortedList() { return sortedList; } diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemAlignedChunkLoaderTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemAlignedChunkLoaderTest.java index bae8a0a779fae..a6a9cf9926fe7 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemAlignedChunkLoaderTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemAlignedChunkLoaderTest.java @@ -23,6 +23,7 @@ import org.apache.iotdb.db.storageengine.dataregion.memtable.AlignedReadOnlyMemChunk; import org.apache.iotdb.db.utils.datastructure.AlignedTVList; import org.apache.iotdb.db.utils.datastructure.MergeSortAlignedTVListIterator; +import org.apache.iotdb.db.utils.datastructure.TVList; import org.apache.tsfile.common.conf.TSFileConfig; import org.apache.tsfile.enums.TSDataType; @@ -44,6 +45,7 @@ import java.util.LinkedHashMap; import java.util.List; import java.util.Map; +import java.util.stream.Collectors; import static org.apache.tsfile.read.reader.series.PaginationController.UNLIMITED_PAGINATION_CONTROLLER; import static org.junit.Assert.assertEquals; @@ -109,9 +111,10 @@ public void testMemAlignedChunkLoader() throws IOException { Mockito.when(chunk.getValueColumnsDeletionList()).thenReturn(null); Mockito.when(chunk.getContext()).thenReturn(ctx); - Map alignedTvListMap = buildAlignedTvListMap(); + Map alignedTvListMap = buildAlignedTvListMap(); Mockito.when(chunk.getAligendTvListQueryMap()).thenReturn(alignedTvListMap); - List alignedTvLists = new ArrayList<>(alignedTvListMap.keySet()); + List alignedTvLists = + alignedTvListMap.keySet().stream().map(x -> (AlignedTVList) x).collect(Collectors.toList()); MergeSortAlignedTVListIterator timeValuePairIterator = new MergeSortAlignedTVListIterator(alignedTvLists, dataTypes, null, null, null, false); Mockito.when(chunk.getMergeSortAlignedTVListIterator()).thenReturn(timeValuePairIterator); @@ -179,7 +182,7 @@ private List buildTsDataTypes() { TSDataType.TEXT); } - private Map buildAlignedTvListMap() { + private Map buildAlignedTvListMap() { List dataTypes = buildTsDataTypes(); AlignedTVList tvList1 = AlignedTVList.newAlignedList(dataTypes); tvList1.putAlignedValue( @@ -195,7 +198,7 @@ private Map buildAlignedTvListMap() { true, 1, 1L, 1.1f, null, new Binary(BINARY_STR, TSFileConfig.STRING_CHARSET) }); - Map tvListMap = new LinkedHashMap<>(); + Map tvListMap = new LinkedHashMap<>(); tvListMap.put(tvList1, 2); tvListMap.put(tvList2, 1); return tvListMap; From bab27fdf8e8a66b5092e46fdf267b7cf6c653338 Mon Sep 17 00:00:00 2001 From: shizy Date: Sun, 12 Jan 2025 17:15:31 +0800 Subject: [PATCH 49/54] refactor: clear/clone/expand indices and bitmap --- .../db/utils/datastructure/AlignedTVList.java | 20 ++++---- .../db/utils/datastructure/BinaryTVList.java | 14 ++--- .../db/utils/datastructure/BooleanTVList.java | 10 ++-- .../db/utils/datastructure/DoubleTVList.java | 10 ++-- .../db/utils/datastructure/FloatTVList.java | 10 ++-- .../db/utils/datastructure/IntTVList.java | 10 ++-- .../db/utils/datastructure/LongTVList.java | 10 ++-- .../iotdb/db/utils/datastructure/TVList.java | 51 +++++++++---------- 8 files changed, 72 insertions(+), 63 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/AlignedTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/AlignedTVList.java index 70381f3660ee2..708b741e9b7d7 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/AlignedTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/AlignedTVList.java @@ -148,10 +148,8 @@ public AlignedTVList clone() { cloneList.timeDeletedCnt = this.timeDeletedCnt; System.arraycopy( memoryBinaryChunkSize, 0, cloneList.memoryBinaryChunkSize, 0, dataTypes.size()); - for (int[] indicesArray : indices) { - cloneList.indices.add(cloneIndex(indicesArray)); - } for (int i = 0; i < values.size(); i++) { + // Clone value List columnValues = values.get(i); for (Object valueArray : columnValues) { cloneList.values.get(i).add(cloneValue(dataTypes.get(i), valueArray)); @@ -666,13 +664,7 @@ protected Object cloneValue(TSDataType type, Object value) { } @Override - public void clearValue() { - if (indices != null) { - for (int[] dataArray : indices) { - PrimitiveArrayManager.release(dataArray); - } - indices.clear(); - } + protected void clearValue() { for (int i = 0; i < dataTypes.size(); i++) { List columnValues = values.get(i); if (columnValues != null) { @@ -681,13 +673,19 @@ public void clearValue() { } columnValues.clear(); } + memoryBinaryChunkSize[i] = 0; + } + } + + @Override + protected void clearBitMap() { + for (int i = 0; i < dataTypes.size(); i++) { if (bitMaps != null) { List columnBitMaps = bitMaps.get(i); if (columnBitMaps != null) { columnBitMaps.clear(); } } - memoryBinaryChunkSize[i] = 0; } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BinaryTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BinaryTVList.java index f6f2b53cd44d6..8818ab3643b20 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BinaryTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BinaryTVList.java @@ -64,10 +64,10 @@ public static BinaryTVList newList() { } @Override - public TimBinaryTVList clone() { - TimBinaryTVList cloneList = new TimBinaryTVList(); + public BinaryTVList clone() { + BinaryTVList cloneList = BinaryTVList.newList(); cloneAs(cloneList); - cloneSlicesAndBitMap(cloneList); + cloneBitMap(cloneList); for (Binary[] valueArray : values) { cloneList.values.add(cloneValue(valueArray)); } @@ -112,20 +112,22 @@ public Binary getBinary(int index) { } @Override - void clearValue() { + protected void clearValue() { if (values != null) { for (Binary[] dataArray : values) { PrimitiveArrayManager.release(dataArray); } values.clear(); } - clearSlicesAndBitMap(); } @Override protected void expandValues() { + indices.add((int[]) getPrimitiveArraysByType(TSDataType.INT32)); values.add((Binary[]) getPrimitiveArraysByType(TSDataType.TEXT)); - expandSlicesAndBitMap(); + if (bitMap != null) { + bitMap.add(null); + } } @Override diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BooleanTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BooleanTVList.java index 317d8b62398ba..e92d90a02b264 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BooleanTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BooleanTVList.java @@ -66,7 +66,7 @@ public static BooleanTVList newList() { public BooleanTVList clone() { BooleanTVList cloneList = BooleanTVList.newList(); cloneAs(cloneList); - cloneSlicesAndBitMap(cloneList); + cloneBitMap(cloneList); for (boolean[] valueArray : values) { cloneList.values.add(cloneValue(valueArray)); } @@ -111,20 +111,22 @@ public boolean getBoolean(int index) { } @Override - void clearValue() { + protected void clearValue() { if (values != null) { for (boolean[] dataArray : values) { PrimitiveArrayManager.release(dataArray); } values.clear(); } - clearSlicesAndBitMap(); } @Override protected void expandValues() { + indices.add((int[]) getPrimitiveArraysByType(TSDataType.INT32)); values.add((boolean[]) getPrimitiveArraysByType(TSDataType.BOOLEAN)); - expandSlicesAndBitMap(); + if (bitMap != null) { + bitMap.add(null); + } } @Override diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/DoubleTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/DoubleTVList.java index fc918c2d81472..03a069cfb3d18 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/DoubleTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/DoubleTVList.java @@ -67,7 +67,7 @@ public static DoubleTVList newList() { public DoubleTVList clone() { DoubleTVList cloneList = DoubleTVList.newList(); cloneAs(cloneList); - cloneSlicesAndBitMap(cloneList); + cloneBitMap(cloneList); for (double[] valueArray : values) { cloneList.values.add(cloneValue(valueArray)); } @@ -112,20 +112,22 @@ public double getDouble(int index) { } @Override - void clearValue() { + protected void clearValue() { if (values != null) { for (double[] dataArray : values) { PrimitiveArrayManager.release(dataArray); } values.clear(); } - clearSlicesAndBitMap(); } @Override protected void expandValues() { + indices.add((int[]) getPrimitiveArraysByType(TSDataType.INT32)); values.add((double[]) getPrimitiveArraysByType(TSDataType.DOUBLE)); - expandSlicesAndBitMap(); + if (bitMap != null) { + bitMap.add(null); + } } @Override diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/FloatTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/FloatTVList.java index c39a11565c8a3..d9236185721f5 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/FloatTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/FloatTVList.java @@ -67,7 +67,7 @@ public static FloatTVList newList() { public FloatTVList clone() { FloatTVList cloneList = FloatTVList.newList(); cloneAs(cloneList); - cloneSlicesAndBitMap(cloneList); + cloneBitMap(cloneList); for (float[] valueArray : values) { cloneList.values.add(cloneValue(valueArray)); } @@ -112,20 +112,22 @@ public float getFloat(int index) { } @Override - void clearValue() { + protected void clearValue() { if (values != null) { for (float[] dataArray : values) { PrimitiveArrayManager.release(dataArray); } values.clear(); } - clearSlicesAndBitMap(); } @Override protected void expandValues() { + indices.add((int[]) getPrimitiveArraysByType(TSDataType.INT32)); values.add((float[]) getPrimitiveArraysByType(TSDataType.FLOAT)); - expandSlicesAndBitMap(); + if (bitMap != null) { + bitMap.add(null); + } } @Override diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/IntTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/IntTVList.java index 4f7c0c86f4f44..e12dd4a1e0b88 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/IntTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/IntTVList.java @@ -66,7 +66,7 @@ public static IntTVList newList() { public IntTVList clone() { IntTVList cloneList = IntTVList.newList(); cloneAs(cloneList); - cloneSlicesAndBitMap(cloneList); + cloneBitMap(cloneList); for (int[] valueArray : values) { cloneList.values.add(cloneValue(valueArray)); } @@ -111,20 +111,22 @@ public int getInt(int index) { } @Override - void clearValue() { + protected void clearValue() { if (values != null) { for (int[] dataArray : values) { PrimitiveArrayManager.release(dataArray); } values.clear(); } - clearSlicesAndBitMap(); } @Override protected void expandValues() { + indices.add((int[]) getPrimitiveArraysByType(TSDataType.INT32)); values.add((int[]) getPrimitiveArraysByType(TSDataType.INT32)); - expandSlicesAndBitMap(); + if (bitMap != null) { + bitMap.add(null); + } } @Override diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/LongTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/LongTVList.java index 39def120d2319..1f9f7985b4768 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/LongTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/LongTVList.java @@ -66,7 +66,7 @@ public static LongTVList newList() { public LongTVList clone() { LongTVList cloneList = LongTVList.newList(); cloneAs(cloneList); - cloneSlicesAndBitMap(cloneList); + cloneBitMap(cloneList); for (long[] valueArray : values) { cloneList.values.add(cloneValue(valueArray)); } @@ -111,20 +111,22 @@ public long getLong(int index) { } @Override - void clearValue() { + protected void clearValue() { if (values != null) { for (long[] dataArray : values) { PrimitiveArrayManager.release(dataArray); } values.clear(); } - clearSlicesAndBitMap(); } @Override protected void expandValues() { + indices.add((int[]) getPrimitiveArraysByType(TSDataType.INT32)); values.add((long[]) getPrimitiveArraysByType(TSDataType.INT64)); - expandSlicesAndBitMap(); + if (bitMap != null) { + bitMap.add(null); + } } @Override diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java index fe080ef38b3a9..53174f9bda44c 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java @@ -250,12 +250,7 @@ public boolean isNullValue(int unsortedRowIndex) { return bitMap.get(arrayIndex).isMarked(elementIndex); } - protected void cloneSlicesAndBitMap(TVList cloneList) { - if (indices != null) { - for (int[] indicesArray : indices) { - cloneList.indices.add(cloneIndex(indicesArray)); - } - } + protected void cloneBitMap(TVList cloneList) { if (bitMap != null) { cloneList.bitMap = new ArrayList<>(); for (BitMap bm : bitMap) { @@ -264,25 +259,6 @@ protected void cloneSlicesAndBitMap(TVList cloneList) { } } - protected void clearSlicesAndBitMap() { - if (indices != null) { - for (int[] dataArray : indices) { - PrimitiveArrayManager.release(dataArray); - } - indices.clear(); - } - if (bitMap != null) { - bitMap.clear(); - } - } - - protected void expandSlicesAndBitMap() { - indices.add((int[]) getPrimitiveArraysByType(TSDataType.INT32)); - if (bitMap != null) { - bitMap.add(null); - } - } - public void putLong(long time, long value) { throw new UnsupportedOperationException(ERR_DATATYPE_NOT_CONSISTENT); } @@ -423,10 +399,16 @@ public int delete(long lowerBound, long upperBound) { return deletedNumber; } + // common clone for both TVList and AlignedTVList protected void cloneAs(TVList cloneList) { + // clone timestamps for (long[] timestampArray : timestamps) { cloneList.timestamps.add(cloneTime(timestampArray)); } + // clone indices + for (int[] indicesArray : indices) { + cloneList.indices.add(cloneIndex(indicesArray)); + } cloneList.rowCount = rowCount; cloneList.seqRowCount = seqRowCount; cloneList.sorted = sorted; @@ -444,6 +426,8 @@ public void clear() { ownerQuery = null; clearTime(); clearValue(); + clearIndices(); + clearBitMap(); } protected void clearTime() { @@ -455,7 +439,22 @@ protected void clearTime() { } } - abstract void clearValue(); + protected abstract void clearValue(); + + protected void clearIndices() { + if (indices != null) { + for (int[] dataArray : indices) { + PrimitiveArrayManager.release(dataArray); + } + indices.clear(); + } + } + + protected void clearBitMap() { + if (bitMap != null) { + bitMap.clear(); + } + } protected void checkExpansion() { if ((rowCount % ARRAY_SIZE) == 0) { From 8f78b21409a2adc3638ca00da462efc8af7248e8 Mon Sep 17 00:00:00 2001 From: shizy Date: Sun, 12 Jan 2025 23:15:41 +0800 Subject: [PATCH 50/54] merge sort using min heap --- .../MergeSortAlignedTVListIterator.java | 80 ++++++++++++------ .../MergeSortTvListIterator.java | 83 ++++++++++++------- 2 files changed, 105 insertions(+), 58 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MergeSortAlignedTVListIterator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MergeSortAlignedTVListIterator.java index bf0dfe16c2c4d..9e4dc609fc5e6 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MergeSortAlignedTVListIterator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MergeSortAlignedTVListIterator.java @@ -24,10 +24,14 @@ import org.apache.tsfile.read.TimeValuePair; import org.apache.tsfile.read.reader.IPointReader; import org.apache.tsfile.utils.BitMap; +import org.apache.tsfile.utils.Pair; import org.apache.tsfile.utils.TsPrimitiveType; import java.io.IOException; import java.util.List; +import java.util.PriorityQueue; +import java.util.stream.Collectors; +import java.util.stream.IntStream; public class MergeSortAlignedTVListIterator implements IPointReader { private final AlignedTVList.AlignedTVListIterator[] alignedTvListIterators; @@ -35,6 +39,12 @@ public class MergeSortAlignedTVListIterator implements IPointReader { private boolean probeNext = false; private boolean hasNext = false; + private final List probeIterators; + + // Min-Heap: minimal timestamp; if same timestamp, maximum TVList index + private final PriorityQueue> minHeap = + new PriorityQueue<>( + (a, b) -> a.left.equals(b.left) ? b.right.compareTo(a.right) : a.left.compareTo(b.left)); private final int[] alignedTvListOffsets; @@ -67,6 +77,8 @@ public MergeSortAlignedTVListIterator( columnAccessInfo[i] = new int[2]; } this.bitMap = new BitMap(columnNum); + this.probeIterators = + IntStream.range(0, alignedTvListIterators.length).boxed().collect(Collectors.toList()); } public MergeSortAlignedTVListIterator( @@ -83,36 +95,47 @@ public MergeSortAlignedTVListIterator( columnAccessInfo[i] = new int[2]; } this.bitMap = new BitMap(columnNum); + this.probeIterators = + IntStream.range(0, alignedTvListIterators.length).boxed().collect(Collectors.toList()); } private void prepareNextRow() { - time = Long.MAX_VALUE; - for (int i = 0; i < alignedTvListIterators.length; i++) { - AlignedTVList.AlignedTVListIterator iterator = alignedTvListIterators[i]; - if (iterator.hasNext() && iterator.currentTime() <= time) { - if (i == 0 || iterator.currentTime() < time) { - for (int columnIndex = 0; columnIndex < columnNum; columnIndex++) { - int rowIndex = iterator.getSelectedIndex(columnIndex); - columnAccessInfo[columnIndex][0] = i; - columnAccessInfo[columnIndex][1] = rowIndex; - if (iterator.isNull(rowIndex, columnIndex)) { - bitMap.mark(columnIndex); - } - } - time = iterator.currentTime(); - } else { - for (int columnIndex = 0; columnIndex < columnNum; columnIndex++) { - int rowIndex = iterator.getSelectedIndex(columnIndex); - // update if the column is not null - if (!iterator.isNull(rowIndex, columnIndex)) { - columnAccessInfo[columnIndex][0] = i; + for (int i : probeIterators) { + TVList.TVListIterator iterator = alignedTvListIterators[i]; + if (iterator.hasNext()) { + minHeap.add(new Pair<>(iterator.currentTime(), i)); + } + } + probeIterators.clear(); + + if (!minHeap.isEmpty()) { + Pair top = minHeap.poll(); + time = top.left; + probeIterators.add(top.right); + for (int columnIndex = 0; columnIndex < columnNum; columnIndex++) { + int rowIndex = alignedTvListIterators[top.right].getSelectedIndex(columnIndex); + columnAccessInfo[columnIndex][0] = top.right; + columnAccessInfo[columnIndex][1] = rowIndex; + if (alignedTvListIterators[top.right].isNull(rowIndex, columnIndex)) { + bitMap.mark(columnIndex); + } + } + while (!minHeap.isEmpty() && minHeap.peek().left == time) { + Pair element = minHeap.poll(); + probeIterators.add(element.right); + for (int columnIndex = 0; columnIndex < columnNum; columnIndex++) { + // if the column is currently not null, it needs not update + if (bitMap.isMarked(columnIndex)) { + int rowIndex = alignedTvListIterators[element.right].getSelectedIndex(columnIndex); + if (!alignedTvListIterators[element.right].isNull(rowIndex, columnIndex)) { + columnAccessInfo[columnIndex][0] = element.right; columnAccessInfo[columnIndex][1] = rowIndex; bitMap.unmark(columnIndex); } } } - hasNext = true; } + hasNext = true; } probeNext = true; } @@ -154,12 +177,10 @@ private TimeValuePair buildTimeValuePair() { } public void step() { - for (int i = 0; i < alignedTvListIterators.length; i++) { - AlignedTVList.AlignedTVListIterator iterator = alignedTvListIterators[i]; - if (iterator.hasCurrent() && iterator.currentTime() == time) { - alignedTvListIterators[i].step(); - alignedTvListOffsets[i] = alignedTvListIterators[i].getIndex(); - } + for (int index : probeIterators) { + TVList.TVListIterator iterator = alignedTvListIterators[index]; + iterator.step(); + alignedTvListOffsets[index] = iterator.getIndex(); } probeNext = false; hasNext = false; @@ -184,6 +205,11 @@ public void setAlignedTVListOffsets(int[] alignedTvListOffsets) { alignedTvListIterators[i].setIndex(alignedTvListOffsets[i]); this.alignedTvListOffsets[i] = alignedTvListOffsets[i]; } + minHeap.clear(); + probeIterators.clear(); + for (int i = 0; i < alignedTvListIterators.length; i++) { + probeIterators.add(i); + } probeNext = false; hasNext = false; bitMap.reset(); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MergeSortTvListIterator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MergeSortTvListIterator.java index 37ae45121593c..342752a708842 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MergeSortTvListIterator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MergeSortTvListIterator.java @@ -22,17 +22,25 @@ import org.apache.tsfile.file.metadata.enums.TSEncoding; import org.apache.tsfile.read.TimeValuePair; import org.apache.tsfile.read.reader.IPointReader; +import org.apache.tsfile.utils.Pair; import java.io.IOException; import java.util.List; +import java.util.PriorityQueue; +import java.util.stream.Collectors; +import java.util.stream.IntStream; public class MergeSortTvListIterator implements IPointReader { private final TVList.TVListIterator[] tvListIterators; + private final int[] tvListOffsets; - private int selectedTVListIndex = -1; + private boolean probeNext = false; private TimeValuePair currentTvPair; - private final int[] tvListOffsets; + private final List probeIterators; + private final PriorityQueue> minHeap = + new PriorityQueue<>( + (a, b) -> a.left.equals(b.left) ? b.right.compareTo(a.right) : a.left.compareTo(b.left)); public MergeSortTvListIterator(List tvLists) { tvListIterators = new TVList.TVListIterator[tvLists.size()]; @@ -40,6 +48,8 @@ public MergeSortTvListIterator(List tvLists) { tvListIterators[i] = tvLists.get(i).iterator(null, null); } this.tvListOffsets = new int[tvLists.size()]; + this.probeIterators = + IntStream.range(0, tvListIterators.length).boxed().collect(Collectors.toList()); } public MergeSortTvListIterator( @@ -49,6 +59,8 @@ public MergeSortTvListIterator( tvListIterators[i] = tvLists.get(i).iterator(floatPrecision, encoding); } this.tvListOffsets = new int[tvLists.size()]; + this.probeIterators = + IntStream.range(0, tvListIterators.length).boxed().collect(Collectors.toList()); } public MergeSortTvListIterator(TVList.TVListIterator[] tvListIterators) { @@ -57,28 +69,38 @@ public MergeSortTvListIterator(TVList.TVListIterator[] tvListIterators) { this.tvListIterators[i] = tvListIterators[i].clone(); } this.tvListOffsets = new int[tvListIterators.length]; + this.probeIterators = + IntStream.range(0, tvListIterators.length).boxed().collect(Collectors.toList()); } - private void prepareNextRow() { - long time = Long.MAX_VALUE; - selectedTVListIndex = -1; - for (int i = 0; i < tvListIterators.length; i++) { + private void prepareNext() { + currentTvPair = null; + for (int i : probeIterators) { TVList.TVListIterator iterator = tvListIterators[i]; - boolean hasNext = iterator.hasNext(); - // update minimum time and remember selected TVList - if (hasNext && iterator.currentTime() <= time) { - time = iterator.currentTime(); - selectedTVListIndex = i; + if (iterator.hasNext()) { + minHeap.add(new Pair<>(iterator.currentTime(), i)); + } + } + probeIterators.clear(); + + if (!minHeap.isEmpty()) { + Pair top = minHeap.poll(); + probeIterators.add(top.right); + currentTvPair = tvListIterators[top.right].current(); + while (!minHeap.isEmpty() && minHeap.peek().left.longValue() == top.left.longValue()) { + Pair element = minHeap.poll(); + probeIterators.add(element.right); } } + probeNext = true; } @Override public boolean hasNextTimeValuePair() { - if (selectedTVListIndex == -1) { - prepareNextRow(); + if (!probeNext) { + prepareNext(); } - return selectedTVListIndex >= 0 && selectedTVListIndex < tvListIterators.length; + return currentTvPair != null; } @Override @@ -86,22 +108,7 @@ public TimeValuePair nextTimeValuePair() { if (!hasNextTimeValuePair()) { return null; } - currentTvPair = tvListIterators[selectedTVListIndex].next(); - tvListOffsets[selectedTVListIndex] = tvListIterators[selectedTVListIndex].getIndex(); - - // call next to skip identical timestamp in other iterators - for (int i = 0; i < tvListIterators.length; i++) { - if (selectedTVListIndex == i) { - continue; - } - TVList.TVListIterator iterator = tvListIterators[i]; - if (iterator.hasCurrent() && iterator.currentTime() == currentTvPair.getTimestamp()) { - tvListIterators[i].step(); - tvListOffsets[i] = tvListIterators[i].getIndex(); - } - } - - selectedTVListIndex = -1; + step(); return currentTvPair; } @@ -113,6 +120,15 @@ public TimeValuePair currentTimeValuePair() { return currentTvPair; } + public void step() { + for (int index : probeIterators) { + TVList.TVListIterator iterator = tvListIterators[index]; + iterator.step(); + tvListOffsets[index] = iterator.getIndex(); + } + probeNext = false; + } + @Override public long getUsedMemorySize() { // not used @@ -131,7 +147,12 @@ public void setTVListOffsets(int[] tvListOffsets) { tvListIterators[i].setIndex(tvListOffsets[i]); this.tvListOffsets[i] = tvListOffsets[i]; } - selectedTVListIndex = -1; + minHeap.clear(); + probeIterators.clear(); + for (int i = 0; i < tvListIterators.length; i++) { + probeIterators.add(i); + } + probeNext = false; } @Override From 11f376c43a1bce6f9e0d01272a55e01c54499b96 Mon Sep 17 00:00:00 2001 From: shizy Date: Mon, 13 Jan 2025 16:22:25 +0800 Subject: [PATCH 51/54] fix: WritableMemChunk deserialize --- .../memtable/AlignedWritableMemChunk.java | 18 +- .../AlignedWritableMemChunkGroup.java | 7 + .../dataregion/memtable/WritableMemChunk.java | 8 + .../db/utils/datastructure/AlignedTVList.java | 4 + .../memtable/MemChunkDeserializeTest.java | 332 ++++++++++++++++++ 5 files changed, 364 insertions(+), 5 deletions(-) create mode 100644 iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/memtable/MemChunkDeserializeTest.java diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedWritableMemChunk.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedWritableMemChunk.java index 1ba99166137e2..be693108e0be8 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedWritableMemChunk.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedWritableMemChunk.java @@ -67,7 +67,7 @@ public class AlignedWritableMemChunk implements IWritableMemChunk { private final List dataTypes; private final List schemaList; private AlignedTVList list; - private final List sortedList; + private List sortedList; private final boolean ignoreAllNullRows; private static final IoTDBConfig CONFIG = IoTDBDescriptor.getInstance().getConfig(); @@ -934,7 +934,7 @@ public int serializedSize() { for (IMeasurementSchema schema : schemaList) { size += schema.serializedSize(); } - + size += Integer.BYTES; for (AlignedTVList alignedTvList : sortedList) { size += alignedTvList.serializedSize(); } @@ -950,6 +950,7 @@ public void serializeToWAL(IWALByteBufferView buffer) { schema.serializeTo(ByteBuffer.wrap(bytes)); buffer.put(bytes); } + buffer.putInt(sortedList.size()); for (AlignedTVList alignedTvList : sortedList) { alignedTvList.serializeToWAL(buffer); } @@ -964,9 +965,16 @@ public static AlignedWritableMemChunk deserialize(DataInputStream stream, boolea IMeasurementSchema schema = MeasurementSchema.deserializeFrom(stream); schemaList.add(schema); } - - AlignedTVList list = (AlignedTVList) TVList.deserialize(stream); - return new AlignedWritableMemChunk(schemaList, list, isTableModel); + int sortedListSize = stream.readInt(); + List sortedList = new ArrayList<>(); + for (int i = 0; i < sortedListSize; i++) { + AlignedTVList tvList = AlignedTVList.deserialize(stream); + sortedList.add(tvList); + } + AlignedTVList list = AlignedTVList.deserialize(stream); + AlignedWritableMemChunk chunk = new AlignedWritableMemChunk(schemaList, list, isTableModel); + chunk.sortedList = sortedList; + return chunk; } public List getSchemaList() { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedWritableMemChunkGroup.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedWritableMemChunkGroup.java index da1ec156e32ee..8f22d33d196dd 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedWritableMemChunkGroup.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedWritableMemChunkGroup.java @@ -21,6 +21,7 @@ import org.apache.iotdb.common.rpc.thrift.TSStatus; import org.apache.iotdb.commons.path.AlignedPath; +import org.apache.iotdb.commons.utils.TestOnly; import org.apache.iotdb.db.storageengine.dataregion.modification.ModEntry; import org.apache.iotdb.db.storageengine.dataregion.wal.buffer.IWALByteBufferView; @@ -44,6 +45,12 @@ public AlignedWritableMemChunkGroup(List schemaList, boolean memChunk = new AlignedWritableMemChunk(schemaList, isTableModel); } + @TestOnly + public AlignedWritableMemChunkGroup( + AlignedWritableMemChunk memChunk, List schemaList, boolean isTableModel) { + this.memChunk = memChunk; + } + private AlignedWritableMemChunkGroup() { // Empty constructor } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/WritableMemChunk.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/WritableMemChunk.java index c615bd3489a03..e946e871ff26c 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/WritableMemChunk.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/WritableMemChunk.java @@ -728,6 +728,7 @@ public void release() { @Override public int serializedSize() { int serializedSize = schema.serializedSize() + list.serializedSize(); + serializedSize += Integer.BYTES; for (TVList tvList : sortedList) { serializedSize += tvList.serializedSize(); } @@ -739,6 +740,7 @@ public void serializeToWAL(IWALByteBufferView buffer) { byte[] bytes = new byte[schema.serializedSize()]; schema.serializeTo(ByteBuffer.wrap(bytes)); buffer.put(bytes); + buffer.putInt(sortedList.size()); for (TVList tvList : sortedList) { tvList.serializeToWAL(buffer); } @@ -748,6 +750,12 @@ public void serializeToWAL(IWALByteBufferView buffer) { public static WritableMemChunk deserialize(DataInputStream stream) throws IOException { WritableMemChunk memChunk = new WritableMemChunk(); memChunk.schema = MeasurementSchema.deserializeFrom(stream); + int sortedListSize = stream.readInt(); + memChunk.sortedList = new ArrayList<>(); + for (int i = 0; i < sortedListSize; i++) { + TVList tvList = TVList.deserialize(stream); + memChunk.sortedList.add(tvList); + } memChunk.list = TVList.deserialize(stream); return memChunk; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/AlignedTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/AlignedTVList.java index 708b741e9b7d7..d600e83ddbf3d 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/AlignedTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/AlignedTVList.java @@ -1297,6 +1297,10 @@ public void serializeToWAL(IWALByteBufferView buffer) { } public static AlignedTVList deserialize(DataInputStream stream) throws IOException { + TSDataType dataType = ReadWriteIOUtils.readDataType(stream); + if (dataType != TSDataType.VECTOR) { + throw new UnSupportedDataTypeException("Unsupported data type:" + dataType); + } int dataTypeNum = stream.readInt(); List dataTypes = new ArrayList<>(dataTypeNum); for (int columnIndex = 0; columnIndex < dataTypeNum; ++columnIndex) { diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/memtable/MemChunkDeserializeTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/memtable/MemChunkDeserializeTest.java new file mode 100644 index 0000000000000..d0edfdab33002 --- /dev/null +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/memtable/MemChunkDeserializeTest.java @@ -0,0 +1,332 @@ +/* + * 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.storageengine.dataregion.memtable; + +import org.apache.iotdb.commons.exception.MetadataException; +import org.apache.iotdb.commons.path.AlignedFullPath; +import org.apache.iotdb.commons.path.NonAlignedFullPath; +import org.apache.iotdb.db.conf.IoTDBConfig; +import org.apache.iotdb.db.conf.IoTDBDescriptor; +import org.apache.iotdb.db.exception.query.QueryProcessException; +import org.apache.iotdb.db.queryengine.execution.fragment.QueryContext; +import org.apache.iotdb.db.storageengine.dataregion.wal.utils.WALByteBufferForTest; +import org.apache.iotdb.db.utils.EnvironmentUtils; + +import org.apache.tsfile.common.conf.TSFileConfig; +import org.apache.tsfile.common.conf.TSFileDescriptor; +import org.apache.tsfile.enums.TSDataType; +import org.apache.tsfile.file.metadata.IDeviceID; +import org.apache.tsfile.file.metadata.enums.CompressionType; +import org.apache.tsfile.file.metadata.enums.TSEncoding; +import org.apache.tsfile.read.TimeValuePair; +import org.apache.tsfile.read.reader.IPointReader; +import org.apache.tsfile.utils.Binary; +import org.apache.tsfile.utils.TsPrimitiveType; +import org.apache.tsfile.write.schema.IMeasurementSchema; +import org.apache.tsfile.write.schema.MeasurementSchema; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.io.ByteArrayInputStream; +import java.io.DataInputStream; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +public class MemChunkDeserializeTest { + private static final IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig(); + private String storageGroup = "sg1"; + private String dataRegionId = "1"; + + private IDeviceID deviceID = IDeviceID.Factory.DEFAULT_FACTORY.create("d1"); + double delta; + + @Before + public void setUp() throws Exception { + delta = Math.pow(0.1, TSFileDescriptor.getInstance().getConfig().getFloatPrecision()); + config.setTVListSortThreshold(100); + EnvironmentUtils.envSetUp(); + } + + @After + public void tearDown() throws Exception { + EnvironmentUtils.cleanEnv(); + } + + @Test + public void testNonAlignedBoolean() throws IOException, QueryProcessException, MetadataException { + TSDataType dataType = TSDataType.BOOLEAN; + WritableMemChunk series = + new WritableMemChunk(new MeasurementSchema("s1", dataType, TSEncoding.PLAIN)); + int count = 1000; + for (int i = 0; i < count; i++) { + series.writeNonAlignedPoint(i, i % 2 == 0); + } + series.delete(100, 200); + + WritableMemChunk memChunk = createWritableMemChunkFromBytes(series); + ReadOnlyMemChunk readableChunk = getReadOnlyChunk(memChunk, dataType); + IPointReader it = readableChunk.getPointReader(); + for (int i = 0; i < count; i++) { + if (i >= 100 && i <= 200) { + continue; + } + it.hasNextTimeValuePair(); + TimeValuePair timeValuePair = it.nextTimeValuePair(); + Assert.assertEquals(i, timeValuePair.getTimestamp()); + Assert.assertEquals(i % 2 == 0, timeValuePair.getValue().getBoolean()); + } + } + + @Test + public void testNonAlignedInt32() throws IOException, QueryProcessException, MetadataException { + TSDataType dataType = TSDataType.INT32; + WritableMemChunk series = + new WritableMemChunk(new MeasurementSchema("s1", dataType, TSEncoding.PLAIN)); + int count = 1000; + for (int i = 0; i < count; i++) { + series.writeNonAlignedPoint(i, i); + } + series.delete(100, 200); + + WritableMemChunk memChunk = createWritableMemChunkFromBytes(series); + ReadOnlyMemChunk readableChunk = getReadOnlyChunk(memChunk, dataType); + IPointReader it = readableChunk.getPointReader(); + for (int i = 0; i < count; i++) { + if (i >= 100 && i <= 200) { + continue; + } + it.hasNextTimeValuePair(); + TimeValuePair timeValuePair = it.nextTimeValuePair(); + Assert.assertEquals(i, timeValuePair.getTimestamp()); + Assert.assertEquals(i, timeValuePair.getValue().getInt()); + } + } + + @Test + public void testNonAlignedInt64() throws IOException, QueryProcessException, MetadataException { + TSDataType dataType = TSDataType.INT64; + WritableMemChunk series = + new WritableMemChunk(new MeasurementSchema("s1", dataType, TSEncoding.PLAIN)); + int count = 1000; + for (int i = 0; i < count; i++) { + series.writeNonAlignedPoint(i, (long) i); + } + series.delete(100, 200); + + WritableMemChunk memChunk = createWritableMemChunkFromBytes(series); + ReadOnlyMemChunk readableChunk = getReadOnlyChunk(memChunk, dataType); + IPointReader it = readableChunk.getPointReader(); + for (int i = 0; i < count; i++) { + if (i >= 100 && i <= 200) { + continue; + } + it.hasNextTimeValuePair(); + TimeValuePair timeValuePair = it.nextTimeValuePair(); + Assert.assertEquals(i, timeValuePair.getTimestamp()); + Assert.assertEquals(i, timeValuePair.getValue().getLong()); + } + } + + @Test + public void testNonAlignedFloat() throws IOException, QueryProcessException, MetadataException { + TSDataType dataType = TSDataType.FLOAT; + WritableMemChunk series = + new WritableMemChunk(new MeasurementSchema("s1", dataType, TSEncoding.PLAIN)); + int count = 1000; + for (int i = 0; i < count; i++) { + series.writeNonAlignedPoint(i, (float) i); + } + series.delete(100, 200); + + WritableMemChunk memChunk = createWritableMemChunkFromBytes(series); + ReadOnlyMemChunk readableChunk = getReadOnlyChunk(memChunk, dataType); + IPointReader it = readableChunk.getPointReader(); + for (int i = 0; i < count; i++) { + if (i >= 100 && i <= 200) { + continue; + } + it.hasNextTimeValuePair(); + TimeValuePair timeValuePair = it.nextTimeValuePair(); + Assert.assertEquals(i, timeValuePair.getTimestamp()); + Assert.assertEquals(timeValuePair.getValue().getFloat(), i, delta); + } + } + + @Test + public void testNonAlignedDouble() throws IOException, QueryProcessException, MetadataException { + TSDataType dataType = TSDataType.DOUBLE; + WritableMemChunk series = + new WritableMemChunk(new MeasurementSchema("s1", dataType, TSEncoding.PLAIN)); + int count = 1000; + for (int i = 0; i < count; i++) { + series.writeNonAlignedPoint(i, (double) i); + } + series.delete(100, 200); + + WritableMemChunk memChunk = createWritableMemChunkFromBytes(series); + ReadOnlyMemChunk readableChunk = getReadOnlyChunk(memChunk, dataType); + IPointReader it = readableChunk.getPointReader(); + for (int i = 0; i < count; i++) { + if (i >= 100 && i <= 200) { + continue; + } + it.hasNextTimeValuePair(); + TimeValuePair timeValuePair = it.nextTimeValuePair(); + Assert.assertEquals(i, timeValuePair.getTimestamp()); + Assert.assertEquals(timeValuePair.getValue().getDouble(), i, delta); + } + } + + @Test + public void testNonAlignedBinary() throws IOException, QueryProcessException, MetadataException { + TSDataType dataType = TSDataType.TEXT; + WritableMemChunk series = + new WritableMemChunk(new MeasurementSchema("s1", dataType, TSEncoding.PLAIN)); + int count = 1000; + for (int i = 0; i < count; i++) { + series.writeNonAlignedPoint(i, new Binary("text" + i, StandardCharsets.UTF_8)); + } + series.delete(100, 200); + + WritableMemChunk memChunk = createWritableMemChunkFromBytes(series); + ReadOnlyMemChunk readableChunk = getReadOnlyChunk(memChunk, dataType); + IPointReader it = readableChunk.getPointReader(); + for (int i = 0; i < count; i++) { + if (i >= 100 && i <= 200) { + continue; + } + it.hasNextTimeValuePair(); + TimeValuePair timeValuePair = it.nextTimeValuePair(); + Assert.assertEquals(i, timeValuePair.getTimestamp()); + Assert.assertEquals(timeValuePair.getValue().getBinary().toString(), "text" + i); + } + } + + @Test + public void testAlignedSeries() throws IOException, QueryProcessException, MetadataException { + // IMemTable memTable = new PrimitiveMemTable(storageGroup, dataRegionId); + List measurementList = Arrays.asList("s1", "s2", "s3", "s4", "s5", "s6"); + List schemaList = + Arrays.asList( + new MeasurementSchema("s1", TSDataType.BOOLEAN), + new MeasurementSchema("s2", TSDataType.INT32), + new MeasurementSchema("s3", TSDataType.INT64), + new MeasurementSchema("s4", TSDataType.FLOAT), + new MeasurementSchema("s5", TSDataType.DOUBLE), + new MeasurementSchema("s6", TSDataType.TEXT)); + AlignedWritableMemChunk series = new AlignedWritableMemChunk(schemaList, false); + + int count = 1000; + for (int i = 0; i < count; i++) { + Object[] data = + new Object[] { + i % 2 == 0, + i, + (long) i, + (float) i, + (double) i, + new Binary("text" + i, TSFileConfig.STRING_CHARSET) + }; + series.writeAlignedPoints(i, data, schemaList); + } + series.delete(100, 200); + + int serializedSize = series.serializedSize(); + WALByteBufferForTest walBuffer = new WALByteBufferForTest(ByteBuffer.allocate(serializedSize)); + series.serializeToWAL(walBuffer); + DataInputStream inputStream = + new DataInputStream(new ByteArrayInputStream(walBuffer.getBuffer().array())); + AlignedWritableMemChunk memChunk = AlignedWritableMemChunk.deserialize(inputStream, false); + + AlignedReadOnlyMemChunk readableChunk = + (AlignedReadOnlyMemChunk) getAlignedReadOnlyChunk(memChunk, schemaList, measurementList); + IPointReader it = readableChunk.getPointReader(); + for (int i = 0; i < count; i++) { + if (i >= 100 && i <= 200) { + continue; + } + it.hasNextTimeValuePair(); + TimeValuePair timeValuePair = it.nextTimeValuePair(); + Assert.assertEquals(i, timeValuePair.getTimestamp()); + TsPrimitiveType[] values = timeValuePair.getValue().getVector(); + Assert.assertEquals(values[0].getBoolean(), i % 2 == 0); + Assert.assertEquals(values[1].getInt(), i); + Assert.assertEquals(values[2].getLong(), i); + Assert.assertEquals(values[3].getFloat(), i, delta); + Assert.assertEquals(values[4].getDouble(), i, delta); + Assert.assertEquals(values[5].getBinary().toString(), "text" + i); + } + } + + private WritableMemChunk createWritableMemChunkFromBytes(WritableMemChunk series) + throws IOException { + int serializedSize = series.serializedSize(); + WALByteBufferForTest walBuffer = new WALByteBufferForTest(ByteBuffer.allocate(serializedSize)); + series.serializeToWAL(walBuffer); + DataInputStream inputStream = + new DataInputStream(new ByteArrayInputStream(walBuffer.getBuffer().array())); + return WritableMemChunk.deserialize(inputStream); + } + + private ReadOnlyMemChunk getReadOnlyChunk(WritableMemChunk memChunk, TSDataType dataType) + throws QueryProcessException, IOException, MetadataException { + WritableMemChunkGroup memChunkGroup = new WritableMemChunkGroup(); + memChunkGroup.getMemChunkMap().put("s1", memChunk); + Map memTableMap = new HashMap<>(); + memTableMap.put(deviceID, memChunkGroup); + IMemTable memTable = new PrimitiveMemTable(storageGroup, dataRegionId, memTableMap); + + QueryContext context = new QueryContext(); + NonAlignedFullPath nonAlignedFullPath = + new NonAlignedFullPath( + deviceID, + new MeasurementSchema( + "s1", + dataType, + TSEncoding.RLE, + CompressionType.UNCOMPRESSED, + Collections.emptyMap())); + return memTable.query(context, nonAlignedFullPath, Long.MIN_VALUE, null, null); + } + + private ReadOnlyMemChunk getAlignedReadOnlyChunk( + AlignedWritableMemChunk memChunk, + List schemaList, + List measurementList) + throws QueryProcessException, IOException, MetadataException { + AlignedWritableMemChunkGroup memChunkGroup = + new AlignedWritableMemChunkGroup(memChunk, schemaList, false); + Map memTableMap = new HashMap<>(); + memTableMap.put(deviceID, memChunkGroup); + IMemTable memTable = new PrimitiveMemTable(storageGroup, dataRegionId, memTableMap); + + QueryContext context = new QueryContext(); + AlignedFullPath alignedFullPath = new AlignedFullPath(deviceID, measurementList, schemaList); + return memTable.query(context, alignedFullPath, Long.MIN_VALUE, null, null); + } +} From 736e59694c429f0b4f018459f165fa5b9687a75a Mon Sep 17 00:00:00 2001 From: shizy Date: Tue, 14 Jan 2025 09:40:25 +0800 Subject: [PATCH 52/54] feat: add index mem cost for TVList --- .../dataregion/memtable/WritableMemChunk.java | 2 +- .../db/utils/datastructure/AlignedTVList.java | 8 +++-- .../iotdb/db/utils/datastructure/TVList.java | 5 ++- .../memtable/PrimitiveMemTableTest.java | 16 +++++++-- .../memtable/TsFileProcessorTest.java | 36 +++++++++---------- 5 files changed, 42 insertions(+), 25 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/WritableMemChunk.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/WritableMemChunk.java index e946e871ff26c..ed69cdaa68169 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/WritableMemChunk.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/WritableMemChunk.java @@ -480,7 +480,7 @@ public String toString() { continue; } size++; - long currentTime = list.getTime(i); + long currentTime = tvList.getTime(i); if (currentTime < minTime) { firstTvPair = tvList.getTimeValuePair(i); minTime = currentTime; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/AlignedTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/AlignedTVList.java index d600e83ddbf3d..8d246dee77b4f 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/AlignedTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/AlignedTVList.java @@ -908,12 +908,13 @@ public static long alignedTvListArrayMemCost( int measurementColumnNum = 0; long size = 0; - // value array mem size + // value & bitmap array mem size for (int i = 0; i < types.length; i++) { TSDataType type = types[i]; if (type != null && (columnCategories == null || columnCategories[i] == TsTableColumnCategory.FIELD)) { size += (long) ARRAY_SIZE * (long) type.getDataTypeSize(); + size += (long) ARRAY_SIZE / 8 + 1; measurementColumnNum++; } } @@ -940,10 +941,11 @@ public static long alignedTvListArrayMemCost( */ public static long alignedTvListArrayMemCost(List types) { long size = 0; - // value array mem size + // value & bitmap array mem size for (TSDataType type : types) { if (type != null) { size += (long) PrimitiveArrayManager.ARRAY_SIZE * (long) type.getDataTypeSize(); + size += (long) PrimitiveArrayManager.ARRAY_SIZE / 8 + 1; } } // size is 0 when all types are null @@ -971,6 +973,8 @@ public static long valueListArrayMemCost(TSDataType type) { long size = 0; // value array mem size size += (long) PrimitiveArrayManager.ARRAY_SIZE * (long) type.getDataTypeSize(); + // bitmap array mem size + size += (long) PrimitiveArrayManager.ARRAY_SIZE / 8 + 1; // array headers mem size size += NUM_BYTES_ARRAY_HEADER; // Object references size in ArrayList diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java index 53174f9bda44c..a04580f429a1a 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java @@ -119,13 +119,16 @@ public static TVList newList(TSDataType dataType) { return null; } - // TODO: memory cost for indices and bitmap public static long tvListArrayMemCost(TSDataType type) { long size = 0; // time array mem size size += PrimitiveArrayManager.ARRAY_SIZE * 8L; // value array mem size size += PrimitiveArrayManager.ARRAY_SIZE * (long) type.getDataTypeSize(); + // index array mem size + size += PrimitiveArrayManager.ARRAY_SIZE * 4L; + // bimap array mem size + size += PrimitiveArrayManager.ARRAY_SIZE / 8 + 1L; // two array headers mem size size += NUM_BYTES_ARRAY_HEADER * 2L; // Object references size in ArrayList diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/memtable/PrimitiveMemTableTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/memtable/PrimitiveMemTableTest.java index 30b2f2605f8e4..3d2387674899b 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/memtable/PrimitiveMemTableTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/memtable/PrimitiveMemTableTest.java @@ -33,6 +33,7 @@ import org.apache.iotdb.db.storageengine.dataregion.modification.TreeDeletionEntry; import org.apache.iotdb.db.storageengine.dataregion.wal.utils.WALByteBufferForTest; import org.apache.iotdb.db.utils.MathUtils; +import org.apache.iotdb.db.utils.datastructure.TVList; import org.apache.tsfile.common.conf.TSFileConfig; import org.apache.tsfile.common.conf.TSFileDescriptor; @@ -57,7 +58,9 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; +import java.util.HashMap; import java.util.List; +import java.util.Map; import java.util.Random; public class PrimitiveMemTableTest { @@ -96,7 +99,7 @@ public void setUp() { } @Test - public void memSeriesSortIteratorTest() throws IOException { + public void memSeriesSortIteratorTest() throws IOException, QueryProcessException { TSDataType dataType = TSDataType.INT32; WritableMemChunk series = new WritableMemChunk(new MeasurementSchema("s1", dataType, TSEncoding.PLAIN)); @@ -104,8 +107,15 @@ public void memSeriesSortIteratorTest() throws IOException { for (int i = 0; i < count; i++) { series.writeNonAlignedPoint(i, i); } - IPointReader it = - series.getSortedTvListForQuery().buildTsBlock().getTsBlockSingleColumnIterator(); + Map tvListQueryMap = new HashMap<>(); + for (TVList tvList : series.getSortedList()) { + tvListQueryMap.put(tvList, tvList.rowCount()); + } + tvListQueryMap.put(series.getWorkingTVList(), series.getWorkingTVList().rowCount()); + ReadOnlyMemChunk readableChunk = + new ReadOnlyMemChunk( + new QueryContext(), "s1", dataType, TSEncoding.PLAIN, tvListQueryMap, null, null); + IPointReader it = readableChunk.getPointReader(); int i = 0; while (it.hasNextTimeValuePair()) { Assert.assertEquals(i, it.nextTimeValuePair().getTimestamp()); diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/memtable/TsFileProcessorTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/memtable/TsFileProcessorTest.java index fc8736457d0df..a79e935ff32a5 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/memtable/TsFileProcessorTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/memtable/TsFileProcessorTest.java @@ -685,21 +685,21 @@ public void alignedTvListRamCostTest() true, new long[5]); IMemTable memTable = processor.getWorkMemTable(); - Assert.assertEquals(1596808, memTable.getTVListsRamCost()); + Assert.assertEquals(1623808, memTable.getTVListsRamCost()); processor.insertTablet( genInsertTableNode(100, true), Collections.singletonList(new int[] {0, 10}), new TSStatus[10], true, new long[5]); - Assert.assertEquals(1596808, memTable.getTVListsRamCost()); + Assert.assertEquals(1623808, memTable.getTVListsRamCost()); processor.insertTablet( genInsertTableNode(200, true), Collections.singletonList(new int[] {0, 10}), new TSStatus[10], true, new long[5]); - Assert.assertEquals(1596808, memTable.getTVListsRamCost()); + Assert.assertEquals(1623808, memTable.getTVListsRamCost()); Assert.assertEquals(90000, memTable.getTotalPointsNum()); Assert.assertEquals(720360, memTable.memSize()); // Test records @@ -708,7 +708,7 @@ public void alignedTvListRamCostTest() record.addTuple(DataPoint.getDataPoint(dataType, measurementId, String.valueOf(i))); processor.insert(buildInsertRowNodeByTSRecord(record), new long[5]); } - Assert.assertEquals(1598424, memTable.getTVListsRamCost()); + Assert.assertEquals(1625954, memTable.getTVListsRamCost()); Assert.assertEquals(90100, memTable.getTotalPointsNum()); Assert.assertEquals(721560, memTable.memSize()); } @@ -736,56 +736,56 @@ public void alignedTvListRamCostTest2() true, new long[5]); IMemTable memTable = processor.getWorkMemTable(); - Assert.assertEquals(1596808, memTable.getTVListsRamCost()); + Assert.assertEquals(1623808, memTable.getTVListsRamCost()); processor.insertTablet( genInsertTableNodeFors3000ToS6000(0, true), Collections.singletonList(new int[] {0, 10}), new TSStatus[10], true, new long[5]); - Assert.assertEquals(3192808, memTable.getTVListsRamCost()); + Assert.assertEquals(3246808, memTable.getTVListsRamCost()); processor.insertTablet( genInsertTableNode(100, true), Collections.singletonList(new int[] {0, 10}), new TSStatus[10], true, new long[5]); - Assert.assertEquals(3192808, memTable.getTVListsRamCost()); + Assert.assertEquals(3246808, memTable.getTVListsRamCost()); processor.insertTablet( genInsertTableNodeFors3000ToS6000(100, true), Collections.singletonList(new int[] {0, 10}), new TSStatus[10], true, new long[5]); - Assert.assertEquals(3192808, memTable.getTVListsRamCost()); + Assert.assertEquals(3246808, memTable.getTVListsRamCost()); processor.insertTablet( genInsertTableNode(200, true), Collections.singletonList(new int[] {0, 10}), new TSStatus[10], true, new long[5]); - Assert.assertEquals(3192808, memTable.getTVListsRamCost()); + Assert.assertEquals(3246808, memTable.getTVListsRamCost()); processor.insertTablet( genInsertTableNodeFors3000ToS6000(200, true), Collections.singletonList(new int[] {0, 10}), new TSStatus[10], true, new long[5]); - Assert.assertEquals(3192808, memTable.getTVListsRamCost()); + Assert.assertEquals(3246808, memTable.getTVListsRamCost()); processor.insertTablet( genInsertTableNode(300, true), Collections.singletonList(new int[] {0, 10}), new TSStatus[10], true, new long[5]); - Assert.assertEquals(6385616, memTable.getTVListsRamCost()); + Assert.assertEquals(6493616, memTable.getTVListsRamCost()); processor.insertTablet( genInsertTableNodeFors3000ToS6000(300, true), Collections.singletonList(new int[] {0, 10}), new TSStatus[10], true, new long[5]); - Assert.assertEquals(6385616, memTable.getTVListsRamCost()); + Assert.assertEquals(6493616, memTable.getTVListsRamCost()); Assert.assertEquals(240000, memTable.getTotalPointsNum()); Assert.assertEquals(1920960, memTable.memSize()); @@ -795,14 +795,14 @@ public void alignedTvListRamCostTest2() record.addTuple(DataPoint.getDataPoint(dataType, measurementId, String.valueOf(i))); processor.insert(buildInsertRowNodeByTSRecord(record), new long[5]); } - Assert.assertEquals(6387232, memTable.getTVListsRamCost()); + Assert.assertEquals(6495762, memTable.getTVListsRamCost()); // Test records for (int i = 1; i <= 100; i++) { TSRecord record = new TSRecord(deviceId, i); record.addTuple(DataPoint.getDataPoint(dataType, "s1", String.valueOf(i))); processor.insert(buildInsertRowNodeByTSRecord(record), new long[5]); } - Assert.assertEquals(6388848, memTable.getTVListsRamCost()); + Assert.assertEquals(6497908, memTable.getTVListsRamCost()); Assert.assertEquals(240200, memTable.getTotalPointsNum()); Assert.assertEquals(1923360, memTable.memSize()); } @@ -830,21 +830,21 @@ public void nonAlignedTvListRamCostTest() true, new long[5]); IMemTable memTable = processor.getWorkMemTable(); - Assert.assertEquals(3192000, memTable.getTVListsRamCost()); + Assert.assertEquals(3987000, memTable.getTVListsRamCost()); processor.insertTablet( genInsertTableNode(100, false), Collections.singletonList(new int[] {0, 10}), new TSStatus[10], true, new long[5]); - Assert.assertEquals(3192000, memTable.getTVListsRamCost()); + Assert.assertEquals(3987000, memTable.getTVListsRamCost()); processor.insertTablet( genInsertTableNode(200, false), Collections.singletonList(new int[] {0, 10}), new TSStatus[10], true, new long[5]); - Assert.assertEquals(3192000, memTable.getTVListsRamCost()); + Assert.assertEquals(3987000, memTable.getTVListsRamCost()); Assert.assertEquals(90000, memTable.getTotalPointsNum()); Assert.assertEquals(1440000, memTable.memSize()); // Test records @@ -853,7 +853,7 @@ public void nonAlignedTvListRamCostTest() record.addTuple(DataPoint.getDataPoint(dataType, measurementId, String.valueOf(i))); processor.insert(buildInsertRowNodeByTSRecord(record), new long[5]); } - Assert.assertEquals(3193616, memTable.getTVListsRamCost()); + Assert.assertEquals(3989146, memTable.getTVListsRamCost()); Assert.assertEquals(90100, memTable.getTotalPointsNum()); Assert.assertEquals(1441200, memTable.memSize()); } From 7fe1f855f43e4924f9772a8c4996a70f08c2e1df Mon Sep 17 00:00:00 2001 From: shizy Date: Tue, 14 Jan 2025 14:16:24 +0800 Subject: [PATCH 53/54] fix: hot-load tvlist_sort_threshold setting --- .../java/org/apache/iotdb/db/conf/IoTDBDescriptor.java | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java index 841d854ec2b84..f01f7ac9b4175 100755 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java @@ -2069,11 +2069,9 @@ public synchronized void loadHotModifiedProps(TrimProperties properties) // tvlist_sort_threshold conf.setTVListSortThreshold( Integer.parseInt( - Optional.ofNullable( - properties.getProperty( - "tvlist_sort_threshold", String.valueOf(conf.getTvListSortThreshold()))) - .map(String::trim) - .orElse(String.valueOf(conf.getTvListSortThreshold())))); + properties.getProperty( + "tvlist_sort_threshold", + ConfigurationFileUtils.getConfigurationDefaultValue("tvlist_sort_threshold")))); } catch (Exception e) { if (e instanceof InterruptedException) { Thread.currentThread().interrupt(); From 090001135be788bdefe35607de2266bc3eb6a2a7 Mon Sep 17 00:00:00 2001 From: shizy Date: Thu, 23 Jan 2025 08:30:50 +0800 Subject: [PATCH 54/54] remove needless line in property template --- .../src/assembly/resources/conf/iotdb-system.properties.template | 1 - 1 file changed, 1 deletion(-) diff --git a/iotdb-core/node-commons/src/assembly/resources/conf/iotdb-system.properties.template b/iotdb-core/node-commons/src/assembly/resources/conf/iotdb-system.properties.template index d9e1c9d94d4df..a647a152123e3 100644 --- a/iotdb-core/node-commons/src/assembly/resources/conf/iotdb-system.properties.template +++ b/iotdb-core/node-commons/src/assembly/resources/conf/iotdb-system.properties.template @@ -1148,7 +1148,6 @@ tvlist_sort_algorithm=TIM # Datatype: int tvlist_sort_threshold=0 -# When the average point number of timeseries in memtable exceeds this, the memtable is flushed to disk. The default threshold is 100000. # The target point nums in one chunk in flushing and compaction. # If the point number of a timeseries in memtable exceeds this, the data will be flushed to multiple chunks. # effectiveMode: restart