This is an automated email from the ASF dual-hosted git repository.
jiangtian pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/iotdb.git
The following commit(s) were added to refs/heads/master by this push:
new f2e20b54408 Modify tablet usage & don't cache TableSchema map in
default option (#14635)
f2e20b54408 is described below
commit f2e20b54408eeb5607e328e2843141f039fda878
Author: shuwenwei <[email protected]>
AuthorDate: Tue Jan 14 09:49:29 2025 +0800
Modify tablet usage & don't cache TableSchema map in default option (#14635)
* modify tablet usage
* update tsfile version
* fix it
* fix compile
* update tsfile version
---------
Co-authored-by: Jiang Tian <[email protected]>
---
.../iotdb/AlignedTimeseriesSessionExample.java | 67 +------------
.../org/apache/iotdb/SessionConcurrentExample.java | 23 -----
.../main/java/org/apache/iotdb/SessionExample.java | 107 +--------------------
.../org/apache/iotdb/trigger/LoggerTrigger.java | 12 +--
.../iotdb/trigger/StatisticsUpdaterTrigger.java | 4 +-
.../org/apache/iotdb/it/utils/TsFileGenerator.java | 85 ++++++----------
.../iotdb/it/utils/TsFileTableGenerator.java | 78 ++++++---------
.../org/apache/iotdb/db/it/IoTDBLoadTsFileIT.java | 2 +-
.../pipe/it/autocreate/IoTDBPipeNullValueIT.java | 28 +++---
.../pipe/it/manual/IoTDBPipeReqAutoSliceIT.java | 8 +-
.../manual/IoTDBPipeTypeConversionISessionIT.java | 10 +-
.../IoTDBPipeTypeConversionISessionIT.java | 22 ++---
.../iotdb/pipe/it/tablemodel/TableModelUtils.java | 21 ++--
.../iotdb/session/it/IoTDBSessionComplexIT.java | 8 +-
.../iotdb/session/it/IoTDBSessionSimpleIT.java | 8 +-
.../session/it/IoTDBSessionSyntaxConventionIT.java | 18 +---
.../org/apache/iotdb/session/it/SessionIT.java | 32 ++----
.../it/local/IoTDBSubscriptionDataTypeIT.java | 8 +-
.../org/apache/iotdb/tool/tsfile/ExportTsFile.java | 3 -
.../java/org/apache/iotdb/session/Session.java | 27 +++---
.../payload/SubscriptionSessionDataSet.java | 12 ++-
.../apache/iotdb/session/util/SessionUtils.java | 25 ++---
.../java/org/apache/iotdb/session/TabletTest.java | 28 +++---
.../iotdb/session/util/SessionUtilsTest.java | 16 +--
.../connector/protocol/opcua/OpcUaNameSpace.java | 36 +++----
.../util/builder/PipeTableModeTsFileBuilder.java | 6 +-
.../util/builder/PipeTreeModelTsFileBuilder.java | 4 +-
.../sorter/PipeTableModelTabletEventSorter.java | 54 ++++++-----
.../sorter/PipeTreeModelTabletEventSorter.java | 24 ++---
.../db/pipe/event/common/row/PipeRowCollector.java | 2 +-
.../common/tablet/PipeRawTabletInsertionEvent.java | 2 +-
.../tablet/parser/TabletInsertionEventParser.java | 8 +-
.../TabletInsertionEventTreePatternParser.java | 13 +--
.../scan/TsFileInsertionEventScanParser.java | 42 +++-----
.../table/TsFileInsertionEventTableParser.java | 2 +-
.../pipe/resource/memory/PipeMemoryWeightUtil.java | 21 ++--
.../load/LoadTsFileToTableModelAnalyzer.java | 7 +-
.../load/LoadTsFileToTreeModelAnalyzer.java | 5 +-
.../utils/CompactionTableSchemaCollector.java | 2 +-
.../pipe/connector/PipeTabletEventSorterTest.java | 61 ++++++------
.../pipe/event/PipeTabletInsertionEventTest.java | 25 +++--
.../storageengine/buffer/BloomFilterCacheTest.java | 5 +-
.../compaction/CompactionValidationTest.java | 7 +-
.../TableModelFastCompactionPerformerTest.java | 13 +--
...TableModelReadChunkCompactionPerformerTest.java | 13 +--
...TableModelReadPointCompactionPerformerTest.java | 13 +--
.../file/UnsealedTsFileRecoverPerformerTest.java | 4 +-
.../iotdb/db/tools/TsFileSelfCheckToolTest.java | 6 +-
.../iotdb/db/tools/TsFileSketchToolTest.java | 10 +-
pom.xml | 2 +-
50 files changed, 371 insertions(+), 668 deletions(-)
diff --git
a/example/session/src/main/java/org/apache/iotdb/AlignedTimeseriesSessionExample.java
b/example/session/src/main/java/org/apache/iotdb/AlignedTimeseriesSessionExample.java
index 208ef6b2b92..a9d3150e23b 100644
---
a/example/session/src/main/java/org/apache/iotdb/AlignedTimeseriesSessionExample.java
+++
b/example/session/src/main/java/org/apache/iotdb/AlignedTimeseriesSessionExample.java
@@ -30,7 +30,6 @@ import org.apache.iotdb.session.template.MeasurementNode;
import org.apache.tsfile.enums.TSDataType;
import org.apache.tsfile.file.metadata.enums.CompressionType;
import org.apache.tsfile.file.metadata.enums.TSEncoding;
-import org.apache.tsfile.utils.BitMap;
import org.apache.tsfile.write.record.Tablet;
import org.apache.tsfile.write.schema.IMeasurementSchema;
import org.apache.tsfile.write.schema.MeasurementSchema;
@@ -341,18 +340,14 @@ public class AlignedTimeseriesSessionExample {
schemaList.add(new MeasurementSchema("s2", TSDataType.INT32));
Tablet tablet = new Tablet(ROOT_SG1_D1_VECTOR2, schemaList);
- long[] timestamps = tablet.timestamps;
- Object[] values = tablet.values;
for (long time = 100; time < 200; time++) {
int row = tablet.getRowSize();
tablet.addTimestamp(row, time);
- long[] sensor1 = (long[]) values[0];
- sensor1[row] = new SecureRandom().nextLong();
+ tablet.addValue(row, 0, new SecureRandom().nextLong());
- int[] sensor2 = (int[]) values[1];
- sensor2[row] = new SecureRandom().nextInt();
+ tablet.addValue(row, 1, new SecureRandom().nextInt());
if (tablet.getRowSize() == tablet.getMaxRowNumber()) {
session.insertAlignedTablet(tablet, true);
@@ -378,32 +373,19 @@ public class AlignedTimeseriesSessionExample {
Tablet tablet = new Tablet(ROOT_SG1_D1_VECTOR3, schemaList);
- long[] timestamps = tablet.timestamps;
- Object[] values = tablet.values;
- // Use the bitMap to mark the null value point
- BitMap[] bitMaps = new BitMap[values.length];
- tablet.bitMaps = bitMaps;
-
- bitMaps[1] = new BitMap(tablet.getMaxRowNumber());
for (long time = 200; time < 300; time++) {
int row = tablet.getRowSize();
tablet.addTimestamp(row, time);
- long[] sensor1 = (long[]) values[0];
- sensor1[row] = new SecureRandom().nextLong();
-
- int[] sensor2 = (int[]) values[1];
- sensor2[row] = new SecureRandom().nextInt();
+ tablet.addValue(row, 0, new SecureRandom().nextLong());
- // mark this point as null value
- if (time % 5 == 0) {
- bitMaps[1].mark(row);
+ if (time % 5 != 0) {
+ tablet.addValue(row, 1, new SecureRandom().nextInt());
}
if (tablet.getRowSize() == tablet.getMaxRowNumber()) {
session.insertAlignedTablet(tablet, true);
tablet.reset();
- bitMaps[1].reset();
}
}
@@ -595,44 +577,5 @@ public class AlignedTimeseriesSessionExample {
tablet2.reset();
tablet3.reset();
}
-
- // Method 2 to add tablet data
- long[] timestamps1 = tablet1.timestamps;
- Object[] values1 = tablet1.values;
- long[] timestamps2 = tablet2.timestamps;
- Object[] values2 = tablet2.values;
- long[] timestamps3 = tablet3.timestamps;
- Object[] values3 = tablet3.values;
-
- for (long time = 0; time < 100; time++) {
- int row1 = tablet1.getRowSize();
- int row2 = tablet2.getRowSize();
- int row3 = tablet3.getRowSize();
- tablet1.addTimestamp(row1, time);
- tablet2.addTimestamp(row2, time);
- tablet3.addTimestamp(row3, time);
- for (int i = 0; i < 2; i++) {
- long[] sensor1 = (long[]) values1[i];
- sensor1[row1] = i;
- long[] sensor2 = (long[]) values2[i];
- sensor2[row2] = i;
- long[] sensor3 = (long[]) values3[i];
- sensor3[row3] = i;
- }
- if (tablet1.getRowSize() == tablet1.getMaxRowNumber()) {
- session.insertAlignedTablets(tabletMap, true);
-
- tablet1.reset();
- tablet2.reset();
- tablet3.reset();
- }
- }
-
- if (tablet1.getRowSize() != 0) {
- session.insertAlignedTablets(tabletMap, true);
- tablet1.reset();
- tablet2.reset();
- tablet3.reset();
- }
}
}
diff --git
a/example/session/src/main/java/org/apache/iotdb/SessionConcurrentExample.java
b/example/session/src/main/java/org/apache/iotdb/SessionConcurrentExample.java
index 694789fe5a6..40d3ed12e80 100644
---
a/example/session/src/main/java/org/apache/iotdb/SessionConcurrentExample.java
+++
b/example/session/src/main/java/org/apache/iotdb/SessionConcurrentExample.java
@@ -151,7 +151,6 @@ public class SessionConcurrentExample {
Tablet tablet = new Tablet(deviceId, schemaList, 100);
- // Method 1 to add tablet data
long timestamp = System.currentTimeMillis();
for (long row = 0; row < 100; row++) {
int rowIndex = tablet.getRowSize();
@@ -171,27 +170,5 @@ public class SessionConcurrentExample {
session.insertTablet(tablet);
tablet.reset();
}
-
- // Method 2 to add tablet data
- long[] timestamps = tablet.timestamps;
- Object[] values = tablet.values;
-
- for (long time = 0; time < 100; time++) {
- int row = tablet.getRowSize();
- tablet.addTimestamp(row, time);
- for (int i = 0; i < 3; i++) {
- long[] sensor = (long[]) values[i];
- sensor[row] = i;
- }
- if (tablet.getRowSize() == tablet.getMaxRowNumber()) {
- session.insertTablet(tablet, true);
- tablet.reset();
- }
- }
-
- if (tablet.getRowSize() != 0) {
- session.insertTablet(tablet);
- tablet.reset();
- }
}
}
diff --git a/example/session/src/main/java/org/apache/iotdb/SessionExample.java
b/example/session/src/main/java/org/apache/iotdb/SessionExample.java
index ede303b18c5..e894c08406f 100644
--- a/example/session/src/main/java/org/apache/iotdb/SessionExample.java
+++ b/example/session/src/main/java/org/apache/iotdb/SessionExample.java
@@ -35,7 +35,6 @@ import org.apache.tsfile.enums.TSDataType;
import org.apache.tsfile.file.metadata.enums.CompressionType;
import org.apache.tsfile.file.metadata.enums.TSEncoding;
import org.apache.tsfile.utils.Binary;
-import org.apache.tsfile.utils.BitMap;
import org.apache.tsfile.write.record.Tablet;
import org.apache.tsfile.write.schema.IMeasurementSchema;
import org.apache.tsfile.write.schema.MeasurementSchema;
@@ -401,7 +400,6 @@ public class SessionExample {
Tablet tablet = new Tablet(ROOT_SG1_D1, schemaList, 100);
- // Method 1 to add tablet data
long timestamp = System.currentTimeMillis();
for (long row = 0; row < 100; row++) {
@@ -422,28 +420,6 @@ public class SessionExample {
session.insertTablet(tablet);
tablet.reset();
}
-
- // Method 2 to add tablet data
- long[] timestamps = tablet.timestamps;
- Object[] values = tablet.values;
-
- for (long time = 0; time < 100; time++) {
- int row = tablet.getRowSize();
- tablet.addTimestamp(row, time);
- for (int i = 0; i < 3; i++) {
- long[] sensor = (long[]) values[i];
- sensor[row] = i;
- }
- if (tablet.getRowSize() == tablet.getMaxRowNumber()) {
- session.insertTablet(tablet, true);
- tablet.reset();
- }
- }
-
- if (tablet.getRowSize() != 0) {
- session.insertTablet(tablet);
- tablet.reset();
- }
}
private static void insertTabletWithNullValues()
@@ -465,11 +441,7 @@ public class SessionExample {
Tablet tablet = new Tablet(ROOT_SG1_D1, schemaList, 100);
- // Method 1 to add tablet data
insertTablet1(schemaList, tablet);
-
- // Method 2 to add tablet data
- insertTablet2(schemaList, tablet);
}
private static void insertTablet1(List<IMeasurementSchema> schemaList,
Tablet tablet)
@@ -482,11 +454,9 @@ public class SessionExample {
tablet.addTimestamp(rowIndex, timestamp);
for (int s = 0; s < 3; s++) {
long value = random.nextLong();
- // mark null value
- if (row % 3 == s) {
- tablet.bitMaps[s].mark((int) row);
+ if (row % 3 != s) {
+ tablet.addValue(schemaList.get(s).getMeasurementName(), rowIndex,
value);
}
- tablet.addValue(schemaList.get(s).getMeasurementName(), rowIndex,
value);
}
if (tablet.getRowSize() == tablet.getMaxRowNumber()) {
session.insertTablet(tablet, true);
@@ -501,39 +471,6 @@ public class SessionExample {
}
}
- private static void insertTablet2(List<IMeasurementSchema> schemaList,
Tablet tablet)
- throws IoTDBConnectionException, StatementExecutionException {
- long[] timestamps = tablet.timestamps;
- Object[] values = tablet.values;
- BitMap[] bitMaps = new BitMap[schemaList.size()];
- for (int s = 0; s < 3; s++) {
- bitMaps[s] = new BitMap(tablet.getMaxRowNumber());
- }
- tablet.bitMaps = bitMaps;
-
- for (long time = 0; time < 100; time++) {
- int row = tablet.getRowSize();
- tablet.addTimestamp(row, time);
- for (int i = 0; i < 3; i++) {
- long[] sensor = (long[]) values[i];
- // mark null value
- if (row % 3 == i) {
- bitMaps[i].mark(row);
- }
- sensor[row] = i;
- }
- if (tablet.getRowSize() == tablet.getMaxRowNumber()) {
- session.insertTablet(tablet, true);
- tablet.reset();
- }
- }
-
- if (tablet.getRowSize() != 0) {
- session.insertTablet(tablet);
- tablet.reset();
- }
- }
-
private static void insertTablets() throws IoTDBConnectionException,
StatementExecutionException {
// The schema of measurements of one device
// only measurementId and data type in MeasurementSchema take effects in
Tablet
@@ -551,7 +488,6 @@ public class SessionExample {
tabletMap.put("root.sg1.d2", tablet2);
tabletMap.put("root.sg1.d3", tablet3);
- // Method 1 to add tablet data
long timestamp = System.currentTimeMillis();
for (long row = 0; row < 100; row++) {
int row1 = tablet1.getRowSize();
@@ -581,45 +517,6 @@ public class SessionExample {
tablet2.reset();
tablet3.reset();
}
-
- // Method 2 to add tablet data
- long[] timestamps1 = tablet1.timestamps;
- Object[] values1 = tablet1.values;
- long[] timestamps2 = tablet2.timestamps;
- Object[] values2 = tablet2.values;
- long[] timestamps3 = tablet3.timestamps;
- Object[] values3 = tablet3.values;
-
- for (long time = 0; time < 100; time++) {
- int row1 = tablet1.getRowSize();
- int row2 = tablet2.getRowSize();
- int row3 = tablet3.getRowSize();
- timestamps1[row1] = time;
- timestamps2[row2] = time;
- timestamps3[row3] = time;
- for (int i = 0; i < 3; i++) {
- long[] sensor1 = (long[]) values1[i];
- sensor1[row1] = i;
- long[] sensor2 = (long[]) values2[i];
- sensor2[row2] = i;
- long[] sensor3 = (long[]) values3[i];
- sensor3[row3] = i;
- }
- if (tablet1.getRowSize() == tablet1.getMaxRowNumber()) {
- session.insertTablets(tabletMap, true);
-
- tablet1.reset();
- tablet2.reset();
- tablet3.reset();
- }
- }
-
- if (tablet1.getRowSize() != 0) {
- session.insertTablets(tabletMap, true);
- tablet1.reset();
- tablet2.reset();
- tablet3.reset();
- }
}
/**
diff --git
a/example/trigger/src/main/java/org/apache/iotdb/trigger/LoggerTrigger.java
b/example/trigger/src/main/java/org/apache/iotdb/trigger/LoggerTrigger.java
index 81aab6e748f..bb95f62de08 100644
--- a/example/trigger/src/main/java/org/apache/iotdb/trigger/LoggerTrigger.java
+++ b/example/trigger/src/main/java/org/apache/iotdb/trigger/LoggerTrigger.java
@@ -39,17 +39,17 @@ public class LoggerTrigger implements Trigger {
List<IMeasurementSchema> measurementSchemaList = tablet.getSchemas();
for (int i = 0, n = measurementSchemaList.size(); i < n; i++) {
if (measurementSchemaList.get(i).getType().equals(TSDataType.DOUBLE)) {
- logDouble((double[]) tablet.values[i]);
+ logDouble((double[]) tablet.getValues()[i]);
} else if
(measurementSchemaList.get(i).getType().equals(TSDataType.FLOAT)) {
- logFloat((float[]) tablet.values[i]);
+ logFloat((float[]) tablet.getValues()[i]);
} else if
(measurementSchemaList.get(i).getType().equals(TSDataType.INT64)) {
- logLong((long[]) tablet.values[i]);
+ logLong((long[]) tablet.getValues()[i]);
} else if
(measurementSchemaList.get(i).getType().equals(TSDataType.INT32)) {
- logInt((int[]) tablet.values[i]);
+ logInt((int[]) tablet.getValues()[i]);
} else if
(measurementSchemaList.get(i).getType().equals(TSDataType.TEXT)) {
- logText((Binary[]) tablet.values[i]);
+ logText((Binary[]) tablet.getValues()[i]);
} else if
(measurementSchemaList.get(i).getType().equals(TSDataType.BOOLEAN)) {
- logBoolean((boolean[]) tablet.values[i]);
+ logBoolean((boolean[]) tablet.getValues()[i]);
}
}
return true;
diff --git
a/example/trigger/src/main/java/org/apache/iotdb/trigger/StatisticsUpdaterTrigger.java
b/example/trigger/src/main/java/org/apache/iotdb/trigger/StatisticsUpdaterTrigger.java
index c8d1c3ece8e..4f1217b6f66 100644
---
a/example/trigger/src/main/java/org/apache/iotdb/trigger/StatisticsUpdaterTrigger.java
+++
b/example/trigger/src/main/java/org/apache/iotdb/trigger/StatisticsUpdaterTrigger.java
@@ -81,12 +81,12 @@ public class StatisticsUpdaterTrigger implements Trigger {
@Override
public boolean fire(Tablet tablet) throws Exception {
ensureSession();
- if (tablet.bitMaps == null) {
+ if (tablet.getBitMaps() == null) {
cnt.addAndGet((long) tablet.getRowSize() * tablet.getSchemas().size());
return true;
}
for (int column = 0; column < tablet.getSchemas().size(); column++) {
- BitMap bitMap = tablet.bitMaps[column];
+ BitMap bitMap = tablet.getBitMaps()[column];
if (bitMap == null) {
cnt.addAndGet(tablet.getRowSize());
} else {
diff --git
a/integration-test/src/main/java/org/apache/iotdb/it/utils/TsFileGenerator.java
b/integration-test/src/main/java/org/apache/iotdb/it/utils/TsFileGenerator.java
index a248ceba018..e0035353bf3 100644
---
a/integration-test/src/main/java/org/apache/iotdb/it/utils/TsFileGenerator.java
+++
b/integration-test/src/main/java/org/apache/iotdb/it/utils/TsFileGenerator.java
@@ -25,10 +25,8 @@ import
org.apache.iotdb.db.storageengine.dataregion.modification.ModEntry;
import
org.apache.iotdb.db.storageengine.dataregion.modification.ModificationFile;
import
org.apache.iotdb.db.storageengine.dataregion.modification.TreeDeletionEntry;
-import org.apache.tsfile.common.conf.TSFileConfig;
import org.apache.tsfile.exception.write.WriteProcessException;
import org.apache.tsfile.read.common.Path;
-import org.apache.tsfile.utils.Binary;
import org.apache.tsfile.write.TsFileWriter;
import org.apache.tsfile.write.record.Tablet;
import org.apache.tsfile.write.schema.IMeasurementSchema;
@@ -98,8 +96,6 @@ public class TsFileGenerator implements AutoCloseable {
final List<IMeasurementSchema> schemas =
device2MeasurementSchema.get(device);
final TreeSet<Long> timeSet = device2TimeSet.get(device);
final Tablet tablet = new Tablet(device, schemas);
- final long[] timestamps = tablet.timestamps;
- final Object[] values = tablet.values;
final long sensorNum = schemas.size();
long startTime = timeSet.isEmpty() ? 0L : timeSet.last();
@@ -109,25 +105,17 @@ public class TsFileGenerator implements AutoCloseable {
tablet.addTimestamp(row, startTime);
timeSet.add(startTime);
for (int i = 0; i < sensorNum; i++) {
- generateDataPoint(values[i], row, schemas.get(i));
+ generateDataPoint(tablet, i, row, schemas.get(i));
}
// write
if (tablet.getRowSize() == tablet.getMaxRowNumber()) {
- if (!isAligned) {
- writer.writeTree(tablet);
- } else {
- writer.writeAligned(tablet);
- }
+ writer.writeTree(tablet);
tablet.reset();
}
}
// write
if (tablet.getRowSize() != 0) {
- if (!isAligned) {
- writer.writeTree(tablet);
- } else {
- writer.writeAligned(tablet);
- }
+ writer.writeTree(tablet);
tablet.reset();
}
@@ -144,8 +132,6 @@ public class TsFileGenerator implements AutoCloseable {
final List<IMeasurementSchema> schemas =
device2MeasurementSchema.get(device);
final TreeSet<Long> timeSet = device2TimeSet.get(device);
final Tablet tablet = new Tablet(device, schemas);
- final long[] timestamps = tablet.timestamps;
- final Object[] values = tablet.values;
final long sensorNum = schemas.size();
long startTime = startTimestamp;
@@ -155,15 +141,11 @@ public class TsFileGenerator implements AutoCloseable {
tablet.addTimestamp(row, startTime);
timeSet.add(startTime);
for (int i = 0; i < sensorNum; i++) {
- generateDataPoint(values[i], row, schemas.get(i));
+ generateDataPoint(tablet, i, row, schemas.get(i));
}
// write
if (tablet.getRowSize() == tablet.getMaxRowNumber()) {
- if (!isAligned) {
- writer.writeTree(tablet);
- } else {
- writer.writeAligned(tablet);
- }
+ writer.writeTree(tablet);
tablet.reset();
}
}
@@ -180,72 +162,67 @@ public class TsFileGenerator implements AutoCloseable {
LOGGER.info("Write {} points into device {}", number, device);
}
- private void generateDataPoint(final Object obj, final int row, final
IMeasurementSchema schema) {
+ private void generateDataPoint(
+ final Tablet tablet, final int column, final int row, final
IMeasurementSchema schema) {
switch (schema.getType()) {
case INT32:
- generateINT32(obj, row);
+ generateINT32(tablet, column, row);
break;
case DATE:
- generateDATE(obj, row);
+ generateDATE(tablet, column, row);
break;
case INT64:
case TIMESTAMP:
- generateINT64(obj, row);
+ generateINT64(tablet, column, row);
break;
case FLOAT:
- generateFLOAT(obj, row);
+ generateFLOAT(tablet, column, row);
break;
case DOUBLE:
- generateDOUBLE(obj, row);
+ generateDOUBLE(tablet, column, row);
break;
case BOOLEAN:
- generateBOOLEAN(obj, row);
+ generateBOOLEAN(tablet, column, row);
break;
case TEXT:
case BLOB:
case STRING:
- generateTEXT(obj, row);
+ generateTEXT(tablet, column, row);
break;
default:
LOGGER.error("Wrong data type {}.", schema.getType());
}
}
- private void generateINT32(final Object obj, final int row) {
- final int[] ints = (int[]) obj;
- ints[row] = random.nextInt();
+ private void generateINT32(final Tablet tablet, final int column, final int
row) {
+ tablet.addValue(row, column, random.nextInt());
}
- private void generateDATE(final Object obj, final int row) {
- final LocalDate[] dates = (LocalDate[]) obj;
- dates[row] =
- LocalDate.of(1000 + random.nextInt(9000), 1 + random.nextInt(12), 1 +
random.nextInt(28));
+ private void generateDATE(final Tablet tablet, final int column, final int
row) {
+ tablet.addValue(
+ row,
+ column,
+ LocalDate.of(1000 + random.nextInt(9000), 1 + random.nextInt(12), 1 +
random.nextInt(28)));
}
- private void generateINT64(final Object obj, final int row) {
- final long[] longs = (long[]) obj;
- longs[row] = random.nextLong();
+ private void generateINT64(final Tablet tablet, final int column, final int
row) {
+ tablet.addValue(row, column, random.nextLong());
}
- private void generateFLOAT(final Object obj, final int row) {
- final float[] floats = (float[]) obj;
- floats[row] = random.nextFloat();
+ private void generateFLOAT(final Tablet tablet, final int column, final int
row) {
+ tablet.addValue(row, column, random.nextFloat());
}
- private void generateDOUBLE(final Object obj, final int row) {
- final double[] doubles = (double[]) obj;
- doubles[row] = random.nextDouble();
+ private void generateDOUBLE(final Tablet tablet, final int column, final int
row) {
+ tablet.addValue(row, column, random.nextDouble());
}
- private void generateBOOLEAN(final Object obj, final int row) {
- final boolean[] booleans = (boolean[]) obj;
- booleans[row] = random.nextBoolean();
+ private void generateBOOLEAN(final Tablet tablet, final int column, final
int row) {
+ tablet.addValue(row, column, random.nextBoolean());
}
- private void generateTEXT(final Object obj, final int row) {
- final Binary[] binaries = (Binary[]) obj;
- binaries[row] =
- new Binary(String.format("test point %d", random.nextInt()),
TSFileConfig.STRING_CHARSET);
+ private void generateTEXT(final Tablet tablet, final int column, final int
row) {
+ tablet.addValue(row, column, String.format("test point %d",
random.nextInt()));
}
public void generateDeletion(final String device, final int number)
diff --git
a/integration-test/src/main/java/org/apache/iotdb/it/utils/TsFileTableGenerator.java
b/integration-test/src/main/java/org/apache/iotdb/it/utils/TsFileTableGenerator.java
index 7808098eca2..ebcb58093f2 100644
---
a/integration-test/src/main/java/org/apache/iotdb/it/utils/TsFileTableGenerator.java
+++
b/integration-test/src/main/java/org/apache/iotdb/it/utils/TsFileTableGenerator.java
@@ -19,11 +19,9 @@
package org.apache.iotdb.it.utils;
-import org.apache.tsfile.common.conf.TSFileConfig;
import org.apache.tsfile.enums.TSDataType;
import org.apache.tsfile.exception.write.WriteProcessException;
import org.apache.tsfile.file.metadata.TableSchema;
-import org.apache.tsfile.utils.Binary;
import org.apache.tsfile.write.TsFileWriter;
import org.apache.tsfile.write.record.Tablet;
import org.apache.tsfile.write.schema.IMeasurementSchema;
@@ -74,8 +72,7 @@ public class TsFileTableGenerator implements AutoCloseable {
table2ColumnCategory.put(tableName, columnCategoryList);
}
- public void generateData(
- final String tableName, final int number, final long timeGap, final
boolean isAligned)
+ public void generateData(final String tableName, final int number, final
long timeGap)
throws IOException, WriteProcessException {
final List<IMeasurementSchema> schemas =
table2MeasurementSchema.get(tableName);
final List<String> columnNameList =
@@ -85,7 +82,7 @@ public class TsFileTableGenerator implements AutoCloseable {
final List<Tablet.ColumnCategory> columnCategoryList =
table2ColumnCategory.get(tableName);
final TreeSet<Long> timeSet = table2TimeSet.get(tableName);
final Tablet tablet = new Tablet(tableName, columnNameList, dataTypeList,
columnCategoryList);
- final Object[] values = tablet.values;
+ final Object[] values = tablet.getValues();
final long sensorNum = schemas.size();
long startTime = timeSet.isEmpty() ? 0L : timeSet.last();
@@ -95,97 +92,84 @@ public class TsFileTableGenerator implements AutoCloseable {
tablet.addTimestamp(row, startTime);
timeSet.add(startTime);
for (int i = 0; i < sensorNum; i++) {
- generateDataPoint(values[i], row, schemas.get(i));
+ generateDataPoint(tablet, i, row, schemas.get(i));
}
// write
if (tablet.getRowSize() == tablet.getMaxRowNumber()) {
- if (!isAligned) {
- writer.writeTable(tablet);
- } else {
- writer.writeAligned(tablet);
- }
+ writer.writeTable(tablet);
tablet.reset();
}
}
// write
if (tablet.getRowSize() != 0) {
- if (!isAligned) {
- writer.writeTable(tablet);
- } else {
- writer.writeAligned(tablet);
- }
+ writer.writeTable(tablet);
tablet.reset();
}
LOGGER.info("Write {} points into table {}", number, tableName);
}
- private void generateDataPoint(final Object obj, final int row, final
IMeasurementSchema schema) {
+ private void generateDataPoint(
+ final Tablet tablet, final int column, final int row, final
IMeasurementSchema schema) {
switch (schema.getType()) {
case INT32:
- generateINT32(obj, row);
+ generateINT32(tablet, column, row);
break;
case DATE:
- generateDATE(obj, row);
+ generateDATE(tablet, column, row);
break;
case INT64:
case TIMESTAMP:
- generateINT64(obj, row);
+ generateINT64(tablet, column, row);
break;
case FLOAT:
- generateFLOAT(obj, row);
+ generateFLOAT(tablet, column, row);
break;
case DOUBLE:
- generateDOUBLE(obj, row);
+ generateDOUBLE(tablet, column, row);
break;
case BOOLEAN:
- generateBOOLEAN(obj, row);
+ generateBOOLEAN(tablet, column, row);
break;
case TEXT:
case BLOB:
case STRING:
- generateTEXT(obj, row);
+ generateTEXT(tablet, column, row);
break;
default:
LOGGER.error("Wrong data type {}.", schema.getType());
}
}
- private void generateINT32(final Object obj, final int row) {
- final int[] ints = (int[]) obj;
- ints[row] = random.nextInt();
+ private void generateINT32(final Tablet tablet, final int column, final int
row) {
+ tablet.addValue(row, column, random.nextInt());
}
- private void generateDATE(final Object obj, final int row) {
- final LocalDate[] dates = (LocalDate[]) obj;
- dates[row] =
- LocalDate.of(1000 + random.nextInt(9000), 1 + random.nextInt(12), 1 +
random.nextInt(28));
+ private void generateDATE(final Tablet tablet, final int column, final int
row) {
+ tablet.addValue(
+ row,
+ column,
+ LocalDate.of(1000 + random.nextInt(9000), 1 + random.nextInt(12), 1 +
random.nextInt(28)));
}
- private void generateINT64(final Object obj, final int row) {
- final long[] longs = (long[]) obj;
- longs[row] = random.nextLong();
+ private void generateINT64(final Tablet tablet, final int column, final int
row) {
+ tablet.addValue(row, column, random.nextLong());
}
- private void generateFLOAT(final Object obj, final int row) {
- final float[] floats = (float[]) obj;
- floats[row] = random.nextFloat();
+ private void generateFLOAT(final Tablet tablet, final int column, final int
row) {
+ tablet.addValue(row, column, random.nextFloat());
}
- private void generateDOUBLE(final Object obj, final int row) {
- final double[] doubles = (double[]) obj;
- doubles[row] = random.nextDouble();
+ private void generateDOUBLE(final Tablet tablet, final int column, final int
row) {
+ tablet.addValue(row, column, random.nextDouble());
}
- private void generateBOOLEAN(final Object obj, final int row) {
- final boolean[] booleans = (boolean[]) obj;
- booleans[row] = random.nextBoolean();
+ private void generateBOOLEAN(final Tablet tablet, final int column, final
int row) {
+ tablet.addValue(row, column, random.nextBoolean());
}
- private void generateTEXT(final Object obj, final int row) {
- final Binary[] binaries = (Binary[]) obj;
- binaries[row] =
- new Binary(String.format("test point %d", random.nextInt()),
TSFileConfig.STRING_CHARSET);
+ private void generateTEXT(final Tablet tablet, final int column, final int
row) {
+ tablet.addValue(row, column, String.format("test point %d",
random.nextInt()));
}
public long getTotalNumber() {
diff --git
a/integration-test/src/test/java/org/apache/iotdb/db/it/IoTDBLoadTsFileIT.java
b/integration-test/src/test/java/org/apache/iotdb/db/it/IoTDBLoadTsFileIT.java
index fe31f97b451..b8e431356f5 100644
---
a/integration-test/src/test/java/org/apache/iotdb/db/it/IoTDBLoadTsFileIT.java
+++
b/integration-test/src/test/java/org/apache/iotdb/db/it/IoTDBLoadTsFileIT.java
@@ -991,7 +991,7 @@ public class IoTDBLoadTsFileIT {
try (final TsFileTableGenerator generator = new
TsFileTableGenerator(file)) {
generator.registerTable(SchemaConfig.TABLE_0, schemaList2,
columnCategories);
- generator.generateData(SchemaConfig.TABLE_0, lineCount,
PARTITION_INTERVAL / 10_000, false);
+ generator.generateData(SchemaConfig.TABLE_0, lineCount,
PARTITION_INTERVAL / 10_000);
}
try (final Connection connection =
diff --git
a/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeNullValueIT.java
b/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeNullValueIT.java
index d8a1cce4d75..3231fc523aa 100644
---
a/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeNullValueIT.java
+++
b/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeNullValueIT.java
@@ -113,17 +113,23 @@ public class IoTDBPipeNullValueIT extends
AbstractPipeDualAutoIT {
valuesForAllNull[0] = new float[] {0F};
valuesForAllNull[1] = new float[] {0F};
- partialNullTablet = new Tablet(deviceId, Arrays.asList(schemas), 1);
- partialNullTablet.values = valuesForPartialNull;
- partialNullTablet.timestamps = new long[] {3};
- partialNullTablet.setRowSize(1);
- partialNullTablet.bitMaps = bitMapsForPartialNull;
-
- allNullTablet = new Tablet(deviceId, Arrays.asList(schemas), 1);
- allNullTablet.values = valuesForAllNull;
- allNullTablet.timestamps = new long[] {4};
- allNullTablet.setRowSize(1);
- allNullTablet.bitMaps = bitMapsForAllNull;
+ partialNullTablet =
+ new Tablet(
+ deviceId,
+ Arrays.asList(schemas),
+ new long[] {3},
+ valuesForPartialNull,
+ bitMapsForPartialNull,
+ 1);
+
+ allNullTablet =
+ new Tablet(
+ deviceId,
+ Arrays.asList(schemas),
+ new long[] {4},
+ valuesForAllNull,
+ bitMapsForAllNull,
+ 1);
}
@Override
diff --git
a/integration-test/src/test/java/org/apache/iotdb/pipe/it/manual/IoTDBPipeReqAutoSliceIT.java
b/integration-test/src/test/java/org/apache/iotdb/pipe/it/manual/IoTDBPipeReqAutoSliceIT.java
index 78a799601cd..953f1faec33 100644
---
a/integration-test/src/test/java/org/apache/iotdb/pipe/it/manual/IoTDBPipeReqAutoSliceIT.java
+++
b/integration-test/src/test/java/org/apache/iotdb/pipe/it/manual/IoTDBPipeReqAutoSliceIT.java
@@ -358,7 +358,7 @@ public class IoTDBPipeReqAutoSliceIT extends
AbstractPipeDualManualIT {
set.add(
String.format(
"%d,%d,%d,",
- tablet.timestamps[i], ((int[]) tablet.values[0])[i], ((int[])
tablet.values[1])[i]));
+ tablet.getTimestamp(i), (int) tablet.getValue(i, 0), (int)
tablet.getValue(i, 1)));
}
TestUtils.assertDataEventuallyOnEnv(
receiverEnv,
@@ -400,7 +400,7 @@ public class IoTDBPipeReqAutoSliceIT extends
AbstractPipeDualManualIT {
}
private List<Long> getTimestampList(Tablet tablet) {
- long[] timestamps = tablet.timestamps;
+ long[] timestamps = tablet.getTimestamps();
List<Long> data = new ArrayList<>(timestamps.length);
for (long timestamp : timestamps) {
data.add(timestamp);
@@ -438,7 +438,7 @@ public class IoTDBPipeReqAutoSliceIT extends
AbstractPipeDualManualIT {
private List<List<Object>> generateTabletInsertRecordForTable(final Tablet
tablet) {
List<List<Object>> insertRecords = new ArrayList<>(tablet.getRowSize());
final List<IMeasurementSchema> schemas = tablet.getSchemas();
- final Object[] values = tablet.values;
+ final Object[] values = tablet.getValues();
for (int i = 0; i < tablet.getRowSize(); i++) {
List<Object> insertRecord = new ArrayList<>();
for (int j = 0; j < schemas.size(); j++) {
@@ -461,7 +461,7 @@ public class IoTDBPipeReqAutoSliceIT extends
AbstractPipeDualManualIT {
private List<List<String>> generateTabletInsertStrRecordForTable(Tablet
tablet) {
List<List<String>> insertRecords = new ArrayList<>(tablet.getRowSize());
final List<IMeasurementSchema> schemas = tablet.getSchemas();
- final Object[] values = tablet.values;
+ final Object[] values = tablet.getValues();
for (int i = 0; i < tablet.getRowSize(); i++) {
List<String> insertRecord = new ArrayList<>();
for (int j = 0; j < schemas.size(); j++) {
diff --git
a/integration-test/src/test/java/org/apache/iotdb/pipe/it/manual/IoTDBPipeTypeConversionISessionIT.java
b/integration-test/src/test/java/org/apache/iotdb/pipe/it/manual/IoTDBPipeTypeConversionISessionIT.java
index 6d6d22fafce..b94aea2f237 100644
---
a/integration-test/src/test/java/org/apache/iotdb/pipe/it/manual/IoTDBPipeTypeConversionISessionIT.java
+++
b/integration-test/src/test/java/org/apache/iotdb/pipe/it/manual/IoTDBPipeTypeConversionISessionIT.java
@@ -358,7 +358,7 @@ public class IoTDBPipeTypeConversionISessionIT extends
AbstractPipeDualManualIT
validateResultSet(
query(receiverSession, tablet.getSchemas(),
tablet.getDeviceId()),
expectedValues,
- tablet.timestamps);
+ tablet.getTimestamps());
} catch (Exception e) {
fail();
}
@@ -541,7 +541,7 @@ public class IoTDBPipeTypeConversionISessionIT extends
AbstractPipeDualManualIT
}
private List<Long> getTimestampList(Tablet tablet) {
- long[] timestamps = tablet.timestamps;
+ long[] timestamps = tablet.getTimestamps();
List<Long> data = new ArrayList<>(timestamps.length);
for (long timestamp : timestamps) {
data.add(timestamp);
@@ -580,7 +580,7 @@ public class IoTDBPipeTypeConversionISessionIT extends
AbstractPipeDualManualIT
final Tablet tablet, List<Pair<MeasurementSchema, MeasurementSchema>>
pairs) {
List<List<Object>> insertRecords = new ArrayList<>(tablet.getRowSize());
final List<IMeasurementSchema> schemas = tablet.getSchemas();
- final Object[] values = tablet.values;
+ final Object[] values = tablet.getValues();
for (int i = 0; i < tablet.getRowSize(); i++) {
List<Object> insertRecord = new ArrayList<>();
for (int j = 0; j < schemas.size(); j++) {
@@ -648,7 +648,7 @@ public class IoTDBPipeTypeConversionISessionIT extends
AbstractPipeDualManualIT
private List<List<Object>> generateTabletInsertRecordForTable(final Tablet
tablet) {
List<List<Object>> insertRecords = new ArrayList<>(tablet.getRowSize());
final List<IMeasurementSchema> schemas = tablet.getSchemas();
- final Object[] values = tablet.values;
+ final Object[] values = tablet.getValues();
for (int i = 0; i < tablet.getRowSize(); i++) {
List<Object> insertRecord = new ArrayList<>();
for (int j = 0; j < schemas.size(); j++) {
@@ -691,7 +691,7 @@ public class IoTDBPipeTypeConversionISessionIT extends
AbstractPipeDualManualIT
private List<List<String>> generateTabletInsertStrRecordForTable(Tablet
tablet) {
List<List<String>> insertRecords = new ArrayList<>(tablet.getRowSize());
final List<IMeasurementSchema> schemas = tablet.getSchemas();
- final Object[] values = tablet.values;
+ final Object[] values = tablet.getValues();
for (int i = 0; i < tablet.getRowSize(); i++) {
List<String> insertRecord = new ArrayList<>();
for (int j = 0; j < schemas.size(); j++) {
diff --git
a/integration-test/src/test/java/org/apache/iotdb/pipe/it/tablemodel/IoTDBPipeTypeConversionISessionIT.java
b/integration-test/src/test/java/org/apache/iotdb/pipe/it/tablemodel/IoTDBPipeTypeConversionISessionIT.java
index b3e40bb8a10..dcd48928fdc 100644
---
a/integration-test/src/test/java/org/apache/iotdb/pipe/it/tablemodel/IoTDBPipeTypeConversionISessionIT.java
+++
b/integration-test/src/test/java/org/apache/iotdb/pipe/it/tablemodel/IoTDBPipeTypeConversionISessionIT.java
@@ -137,7 +137,7 @@ public class IoTDBPipeTypeConversionISessionIT extends
AbstractPipeTableModelTes
validateResultSet(
query(receiverSession, tablet.getSchemas(),
tablet.getTableName()),
expectedValues,
- tablet.timestamps);
+ tablet.getTimestamps());
} catch (Exception e) {
fail(e.getMessage());
}
@@ -335,7 +335,7 @@ public class IoTDBPipeTypeConversionISessionIT extends
AbstractPipeTableModelTes
final Tablet tablet, List<Pair<MeasurementSchema, MeasurementSchema>>
pairs) {
List<List<Object>> insertRecords = new ArrayList<>(tablet.getRowSize());
final List<IMeasurementSchema> schemas = tablet.getSchemas();
- final Object[] values = tablet.values;
+ final Object[] values = tablet.getValues();
for (int i = 0; i < tablet.getRowSize(); i++) {
List<Object> insertRecord = new ArrayList<>();
for (int j = 0; j < schemas.size(); j++) {
@@ -349,7 +349,7 @@ public class IoTDBPipeTypeConversionISessionIT extends
AbstractPipeTableModelTes
ValueConverter.convert(
sourceType,
targetType,
- tablet.bitMaps[j].isMarked(i) ? null : ((long[])
values[j])[i]);
+ tablet.getBitMaps()[j].isMarked(i) ? null : ((long[])
values[j])[i]);
insertRecord.add(convert(value, targetType));
break;
case INT32:
@@ -357,7 +357,7 @@ public class IoTDBPipeTypeConversionISessionIT extends
AbstractPipeTableModelTes
ValueConverter.convert(
sourceType,
targetType,
- tablet.bitMaps[j].isMarked(i) ? null : ((int[])
values[j])[i]);
+ tablet.getBitMaps()[j].isMarked(i) ? null : ((int[])
values[j])[i]);
insertRecord.add(convert(value, targetType));
break;
case DOUBLE:
@@ -365,7 +365,7 @@ public class IoTDBPipeTypeConversionISessionIT extends
AbstractPipeTableModelTes
ValueConverter.convert(
sourceType,
targetType,
- tablet.bitMaps[j].isMarked(i) ? null : ((double[])
values[j])[i]);
+ tablet.getBitMaps()[j].isMarked(i) ? null : ((double[])
values[j])[i]);
insertRecord.add(convert(value, targetType));
break;
case FLOAT:
@@ -373,7 +373,7 @@ public class IoTDBPipeTypeConversionISessionIT extends
AbstractPipeTableModelTes
ValueConverter.convert(
sourceType,
targetType,
- tablet.bitMaps[j].isMarked(i) ? null : ((float[])
values[j])[i]);
+ tablet.getBitMaps()[j].isMarked(i) ? null : ((float[])
values[j])[i]);
insertRecord.add(convert(value, targetType));
break;
case DATE:
@@ -381,7 +381,7 @@ public class IoTDBPipeTypeConversionISessionIT extends
AbstractPipeTableModelTes
ValueConverter.convert(
sourceType,
targetType,
- tablet.bitMaps[j].isMarked(i)
+ tablet.getBitMaps()[j].isMarked(i)
? null
: DateUtils.parseDateExpressionToInt(((LocalDate[])
values[j])[i]));
insertRecord.add(convert(value, targetType));
@@ -392,7 +392,7 @@ public class IoTDBPipeTypeConversionISessionIT extends
AbstractPipeTableModelTes
ValueConverter.convert(
sourceType,
targetType,
- tablet.bitMaps[j].isMarked(i) ? null : ((Binary[])
values[j])[i]);
+ tablet.getBitMaps()[j].isMarked(i) ? null : ((Binary[])
values[j])[i]);
insertRecord.add(convert(value, targetType));
break;
case BLOB:
@@ -400,7 +400,7 @@ public class IoTDBPipeTypeConversionISessionIT extends
AbstractPipeTableModelTes
ValueConverter.convert(
sourceType,
targetType,
- tablet.bitMaps[j].isMarked(i) ? null : ((Binary[])
values[j])[i]);
+ tablet.getBitMaps()[j].isMarked(i) ? null : ((Binary[])
values[j])[i]);
insertRecord.add(convert(value, targetType));
break;
case BOOLEAN:
@@ -408,12 +408,12 @@ public class IoTDBPipeTypeConversionISessionIT extends
AbstractPipeTableModelTes
ValueConverter.convert(
sourceType,
targetType,
- tablet.bitMaps[j].isMarked(i) ? null : ((boolean[])
values[j])[i]);
+ tablet.getBitMaps()[j].isMarked(i) ? null : ((boolean[])
values[j])[i]);
insertRecord.add(convert(value, targetType));
break;
}
}
- insertRecord.add(tablet.timestamps[i]);
+ insertRecord.add(tablet.getTimestamp(i));
insertRecords.add(insertRecord);
}
diff --git
a/integration-test/src/test/java/org/apache/iotdb/pipe/it/tablemodel/TableModelUtils.java
b/integration-test/src/test/java/org/apache/iotdb/pipe/it/tablemodel/TableModelUtils.java
index 46333e5c470..e234cc6bd5c 100644
---
a/integration-test/src/test/java/org/apache/iotdb/pipe/it/tablemodel/TableModelUtils.java
+++
b/integration-test/src/test/java/org/apache/iotdb/pipe/it/tablemodel/TableModelUtils.java
@@ -271,7 +271,7 @@ public class TableModelUtils {
for (int i = 0; i < tablet.getRowSize(); i++) {
StringBuilder stringBuffer = new StringBuilder();
for (int j = 0; j < tablet.getSchemas().size(); j++) {
- BitMap bitMap = tablet.bitMaps[j];
+ BitMap bitMap = tablet.getBitMaps()[j];
if (bitMap.isMarked(i)) {
stringBuffer.append("null,");
continue;
@@ -280,45 +280,46 @@ public class TableModelUtils {
case TIMESTAMP:
final String time =
RpcUtils.formatDatetime(
- "default", "ms", ((long[]) tablet.values[j])[i],
ZoneOffset.UTC);
+ "default", "ms", (long) tablet.getValue(i, j),
ZoneOffset.UTC);
stringBuffer.append(time);
stringBuffer.append(",");
break;
case DATE:
- stringBuffer.append(((LocalDate[])
tablet.values[j])[i].toString());
+ stringBuffer.append(tablet.getValue(i, j).toString());
stringBuffer.append(",");
break;
case BLOB:
stringBuffer.append(
BytesUtils.parseBlobByteArrayToString(
- ((Binary[]) tablet.values[j])[i].getValues()));
+ ((Binary) tablet.getValue(i, j)).getValues()));
stringBuffer.append(",");
break;
case TEXT:
case STRING:
stringBuffer.append(
- new String(((Binary[]) tablet.values[j])[i].getValues(),
StandardCharsets.UTF_8));
+ new String(((Binary) tablet.getValue(i, j)).getValues(),
StandardCharsets.UTF_8));
stringBuffer.append(",");
break;
case DOUBLE:
- stringBuffer.append(((double[]) tablet.values[j])[i]);
+ stringBuffer.append((double) tablet.getValue(i, j));
stringBuffer.append(",");
break;
case FLOAT:
- stringBuffer.append(((float[]) tablet.values[j])[i]);
+ stringBuffer.append((float) tablet.getValue(i, j));
stringBuffer.append(",");
break;
case INT32:
- stringBuffer.append(((int[]) tablet.values[j])[i]);
+ stringBuffer.append((int) tablet.getValue(i, j));
stringBuffer.append(",");
break;
case INT64:
- stringBuffer.append(((long[]) tablet.values[j])[i]);
+ stringBuffer.append((long) tablet.getValue(i, j));
stringBuffer.append(",");
break;
}
}
- String time = RpcUtils.formatDatetime("default", "ms",
tablet.timestamps[i], ZoneOffset.UTC);
+ String time =
+ RpcUtils.formatDatetime("default", "ms", tablet.getTimestamp(i),
ZoneOffset.UTC);
stringBuffer.append(time);
stringBuffer.append(",");
expectedResSet.add(stringBuffer.toString());
diff --git
a/integration-test/src/test/java/org/apache/iotdb/session/it/IoTDBSessionComplexIT.java
b/integration-test/src/test/java/org/apache/iotdb/session/it/IoTDBSessionComplexIT.java
index 6e8a231c310..eef6614aa62 100644
---
a/integration-test/src/test/java/org/apache/iotdb/session/it/IoTDBSessionComplexIT.java
+++
b/integration-test/src/test/java/org/apache/iotdb/session/it/IoTDBSessionComplexIT.java
@@ -299,18 +299,13 @@ public class IoTDBSessionComplexIT {
if (tablet.getRowSize() != 0) {
session.insertTablet(tablet);
tablet.reset();
- tablet.bitMaps = null;
}
- long[] timestamps = tablet.timestamps;
- Object[] values = tablet.values;
-
for (long time = 0; time < 100; time++) {
int row = tablet.getRowSize();
tablet.addTimestamp(row, time);
for (int i = 0; i < 3; i++) {
- long[] sensor = (long[]) values[i];
- sensor[row] = i;
+ tablet.addValue(row, i, (long) i);
}
if (tablet.getRowSize() == tablet.getMaxRowNumber()) {
session.insertTablet(tablet);
@@ -321,7 +316,6 @@ public class IoTDBSessionComplexIT {
if (tablet.getRowSize() != 0) {
session.insertTablet(tablet);
tablet.reset();
- tablet.bitMaps = null;
}
}
diff --git
a/integration-test/src/test/java/org/apache/iotdb/session/it/IoTDBSessionSimpleIT.java
b/integration-test/src/test/java/org/apache/iotdb/session/it/IoTDBSessionSimpleIT.java
index 77ce160cbf5..ccd9ceedda8 100644
---
a/integration-test/src/test/java/org/apache/iotdb/session/it/IoTDBSessionSimpleIT.java
+++
b/integration-test/src/test/java/org/apache/iotdb/session/it/IoTDBSessionSimpleIT.java
@@ -42,7 +42,6 @@ import org.apache.tsfile.file.metadata.enums.TSEncoding;
import org.apache.tsfile.read.common.Field;
import org.apache.tsfile.read.common.RowRecord;
import org.apache.tsfile.utils.Binary;
-import org.apache.tsfile.utils.BitMap;
import org.apache.tsfile.write.record.Tablet;
import org.apache.tsfile.write.schema.IMeasurementSchema;
import org.apache.tsfile.write.schema.MeasurementSchema;
@@ -507,14 +506,9 @@ public class IoTDBSessionSimpleIT {
new Binary(String.valueOf(time), TSFileConfig.STRING_CHARSET));
}
- BitMap[] bitMaps = new BitMap[schemaList.size()];
for (int i = 0; i < schemaList.size(); i++) {
- if (bitMaps[i] == null) {
- bitMaps[i] = new BitMap(10);
- }
- bitMaps[i].mark(i);
+ tablet.getBitMaps()[i].mark(i);
}
- tablet.bitMaps = bitMaps;
if (tablet.getRowSize() != 0) {
session.insertTablet(tablet);
diff --git
a/integration-test/src/test/java/org/apache/iotdb/session/it/IoTDBSessionSyntaxConventionIT.java
b/integration-test/src/test/java/org/apache/iotdb/session/it/IoTDBSessionSyntaxConventionIT.java
index 3c6acbae52c..080345a5467 100644
---
a/integration-test/src/test/java/org/apache/iotdb/session/it/IoTDBSessionSyntaxConventionIT.java
+++
b/integration-test/src/test/java/org/apache/iotdb/session/it/IoTDBSessionSyntaxConventionIT.java
@@ -33,8 +33,6 @@ import org.apache.iotdb.session.template.MeasurementNode;
import org.apache.tsfile.enums.TSDataType;
import org.apache.tsfile.file.metadata.enums.CompressionType;
import org.apache.tsfile.file.metadata.enums.TSEncoding;
-import org.apache.tsfile.utils.Binary;
-import org.apache.tsfile.utils.BytesUtils;
import org.apache.tsfile.write.record.Tablet;
import org.apache.tsfile.write.schema.IMeasurementSchema;
import org.apache.tsfile.write.schema.MeasurementSchema;
@@ -266,21 +264,13 @@ public class IoTDBSessionSyntaxConventionIT {
schemaList.add(new MeasurementSchema("s4", TSDataType.INT64,
TSEncoding.PLAIN));
Tablet tablet = new Tablet(deviceId, schemaList, 10);
-
- long[] timestamps = tablet.timestamps;
- Object[] values = tablet.values;
-
for (long time = 0; time < 10; time++) {
int row = tablet.getRowSize();
tablet.addTimestamp(row, time);
- long[] sensor = (long[]) values[0];
- sensor[row] = time;
- double[] sensor2 = (double[]) values[1];
- sensor2[row] = 0.1 + time;
- Binary[] sensor3 = (Binary[]) values[2];
- sensor3[row] = BytesUtils.valueOf("ha" + time);
- long[] sensor4 = (long[]) values[3];
- sensor4[row] = time;
+ tablet.addValue(row, 0, time);
+ tablet.addValue(row, 1, 0.1d + time);
+ tablet.addValue(row, 2, "ha" + time);
+ tablet.addValue(row, 3, time);
}
try (ISession session = EnvFactory.getEnv().getSessionConnection()) {
diff --git
a/integration-test/src/test/java/org/apache/iotdb/session/it/SessionIT.java
b/integration-test/src/test/java/org/apache/iotdb/session/it/SessionIT.java
index 92c9d1d0456..3bc7a5984aa 100644
--- a/integration-test/src/test/java/org/apache/iotdb/session/it/SessionIT.java
+++ b/integration-test/src/test/java/org/apache/iotdb/session/it/SessionIT.java
@@ -26,14 +26,12 @@ import org.apache.iotdb.itbase.category.ClusterIT;
import org.apache.iotdb.itbase.category.LocalStandaloneIT;
import org.apache.iotdb.rpc.StatementExecutionException;
-import org.apache.tsfile.common.conf.TSFileConfig;
import org.apache.tsfile.enums.TSDataType;
import org.apache.tsfile.file.metadata.enums.CompressionType;
import org.apache.tsfile.file.metadata.enums.TSEncoding;
import org.apache.tsfile.read.common.Field;
import org.apache.tsfile.read.common.RowRecord;
import org.apache.tsfile.utils.Binary;
-import org.apache.tsfile.utils.BytesUtils;
import org.apache.tsfile.write.record.Tablet;
import org.apache.tsfile.write.schema.IMeasurementSchema;
import org.apache.tsfile.write.schema.MeasurementSchema;
@@ -88,20 +86,13 @@ public class SessionIT {
Tablet tablet = new Tablet("root.sg1.d1", schemaList, 10);
- long[] timestamps = tablet.timestamps;
- Object[] values = tablet.values;
-
for (long time = 0; time < 10; time++) {
int row = tablet.getRowSize();
tablet.addTimestamp(row, time);
- long[] sensor = (long[]) values[0];
- sensor[row] = time;
- double[] sensor2 = (double[]) values[1];
- sensor2[row] = 0.1 + time;
- Binary[] sensor3 = (Binary[]) values[2];
- sensor3[row] = BytesUtils.valueOf("ha" + time);
- long[] sensor4 = (long[]) values[3];
- sensor4[row] = time;
+ tablet.addValue(row, 0, time);
+ tablet.addValue(row, 1, 0.1d + time);
+ tablet.addValue(row, 2, "ha" + time);
+ tablet.addValue(row, 3, time);
}
try {
@@ -279,7 +270,7 @@ public class SessionIT {
byte[] bytes = new byte[2];
bytes[0] = (byte) Integer.parseInt("BA", 16);
bytes[1] = (byte) Integer.parseInt("BE", 16);
- // Method 1 to add tablet data
+
for (long time = 10; time < 15; time++) {
int rowIndex = tablet.getRowSize();
tablet.addTimestamp(rowIndex, time);
@@ -291,17 +282,14 @@ public class SessionIT {
}
session.insertTablet(tablet);
tablet.reset();
- tablet.bitMaps = null;
- // Method 2 to add tablet data
- long[] timestamps = tablet.timestamps;
- Object[] values = tablet.values;
+
for (long time = 15; time < 20; time++) {
int rowIndex = tablet.getRowSize();
tablet.addTimestamp(rowIndex, time);
- ((LocalDate[]) values[0])[rowIndex] = LocalDate.of(2024, 1, (int)
time);
- ((long[]) values[1])[rowIndex] = time;
- ((Binary[]) values[2])[rowIndex] = new Binary(bytes);
- ((Binary[]) values[3])[rowIndex] = new Binary(time + "",
TSFileConfig.STRING_CHARSET);
+ tablet.addValue(rowIndex, 0, LocalDate.of(2024, 1, (int) time));
+ tablet.addValue(rowIndex, 1, time);
+ tablet.addValue(rowIndex, 2, bytes);
+ tablet.addValue(rowIndex, 3, time + "");
}
session.insertTablet(tablet);
tablet.reset();
diff --git
a/integration-test/src/test/java/org/apache/iotdb/subscription/it/local/IoTDBSubscriptionDataTypeIT.java
b/integration-test/src/test/java/org/apache/iotdb/subscription/it/local/IoTDBSubscriptionDataTypeIT.java
index d102b2da081..efa12a88603 100644
---
a/integration-test/src/test/java/org/apache/iotdb/subscription/it/local/IoTDBSubscriptionDataTypeIT.java
+++
b/integration-test/src/test/java/org/apache/iotdb/subscription/it/local/IoTDBSubscriptionDataTypeIT.java
@@ -352,22 +352,16 @@ public class IoTDBSubscriptionDataTypeIT extends
AbstractSubscriptionLocalIT {
private Object getValue(final TSDataType type, final Tablet tablet) {
switch (type) {
case BOOLEAN:
- return ((boolean[]) tablet.values[0])[0];
case INT32:
- return ((int[]) tablet.values[0])[0];
case INT64:
case TIMESTAMP:
- return ((long[]) tablet.values[0])[0];
case FLOAT:
- return ((float[]) tablet.values[0])[0];
case DOUBLE:
- return ((double[]) tablet.values[0])[0];
case TEXT:
case BLOB:
case STRING:
- return ((Binary[]) tablet.values[0])[0];
case DATE:
- return ((LocalDate[]) tablet.values[0])[0];
+ return tablet.getValue(0, 0);
default:
return null;
}
diff --git
a/iotdb-client/cli/src/main/java/org/apache/iotdb/tool/tsfile/ExportTsFile.java
b/iotdb-client/cli/src/main/java/org/apache/iotdb/tool/tsfile/ExportTsFile.java
index 18e8b00ea85..7be26207b16 100644
---
a/iotdb-client/cli/src/main/java/org/apache/iotdb/tool/tsfile/ExportTsFile.java
+++
b/iotdb-client/cli/src/main/java/org/apache/iotdb/tool/tsfile/ExportTsFile.java
@@ -437,9 +437,6 @@ public class ExportTsFile extends AbstractTsFileTool {
IMeasurementSchema measurementSchema = schemas.get(i);
// -1 for time not in fields
Object value = fields.get(columnIndex -
1).getObjectValue(measurementSchema.getType());
- if (value == null) {
- tablet.bitMaps[i].mark(rowIndex);
- }
tablet.addValue(measurementSchema.getMeasurementName(), rowIndex,
value);
}
diff --git
a/iotdb-client/session/src/main/java/org/apache/iotdb/session/Session.java
b/iotdb-client/session/src/main/java/org/apache/iotdb/session/Session.java
index 3fbdb4214ff..1e65be39082 100644
--- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/Session.java
+++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/Session.java
@@ -2787,7 +2787,7 @@ public class Session implements ISession {
v.getRowSize(),
tablet.getValue(finalI, j));
}
- v.addTimestamp(v.getRowSize(), tablet.timestamps[finalI]);
+ v.addTimestamp(v.getRowSize(), tablet.getTimestamp(finalI));
return v;
});
}
@@ -3498,7 +3498,7 @@ public class Session implements ISession {
*/
private boolean checkSorted(Tablet tablet) {
for (int i = 1; i < tablet.getRowSize(); i++) {
- if (tablet.timestamps[i] < tablet.timestamps[i - 1]) {
+ if (tablet.getTimestamp(i) < tablet.getTimestamp(i - 1)) {
return false;
}
}
@@ -3523,31 +3523,32 @@ public class Session implements ISession {
* so we can insert continuous data in value list to get a better
performance
*/
// sort to get index, and use index to sort value list
+ long[] timestamps = tablet.getTimestamps();
+ Object[] values = tablet.getValues();
+ BitMap[] bitMaps = tablet.getBitMaps();
Integer[] index = new Integer[tablet.getRowSize()];
for (int i = 0; i < tablet.getRowSize(); i++) {
index[i] = i;
}
- Arrays.sort(index, Comparator.comparingLong(o -> tablet.timestamps[o]));
- Arrays.sort(tablet.timestamps, 0, tablet.getRowSize());
+ Arrays.sort(index, Comparator.comparingLong(o -> timestamps[o]));
+ Arrays.sort(timestamps, 0, tablet.getRowSize());
int columnIndex = 0;
for (int i = 0; i < tablet.getSchemas().size(); i++) {
IMeasurementSchema schema = tablet.getSchemas().get(i);
if (schema instanceof MeasurementSchema) {
- tablet.values[columnIndex] = sortList(tablet.values[columnIndex],
schema.getType(), index);
- if (tablet.bitMaps != null && tablet.bitMaps[columnIndex] != null) {
- tablet.bitMaps[columnIndex] =
sortBitMap(tablet.bitMaps[columnIndex], index);
+ values[columnIndex] = sortList(values[columnIndex], schema.getType(),
index);
+ if (bitMaps != null && bitMaps[columnIndex] != null) {
+ bitMaps[columnIndex] = sortBitMap(bitMaps[columnIndex], index);
}
columnIndex++;
} else {
int measurementSize = schema.getSubMeasurementsList().size();
for (int j = 0; j < measurementSize; j++) {
- tablet.values[columnIndex] =
+ values[columnIndex] =
sortList(
- tablet.values[columnIndex],
- schema.getSubMeasurementsTSDataTypeList().get(j),
- index);
- if (tablet.bitMaps != null && tablet.bitMaps[columnIndex] != null) {
- tablet.bitMaps[columnIndex] =
sortBitMap(tablet.bitMaps[columnIndex], index);
+ values[columnIndex],
schema.getSubMeasurementsTSDataTypeList().get(j), index);
+ if (bitMaps != null && bitMaps[columnIndex] != null) {
+ bitMaps[columnIndex] = sortBitMap(bitMaps[columnIndex], index);
}
columnIndex++;
}
diff --git
a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/payload/SubscriptionSessionDataSet.java
b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/payload/SubscriptionSessionDataSet.java
index 3e2c4e01f42..713ab2aec6f 100644
---
a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/payload/SubscriptionSessionDataSet.java
+++
b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/payload/SubscriptionSessionDataSet.java
@@ -25,6 +25,7 @@ import org.apache.tsfile.enums.TSDataType;
import org.apache.tsfile.read.common.Field;
import org.apache.tsfile.read.common.RowRecord;
import org.apache.tsfile.utils.Binary;
+import org.apache.tsfile.utils.BitMap;
import org.apache.tsfile.utils.DateUtils;
import org.apache.tsfile.write.UnSupportedDataTypeException;
import org.apache.tsfile.write.record.Tablet;
@@ -103,15 +104,16 @@ public class SubscriptionSessionDataSet implements
ISessionDataSet {
final long timestamp = entry.getKey();
final int rowIndex = entry.getValue();
+ BitMap[] bitMaps = tablet.getBitMaps();
for (int columnIndex = 0; columnIndex < columnSize; ++columnIndex) {
final Field field;
- if (tablet.bitMaps != null
- && tablet.bitMaps[columnIndex] != null
- && tablet.bitMaps[columnIndex].isMarked(rowIndex)) {
+ if (bitMaps != null
+ && bitMaps[columnIndex] != null
+ && bitMaps[columnIndex].isMarked(rowIndex)) {
field = new Field(null);
} else {
final TSDataType dataType =
tablet.getSchemas().get(columnIndex).getType();
- field = generateFieldFromTabletValue(dataType,
tablet.values[columnIndex], rowIndex);
+ field = generateFieldFromTabletValue(dataType,
tablet.getValues()[columnIndex], rowIndex);
}
fields.add(field);
}
@@ -134,7 +136,7 @@ public class SubscriptionSessionDataSet implements
ISessionDataSet {
private void generateRowIterator() {
// timestamp -> row index
- final long[] timestamps = tablet.timestamps;
+ final long[] timestamps = tablet.getTimestamps();
final TreeMap<Long, Integer> timestampToRowIndex = new TreeMap<>();
final int rowSize = timestamps.length;
for (int rowIndex = 0; rowIndex < rowSize; ++rowIndex) {
diff --git
a/iotdb-client/session/src/main/java/org/apache/iotdb/session/util/SessionUtils.java
b/iotdb-client/session/src/main/java/org/apache/iotdb/session/util/SessionUtils.java
index 0ec1eb1621a..8534b808754 100644
---
a/iotdb-client/session/src/main/java/org/apache/iotdb/session/util/SessionUtils.java
+++
b/iotdb-client/session/src/main/java/org/apache/iotdb/session/util/SessionUtils.java
@@ -50,7 +50,7 @@ public class SessionUtils {
public static ByteBuffer getTimeBuffer(Tablet tablet) {
ByteBuffer timeBuffer = ByteBuffer.allocate(getTimeBytesSize(tablet));
for (int i = 0; i < tablet.getRowSize(); i++) {
- timeBuffer.putLong(tablet.timestamps[i]);
+ timeBuffer.putLong(tablet.getTimestamp(i));
}
timeBuffer.flip();
return timeBuffer;
@@ -63,8 +63,9 @@ public class SessionUtils {
IMeasurementSchema schema = tablet.getSchemas().get(i);
getValueBufferOfDataType(schema.getType(), tablet, i, valueBuffer);
}
- if (tablet.bitMaps != null) {
- for (BitMap bitMap : tablet.bitMaps) {
+ BitMap[] bitMaps = tablet.getBitMaps();
+ if (bitMaps != null) {
+ for (BitMap bitMap : bitMaps) {
boolean columnHasNull = bitMap != null &&
!bitMap.isAllUnmarked(tablet.getRowSize());
valueBuffer.put(BytesUtils.boolToByte(columnHasNull));
if (columnHasNull) {
@@ -90,12 +91,12 @@ public class SessionUtils {
int rowSize = tablet.getRowSize();
for (IMeasurementSchema schema : schemas) {
valueOccupation +=
- calOccupationOfOneColumn(schema.getType(), tablet.values,
columnIndex, rowSize);
+ calOccupationOfOneColumn(schema.getType(), tablet.getValues(),
columnIndex, rowSize);
columnIndex++;
}
// Add bitmap size if the tablet has bitMaps
- BitMap[] bitMaps = tablet.bitMaps;
+ BitMap[] bitMaps = tablet.getBitMaps();
if (bitMaps != null) {
for (BitMap bitMap : bitMaps) {
// Marker byte
@@ -265,7 +266,7 @@ public class SessionUtils {
switch (dataType) {
case INT32:
- int[] intValues = (int[]) tablet.values[i];
+ int[] intValues = (int[]) tablet.getValues()[i];
for (int index = 0; index < tablet.getRowSize(); index++) {
if (!tablet.isNull(index, i)) {
valueBuffer.putInt(intValues[index]);
@@ -276,7 +277,7 @@ public class SessionUtils {
break;
case INT64:
case TIMESTAMP:
- long[] longValues = (long[]) tablet.values[i];
+ long[] longValues = (long[]) tablet.getValues()[i];
for (int index = 0; index < tablet.getRowSize(); index++) {
if (!tablet.isNull(index, i)) {
valueBuffer.putLong(longValues[index]);
@@ -286,7 +287,7 @@ public class SessionUtils {
}
break;
case FLOAT:
- float[] floatValues = (float[]) tablet.values[i];
+ float[] floatValues = (float[]) tablet.getValues()[i];
for (int index = 0; index < tablet.getRowSize(); index++) {
if (!tablet.isNull(index, i)) {
valueBuffer.putFloat(floatValues[index]);
@@ -296,7 +297,7 @@ public class SessionUtils {
}
break;
case DOUBLE:
- double[] doubleValues = (double[]) tablet.values[i];
+ double[] doubleValues = (double[]) tablet.getValues()[i];
for (int index = 0; index < tablet.getRowSize(); index++) {
if (!tablet.isNull(index, i)) {
valueBuffer.putDouble(doubleValues[index]);
@@ -306,7 +307,7 @@ public class SessionUtils {
}
break;
case BOOLEAN:
- boolean[] boolValues = (boolean[]) tablet.values[i];
+ boolean[] boolValues = (boolean[]) tablet.getValues()[i];
for (int index = 0; index < tablet.getRowSize(); index++) {
if (!tablet.isNull(index, i)) {
valueBuffer.put(BytesUtils.boolToByte(boolValues[index]));
@@ -318,7 +319,7 @@ public class SessionUtils {
case TEXT:
case STRING:
case BLOB:
- Binary[] binaryValues = (Binary[]) tablet.values[i];
+ Binary[] binaryValues = (Binary[]) tablet.getValues()[i];
for (int index = 0; index < tablet.getRowSize(); index++) {
if (!tablet.isNull(index, i) && binaryValues[index] != null) {
valueBuffer.putInt(binaryValues[index].getLength());
@@ -330,7 +331,7 @@ public class SessionUtils {
}
break;
case DATE:
- LocalDate[] dateValues = (LocalDate[]) tablet.values[i];
+ LocalDate[] dateValues = (LocalDate[]) tablet.getValues()[i];
for (int index = 0; index < tablet.getRowSize(); index++) {
if (!tablet.isNull(index, i) && dateValues[index] != null) {
valueBuffer.putInt(DateUtils.parseDateExpressionToInt(dateValues[index]));
diff --git
a/iotdb-client/session/src/test/java/org/apache/iotdb/session/TabletTest.java
b/iotdb-client/session/src/test/java/org/apache/iotdb/session/TabletTest.java
index a936393e9aa..bc9469e57a4 100644
---
a/iotdb-client/session/src/test/java/org/apache/iotdb/session/TabletTest.java
+++
b/iotdb-client/session/src/test/java/org/apache/iotdb/session/TabletTest.java
@@ -51,8 +51,7 @@ public class TabletTest {
;
// insert three rows data
Tablet tablet = new Tablet("root.sg1.d1", schemaList, 3);
- long[] timestamps = tablet.timestamps;
- Object[] values = tablet.values;
+ Object[] values = tablet.getValues();
/*
inorder data before inserting
@@ -62,9 +61,8 @@ public class TabletTest {
1 2
*/
// inorder timestamps
- timestamps[0] = 2;
- timestamps[1] = 0;
- timestamps[2] = 1;
+ tablet.setTimestamps(new long[] {2, 0, 1});
+
values[0] = new long[] {0, 1, 2};
values[1] = new long[] {0, 1, 2};
values[2] = new int[] {0, 1, 2};
@@ -105,37 +103,37 @@ public class TabletTest {
If the data equal to above tablet, test pass, otherwise test fialed
*/
- long[] resTimestamps = tablet.timestamps;
+ long[] resTimestamps = tablet.getTimestamps();
long[] expectedTimestamps = new long[] {0, 1, 2};
assertArrayEquals(expectedTimestamps, resTimestamps);
- assertArrayEquals(new long[] {1, 2, 0}, ((long[]) tablet.values[0]));
- assertArrayEquals(new long[] {1, 2, 0}, ((long[]) tablet.values[1]));
- assertArrayEquals(new int[] {1, 2, 0}, ((int[]) tablet.values[2]));
+ assertArrayEquals(new long[] {1, 2, 0}, ((long[]) tablet.getValues()[0]));
+ assertArrayEquals(new long[] {1, 2, 0}, ((long[]) tablet.getValues()[1]));
+ assertArrayEquals(new int[] {1, 2, 0}, ((int[]) tablet.getValues()[2]));
assertArrayEquals(
new LocalDate[] {LocalDate.ofEpochDay(1), LocalDate.ofEpochDay(2),
LocalDate.ofEpochDay(0)},
- ((LocalDate[]) tablet.values[3]));
- assertArrayEquals(new boolean[] {false, true, true}, ((boolean[])
tablet.values[4]));
- assertArrayEquals(new double[] {1.0, 2.0, 0.0}, ((double[])
tablet.values[5]), 0.001);
+ ((LocalDate[]) tablet.getValues()[3]));
+ assertArrayEquals(new boolean[] {false, true, true}, ((boolean[])
tablet.getValues()[4]));
+ assertArrayEquals(new double[] {1.0, 2.0, 0.0}, ((double[])
tablet.getValues()[5]), 0.001);
assertArrayEquals(
new Binary[] {
new Binary("1".getBytes(StandardCharsets.UTF_8)),
new Binary("2".getBytes(StandardCharsets.UTF_8)),
new Binary("0".getBytes(StandardCharsets.UTF_8))
},
- ((Binary[]) tablet.values[6]));
+ ((Binary[]) tablet.getValues()[6]));
assertArrayEquals(
new Binary[] {
new Binary("1".getBytes(StandardCharsets.UTF_8)),
new Binary("2".getBytes(StandardCharsets.UTF_8)),
new Binary("0".getBytes(StandardCharsets.UTF_8))
},
- ((Binary[]) tablet.values[7]));
+ ((Binary[]) tablet.getValues()[7]));
assertArrayEquals(
new Binary[] {
new Binary("1".getBytes(StandardCharsets.UTF_8)),
new Binary("2".getBytes(StandardCharsets.UTF_8)),
new Binary("0".getBytes(StandardCharsets.UTF_8))
},
- ((Binary[]) tablet.values[8]));
+ ((Binary[]) tablet.getValues()[8]));
}
}
diff --git
a/iotdb-client/session/src/test/java/org/apache/iotdb/session/util/SessionUtilsTest.java
b/iotdb-client/session/src/test/java/org/apache/iotdb/session/util/SessionUtilsTest.java
index 25b59b07474..5818a89625e 100644
---
a/iotdb-client/session/src/test/java/org/apache/iotdb/session/util/SessionUtilsTest.java
+++
b/iotdb-client/session/src/test/java/org/apache/iotdb/session/util/SessionUtilsTest.java
@@ -190,14 +190,14 @@ public class SessionUtilsTest {
schemas.add(schema);
Tablet tablet = new Tablet("device1", schemas, 2);
- tablet.timestamps = new long[] {1L};
- tablet.values[0] = new int[] {1, 2};
- tablet.values[1] = new long[] {1L, 2L};
- tablet.values[2] = new float[] {1.1f, 1.2f};
- tablet.values[3] = new double[] {0.707, 0.708};
- tablet.values[4] = new Binary[] {null, new Binary(new byte[] {(byte) 16})};
- tablet.values[5] = new boolean[] {true, false};
- tablet.values[6] = new LocalDate[] {null, LocalDate.of(2024, 4, 1)};
+ tablet.setTimestamps(new long[] {1L});
+ tablet.getValues()[0] = new int[] {1, 2};
+ tablet.getValues()[1] = new long[] {1L, 2L};
+ tablet.getValues()[2] = new float[] {1.1f, 1.2f};
+ tablet.getValues()[3] = new double[] {0.707, 0.708};
+ tablet.getValues()[4] = new Binary[] {null, new Binary(new byte[] {(byte)
16})};
+ tablet.getValues()[5] = new boolean[] {true, false};
+ tablet.getValues()[6] = new LocalDate[] {null, LocalDate.of(2024, 4, 1)};
tablet.setRowSize(tablet.getRowSize() + 2);
ByteBuffer timeBuffer = SessionUtils.getValueBuffer(tablet);
diff --git
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/opcua/OpcUaNameSpace.java
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/opcua/OpcUaNameSpace.java
index 834a14f94df..db279cb8cd4 100644
---
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/opcua/OpcUaNameSpace.java
+++
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/opcua/OpcUaNameSpace.java
@@ -124,12 +124,13 @@ public class OpcUaNameSpace extends
ManagedNamespaceWithLifecycle {
for (int i = 0; i < schemas.size(); ++i) {
for (int j = 0; j < tablet.getRowSize(); ++j) {
- if (Objects.isNull(tablet.bitMaps)
- || Objects.isNull(tablet.bitMaps[i])
- || !tablet.bitMaps[i].isMarked(j)) {
+ if (Objects.isNull(tablet.getBitMaps())
+ || Objects.isNull(tablet.getBitMaps()[i])
+ || !tablet.getBitMaps()[i].isMarked(j)) {
newSchemas.add(schemas.get(i));
- timestamps.add(tablet.timestamps[j]);
- values.add(getTabletObjectValue4Opc(tablet.values[i], j,
schemas.get(i).getType()));
+ timestamps.add(tablet.getTimestamp(j));
+ values.add(
+ getTabletObjectValue4Opc(tablet.getValues()[i], j,
schemas.get(i).getType()));
break;
}
}
@@ -161,12 +162,12 @@ public class OpcUaNameSpace extends
ManagedNamespaceWithLifecycle {
transferTabletRowForClientServerModel(
folderSegments,
newSchemas,
- Collections.singletonList(tablet.timestamps[i]),
+ Collections.singletonList(tablet.getTimestamp(i)),
columnIndexes.stream()
.map(
index ->
getTabletObjectValue4Opc(
- tablet.values[index], finalI,
schemas.get(index).getType()))
+ tablet.getValues()[index], finalI,
schemas.get(index).getType()))
.collect(Collectors.toList()));
}
}
@@ -357,7 +358,7 @@ public class OpcUaNameSpace extends
ManagedNamespaceWithLifecycle {
for (int rowIndex = 0; rowIndex < tablet.getRowSize(); ++rowIndex) {
// Filter null value
- if (tablet.bitMaps[columnIndex].isMarked(rowIndex)) {
+ if (tablet.getBitMaps()[columnIndex].isMarked(rowIndex)) {
continue;
}
@@ -373,54 +374,53 @@ public class OpcUaNameSpace extends
ManagedNamespaceWithLifecycle {
}
// Time --> TimeStamp
- eventNode.setTime(new
DateTime(timestampToUtc(tablet.timestamps[rowIndex])));
+ eventNode.setTime(new
DateTime(timestampToUtc(tablet.getTimestamp(rowIndex))));
// Message --> Value
switch (dataType) {
case BOOLEAN:
eventNode.setMessage(
LocalizedText.english(
- Boolean.toString(((boolean[])
tablet.values[columnIndex])[rowIndex])));
+ Boolean.toString((boolean) tablet.getValue(rowIndex,
columnIndex))));
break;
case INT32:
eventNode.setMessage(
LocalizedText.english(
- Integer.toString(((int[])
tablet.values[columnIndex])[rowIndex])));
+ Integer.toString((int) tablet.getValue(rowIndex,
columnIndex))));
break;
case DATE:
eventNode.setMessage(
LocalizedText.english(
- (((LocalDate[]) tablet.values[columnIndex])[rowIndex])
+ ((LocalDate) tablet.getValue(rowIndex, columnIndex))
.atStartOfDay(ZoneId.systemDefault())
.toString()));
break;
case INT64:
eventNode.setMessage(
LocalizedText.english(
- Long.toString(((long[])
tablet.values[columnIndex])[rowIndex])));
+ Long.toString((long) tablet.getValue(rowIndex,
columnIndex))));
break;
case TIMESTAMP:
eventNode.setMessage(
LocalizedText.english(
DateTimeUtils.convertLongToDate(
- ((long[]) tablet.values[columnIndex])[rowIndex])));
+ (long) tablet.getValue(rowIndex, columnIndex))));
break;
case FLOAT:
eventNode.setMessage(
LocalizedText.english(
- Float.toString(((float[])
tablet.values[columnIndex])[rowIndex])));
+ Float.toString((float) tablet.getValue(rowIndex,
columnIndex))));
break;
case DOUBLE:
eventNode.setMessage(
LocalizedText.english(
- Double.toString(((double[])
tablet.values[columnIndex])[rowIndex])));
+ Double.toString((double) tablet.getValue(rowIndex,
columnIndex))));
break;
case TEXT:
case BLOB:
case STRING:
eventNode.setMessage(
- LocalizedText.english(
- ((Binary[])
tablet.values[columnIndex])[rowIndex].toString()));
+ LocalizedText.english(tablet.getValue(rowIndex,
columnIndex).toString()));
break;
case VECTOR:
case UNKNOWN:
diff --git
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/builder/PipeTableModeTsFileBuilder.java
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/builder/PipeTableModeTsFileBuilder.java
index 6ac5ac911bf..13534a22b4d 100644
---
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/builder/PipeTableModeTsFileBuilder.java
+++
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/builder/PipeTableModeTsFileBuilder.java
@@ -123,7 +123,7 @@ public class PipeTableModeTsFileBuilder extends
PipeTsFileBuilder {
final IDeviceID deviceID = o1.right.get(0).left;
final int result;
if ((result = deviceID.compareTo(o2.right.get(0).left)) == 0) {
- return Long.compare(o1.left.timestamps[0],
o2.left.timestamps[0]);
+ return Long.compare(o1.left.getTimestamp(0),
o2.left.getTimestamp(0));
}
return result;
});
@@ -260,12 +260,12 @@ public class PipeTableModeTsFileBuilder extends
PipeTsFileBuilder {
for (Pair<IDeviceID, Integer> deviceTimestampIndexPair : tabletPair.right)
{
final Long lastDeviceTimestamp =
deviceLastTimestampMap.get(deviceTimestampIndexPair.left);
if (lastDeviceTimestamp != null
- && lastDeviceTimestamp >=
tabletPair.left.timestamps[currentTimestampIndex]) {
+ && lastDeviceTimestamp >=
tabletPair.left.getTimestamp(currentTimestampIndex)) {
return false;
}
currentTimestampIndex = deviceTimestampIndexPair.right;
deviceLastTimestampMap.put(
- deviceTimestampIndexPair.left,
tabletPair.left.timestamps[currentTimestampIndex - 1]);
+ deviceTimestampIndexPair.left,
tabletPair.left.getTimestamp(currentTimestampIndex - 1));
}
return true;
diff --git
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/builder/PipeTreeModelTsFileBuilder.java
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/builder/PipeTreeModelTsFileBuilder.java
index 8d30dd0172f..d39dd0141ae 100644
---
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/builder/PipeTreeModelTsFileBuilder.java
+++
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/builder/PipeTreeModelTsFileBuilder.java
@@ -108,7 +108,7 @@ public class PipeTreeModelTsFileBuilder extends
PipeTsFileBuilder {
for (final List<Tablet> tablets : device2Tablets.values()) {
tablets.sort(
// Each tablet has at least one timestamp
- Comparator.comparingLong(tablet -> tablet.timestamps[0]));
+ Comparator.comparingLong(tablet -> tablet.getTimestamp(0)));
}
// Sort the devices by device id
@@ -213,7 +213,7 @@ public class PipeTreeModelTsFileBuilder extends
PipeTsFileBuilder {
final Tablet tablet = tablets.peekFirst();
if (Objects.isNull(lastTablet)
// lastTablet.rowSize is not 0
- || lastTablet.timestamps[lastTablet.getRowSize() - 1] <
tablet.timestamps[0]) {
+ || lastTablet.getTimestamp(lastTablet.getRowSize() - 1) <
tablet.getTimestamp(0)) {
tabletsToWrite.add(tablet);
lastTablet = tablet;
tablets.pollFirst();
diff --git
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/sorter/PipeTableModelTabletEventSorter.java
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/sorter/PipeTableModelTabletEventSorter.java
index 63ac5b847a1..4c618121f5f 100644
---
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/sorter/PipeTableModelTabletEventSorter.java
+++
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/sorter/PipeTableModelTabletEventSorter.java
@@ -61,10 +61,10 @@ public class PipeTableModelTabletEventSorter {
}
HashMap<IDeviceID, List<Pair<Integer, Integer>>> deviceIDToIndexMap = new
HashMap<>();
- final long[] timestamps = tablet.timestamps;
+ final long[] timestamps = tablet.getTimestamps();
IDeviceID lastDevice = tablet.getDeviceID(0);
- long previousTimestamp = tablet.timestamps[0];
+ long previousTimestamp = tablet.getTimestamp(0);
int lasIndex = 0;
for (int i = 1, size = tablet.getRowSize(); i < size; ++i) {
final IDeviceID deviceID = tablet.getDeviceID(i);
@@ -140,20 +140,20 @@ public class PipeTableModelTabletEventSorter {
private void sortAndDeduplicateValuesAndBitMaps() {
int columnIndex = 0;
- tablet.timestamps =
+ tablet.setTimestamps(
(long[])
PipeTabletEventSorter.reorderValueList(
- deduplicatedSize, tablet.timestamps, TSDataType.TIMESTAMP,
index);
+ deduplicatedSize, tablet.getTimestamps(),
TSDataType.TIMESTAMP, index));
for (int i = 0, size = tablet.getSchemas().size(); i < size; i++) {
final IMeasurementSchema schema = tablet.getSchemas().get(i);
if (schema != null) {
- tablet.values[columnIndex] =
+ tablet.getValues()[columnIndex] =
PipeTabletEventSorter.reorderValueList(
- deduplicatedSize, tablet.values[columnIndex],
schema.getType(), index);
- if (tablet.bitMaps != null && tablet.bitMaps[columnIndex] != null) {
- tablet.bitMaps[columnIndex] =
+ deduplicatedSize, tablet.getValues()[columnIndex],
schema.getType(), index);
+ if (tablet.getBitMaps() != null && tablet.getBitMaps()[columnIndex] !=
null) {
+ tablet.getBitMaps()[columnIndex] =
PipeTabletEventSorter.reorderBitMap(
- deduplicatedSize, tablet.bitMaps[columnIndex], index);
+ deduplicatedSize, tablet.getBitMaps()[columnIndex], index);
}
columnIndex++;
}
@@ -163,15 +163,15 @@ public class PipeTableModelTabletEventSorter {
}
private void sortTimestamps(final int startIndex, final int endIndex) {
- Arrays.sort(
- this.index, startIndex, endIndex, Comparator.comparingLong(i ->
tablet.timestamps[i]));
+ Arrays.sort(this.index, startIndex, endIndex,
Comparator.comparingLong(tablet::getTimestamp));
}
private void deduplicateTimestamps(final int startIndex, final int endIndex)
{
- long lastTime = tablet.timestamps[index[startIndex]];
+ long[] timestamps = tablet.getTimestamps();
+ long lastTime = timestamps[index[startIndex]];
index[deduplicatedSize++] = index[startIndex];
for (int i = startIndex + 1; i < endIndex; i++) {
- if (lastTime != (lastTime = tablet.timestamps[index[i]])) {
+ if (lastTime != (lastTime = timestamps[index[i]])) {
index[deduplicatedSize++] = index[i];
}
}
@@ -183,9 +183,10 @@ public class PipeTableModelTabletEventSorter {
return;
}
+ long[] timestamps = tablet.getTimestamps();
for (int i = 1, size = tablet.getRowSize(); i < size; ++i) {
- final long currentTimestamp = tablet.timestamps[i];
- final long previousTimestamp = tablet.timestamps[i - 1];
+ final long currentTimestamp = timestamps[i];
+ final long previousTimestamp = timestamps[i - 1];
if (currentTimestamp < previousTimestamp) {
isUnSorted = true;
@@ -222,26 +223,27 @@ public class PipeTableModelTabletEventSorter {
}
private void sortTimestamps() {
- Arrays.sort(this.index, Comparator.comparingLong(i ->
tablet.timestamps[i]));
- Arrays.sort(tablet.timestamps, 0, tablet.getRowSize());
+ Arrays.sort(this.index, Comparator.comparingLong(tablet::getTimestamp));
+ Arrays.sort(tablet.getTimestamps(), 0, tablet.getRowSize());
}
private void deduplicateTimestamps() {
deduplicatedSize = 1;
- long lastTime = tablet.timestamps[0];
+ long[] timestamps = tablet.getTimestamps();
+ long lastTime = timestamps[0];
IDeviceID deviceID = tablet.getDeviceID(index[0]);
final Set<IDeviceID> deviceIDSet = new HashSet<>();
deviceIDSet.add(deviceID);
for (int i = 1, size = tablet.getRowSize(); i < size; i++) {
deviceID = tablet.getDeviceID(index[i]);
- if ((lastTime == (lastTime = tablet.timestamps[i]))) {
+ if ((lastTime == (lastTime = timestamps[i]))) {
if (!deviceIDSet.contains(deviceID)) {
- tablet.timestamps[deduplicatedSize] = lastTime;
+ timestamps[deduplicatedSize] = lastTime;
index[deduplicatedSize++] = index[i];
deviceIDSet.add(deviceID);
}
} else {
- tablet.timestamps[deduplicatedSize] = lastTime;
+ timestamps[deduplicatedSize] = lastTime;
index[deduplicatedSize++] = index[i];
deviceIDSet.clear();
deviceIDSet.add(deviceID);
@@ -254,13 +256,13 @@ public class PipeTableModelTabletEventSorter {
for (int i = 0, size = tablet.getSchemas().size(); i < size; i++) {
final IMeasurementSchema schema = tablet.getSchemas().get(i);
if (schema != null) {
- tablet.values[columnIndex] =
+ tablet.getValues()[columnIndex] =
PipeTabletEventSorter.reorderValueList(
- deduplicatedSize, tablet.values[columnIndex],
schema.getType(), index);
- if (tablet.bitMaps != null && tablet.bitMaps[columnIndex] != null) {
- tablet.bitMaps[columnIndex] =
+ deduplicatedSize, tablet.getValues()[columnIndex],
schema.getType(), index);
+ if (tablet.getBitMaps() != null && tablet.getBitMaps()[columnIndex] !=
null) {
+ tablet.getBitMaps()[columnIndex] =
PipeTabletEventSorter.reorderBitMap(
- deduplicatedSize, tablet.bitMaps[columnIndex], index);
+ deduplicatedSize, tablet.getBitMaps()[columnIndex], index);
}
columnIndex++;
}
diff --git
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/sorter/PipeTreeModelTabletEventSorter.java
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/sorter/PipeTreeModelTabletEventSorter.java
index 42f8c2dac52..cbde0583639 100644
---
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/sorter/PipeTreeModelTabletEventSorter.java
+++
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/util/sorter/PipeTreeModelTabletEventSorter.java
@@ -45,9 +45,10 @@ public class PipeTreeModelTabletEventSorter {
return;
}
+ long[] timestamps = tablet.getTimestamps();
for (int i = 1, size = tablet.getRowSize(); i < size; ++i) {
- final long currentTimestamp = tablet.timestamps[i];
- final long previousTimestamp = tablet.timestamps[i - 1];
+ final long currentTimestamp = timestamps[i];
+ final long previousTimestamp = timestamps[i - 1];
if (currentTimestamp < previousTimestamp) {
isSorted = false;
@@ -84,16 +85,17 @@ public class PipeTreeModelTabletEventSorter {
}
private void sortTimestamps() {
- Arrays.sort(index, Comparator.comparingLong(i -> tablet.timestamps[i]));
- Arrays.sort(tablet.timestamps, 0, tablet.getRowSize());
+ Arrays.sort(index, Comparator.comparingLong(tablet::getTimestamp));
+ Arrays.sort(tablet.getTimestamps(), 0, tablet.getRowSize());
}
private void deduplicateTimestamps() {
deduplicatedSize = 1;
+ long[] timestamps = tablet.getTimestamps();
for (int i = 1, size = tablet.getRowSize(); i < size; i++) {
- if (tablet.timestamps[i] != tablet.timestamps[i - 1]) {
+ if (timestamps[i] != timestamps[i - 1]) {
index[deduplicatedSize] = index[i];
- tablet.timestamps[deduplicatedSize] = tablet.timestamps[i];
+ timestamps[deduplicatedSize] = timestamps[i];
++deduplicatedSize;
}
@@ -106,13 +108,13 @@ public class PipeTreeModelTabletEventSorter {
for (int i = 0, size = tablet.getSchemas().size(); i < size; i++) {
final IMeasurementSchema schema = tablet.getSchemas().get(i);
if (schema != null) {
- tablet.values[columnIndex] =
+ tablet.getValues()[columnIndex] =
PipeTabletEventSorter.reorderValueList(
- deduplicatedSize, tablet.values[columnIndex],
schema.getType(), index);
- if (tablet.bitMaps != null && tablet.bitMaps[columnIndex] != null) {
- tablet.bitMaps[columnIndex] =
+ deduplicatedSize, tablet.getValues()[columnIndex],
schema.getType(), index);
+ if (tablet.getBitMaps() != null && tablet.getBitMaps()[columnIndex] !=
null) {
+ tablet.getBitMaps()[columnIndex] =
PipeTabletEventSorter.reorderBitMap(
- deduplicatedSize, tablet.bitMaps[columnIndex], index);
+ deduplicatedSize, tablet.getBitMaps()[columnIndex], index);
}
columnIndex++;
}
diff --git
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/row/PipeRowCollector.java
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/row/PipeRowCollector.java
index 8a1fca02fc7..540e23ac56e 100644
---
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/row/PipeRowCollector.java
+++
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/row/PipeRowCollector.java
@@ -90,7 +90,7 @@ public class PipeRowCollector implements RowCollector {
tablet.addValue(measurementSchemaArray[i].getMeasurementName(),
rowIndex, value);
}
if (row.isNull(i)) {
- tablet.bitMaps[i].mark(rowIndex);
+ tablet.getBitMaps()[i].mark(rowIndex);
}
}
diff --git
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tablet/PipeRawTabletInsertionEvent.java
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tablet/PipeRawTabletInsertionEvent.java
index b2eae812987..21e440c63d4 100644
---
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tablet/PipeRawTabletInsertionEvent.java
+++
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tablet/PipeRawTabletInsertionEvent.java
@@ -261,7 +261,7 @@ public class PipeRawTabletInsertionEvent extends
PipeInsertionEvent
@Override
public boolean mayEventTimeOverlappedWithTimeRange() {
- final long[] timestamps = tablet.timestamps;
+ final long[] timestamps = tablet.getTimestamps();
if (Objects.isNull(timestamps) || timestamps.length == 0) {
return false;
}
diff --git
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tablet/parser/TabletInsertionEventParser.java
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tablet/parser/TabletInsertionEventParser.java
index d6011b054f1..13c7c0e5c33 100644
---
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tablet/parser/TabletInsertionEventParser.java
+++
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tablet/parser/TabletInsertionEventParser.java
@@ -295,7 +295,7 @@ public abstract class TabletInsertionEventParser {
final long[] originTimestampColumn =
Arrays.copyOf(
- tablet.timestamps,
+ tablet.getTimestamps(),
tablet.getRowSize()); // tablet.timestamps.length ==
tablet.maxRowNumber
final List<Integer> rowIndexList =
generateRowIndexList(originTimestampColumn);
this.timestampColumn = rowIndexList.stream().mapToLong(i ->
originTimestampColumn[i]).toArray();
@@ -334,14 +334,14 @@ public abstract class TabletInsertionEventParser {
originValueColumnDataTypes[i] =
originMeasurementSchemaList.get(i).getType();
}
final Object[] originValueColumns =
- tablet.values; // we do not reduce value columns here by origin row
size
+ tablet.getValues(); // we do not reduce value columns here by origin
row size
final BitMap[] originBitMapList =
- tablet.bitMaps == null
+ tablet.getBitMaps() == null
? IntStream.range(0, originColumnSize)
.boxed()
.map(o -> new BitMap(tablet.getMaxRowNumber()))
.toArray(BitMap[]::new)
- : tablet.bitMaps; // We do not reduce bitmaps here by origin row
size
+ : tablet.getBitMaps(); // We do not reduce bitmaps here by origin
row size
for (int i = 0; i < originBitMapList.length; i++) {
if (originBitMapList[i] == null) {
originBitMapList[i] = new BitMap(tablet.getMaxRowNumber());
diff --git
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tablet/parser/TabletInsertionEventTreePatternParser.java
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tablet/parser/TabletInsertionEventTreePatternParser.java
index 3c314a39e5d..68fb0e50b95 100644
---
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tablet/parser/TabletInsertionEventTreePatternParser.java
+++
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tablet/parser/TabletInsertionEventTreePatternParser.java
@@ -161,19 +161,14 @@ public class TabletInsertionEventTreePatternParser
extends TabletInsertionEventP
if (tablet != null) {
return tablet;
}
-
- final Tablet newTablet =
+ tablet =
new Tablet(
Objects.nonNull(deviceIdString) ? deviceIdString :
deviceId.toString(),
Arrays.asList(measurementSchemaList),
+ timestampColumn,
+ valueColumns,
+ nullValueColumnBitmaps,
rowCount);
- newTablet.timestamps = timestampColumn;
- newTablet.bitMaps = nullValueColumnBitmaps;
- newTablet.values = valueColumns;
- newTablet.setRowSize(rowCount);
-
- tablet = newTablet;
-
return tablet;
}
}
diff --git
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/parser/scan/TsFileInsertionEventScanParser.java
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/parser/scan/TsFileInsertionEventScanParser.java
index 9534825dd93..a324e1eff96 100644
---
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/parser/scan/TsFileInsertionEventScanParser.java
+++
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/parser/scan/TsFileInsertionEventScanParser.java
@@ -42,7 +42,6 @@ import org.apache.tsfile.read.filter.basic.Filter;
import org.apache.tsfile.read.reader.IChunkReader;
import org.apache.tsfile.read.reader.chunk.AlignedChunkReader;
import org.apache.tsfile.read.reader.chunk.ChunkReader;
-import org.apache.tsfile.utils.Binary;
import org.apache.tsfile.utils.DateUtils;
import org.apache.tsfile.utils.Pair;
import org.apache.tsfile.utils.TsPrimitiveType;
@@ -271,47 +270,36 @@ public class TsFileInsertionEventScanParser extends
TsFileInsertionEventParser {
}
private void putValueToColumns(final BatchData data, final Tablet tablet,
final int rowIndex) {
- final Object[] columns = tablet.values;
-
if (data.getDataType() == TSDataType.VECTOR) {
- for (int i = 0; i < columns.length; ++i) {
+ for (int i = 0; i < tablet.getSchemas().size(); ++i) {
final TsPrimitiveType primitiveType = data.getVector()[i];
if (Objects.isNull(primitiveType)) {
- tablet.bitMaps[i].mark(rowIndex);
- final TSDataType type = tablet.getSchemas().get(i).getType();
- if (type == TSDataType.TEXT || type == TSDataType.BLOB || type ==
TSDataType.STRING) {
- ((Binary[]) columns[i])[rowIndex] = Binary.EMPTY_VALUE;
- }
- if (type == TSDataType.DATE) {
- ((LocalDate[]) columns[i])[rowIndex] = EMPTY_DATE;
- }
continue;
}
switch (tablet.getSchemas().get(i).getType()) {
case BOOLEAN:
- ((boolean[]) columns[i])[rowIndex] = primitiveType.getBoolean();
+ tablet.addValue(rowIndex, i, primitiveType.getBoolean());
break;
case INT32:
- ((int[]) columns[i])[rowIndex] = primitiveType.getInt();
+ tablet.addValue(rowIndex, i, primitiveType.getInt());
break;
case DATE:
- ((LocalDate[]) columns[i])[rowIndex] =
- DateUtils.parseIntToLocalDate(primitiveType.getInt());
+ tablet.addValue(rowIndex, i,
DateUtils.parseIntToLocalDate(primitiveType.getInt()));
break;
case INT64:
case TIMESTAMP:
- ((long[]) columns[i])[rowIndex] = primitiveType.getLong();
+ tablet.addValue(rowIndex, i, primitiveType.getLong());
break;
case FLOAT:
- ((float[]) columns[i])[rowIndex] = primitiveType.getFloat();
+ tablet.addValue(rowIndex, i, primitiveType.getFloat());
break;
case DOUBLE:
- ((double[]) columns[i])[rowIndex] = primitiveType.getDouble();
+ tablet.addValue(rowIndex, i, primitiveType.getDouble());
break;
case TEXT:
case BLOB:
case STRING:
- ((Binary[]) columns[i])[rowIndex] = primitiveType.getBinary();
+ tablet.addValue(rowIndex, i,
primitiveType.getBinary().getValues());
break;
default:
throw new UnSupportedDataTypeException("UnSupported" +
primitiveType.getDataType());
@@ -320,28 +308,28 @@ public class TsFileInsertionEventScanParser extends
TsFileInsertionEventParser {
} else {
switch (tablet.getSchemas().get(0).getType()) {
case BOOLEAN:
- ((boolean[]) columns[0])[rowIndex] = data.getBoolean();
+ tablet.addValue(rowIndex, 0, data.getBoolean());
break;
case INT32:
- ((int[]) columns[0])[rowIndex] = data.getInt();
+ tablet.addValue(rowIndex, 0, data.getInt());
break;
case DATE:
- ((LocalDate[]) columns[0])[rowIndex] =
DateUtils.parseIntToLocalDate(data.getInt());
+ tablet.addValue(rowIndex, 0,
DateUtils.parseIntToLocalDate(data.getInt()));
break;
case INT64:
case TIMESTAMP:
- ((long[]) columns[0])[rowIndex] = data.getLong();
+ tablet.addValue(rowIndex, 0, data.getLong());
break;
case FLOAT:
- ((float[]) columns[0])[rowIndex] = data.getFloat();
+ tablet.addValue(rowIndex, 0, data.getFloat());
break;
case DOUBLE:
- ((double[]) columns[0])[rowIndex] = data.getDouble();
+ tablet.addValue(rowIndex, 0, data.getDouble());
break;
case TEXT:
case BLOB:
case STRING:
- ((Binary[]) columns[0])[rowIndex] = data.getBinary();
+ tablet.addValue(rowIndex, 0, data.getBinary().getValues());
break;
default:
throw new UnSupportedDataTypeException("UnSupported" +
data.getDataType());
diff --git
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/parser/table/TsFileInsertionEventTableParser.java
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/parser/table/TsFileInsertionEventTableParser.java
index 84bff8ca6f9..e1b2f49e8a2 100644
---
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/parser/table/TsFileInsertionEventTableParser.java
+++
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/parser/table/TsFileInsertionEventTableParser.java
@@ -60,7 +60,7 @@ public class TsFileInsertionEventTableParser extends
TsFileInsertionEventParser
try {
tsFileSequenceReader = new TsFileSequenceReader(tsFile.getPath(), true,
true);
filteredTableSchemaIterator =
-
tsFileSequenceReader.readFileMetadata().getTableSchemaMap().entrySet().stream()
+ tsFileSequenceReader.getTableSchemaMap().entrySet().stream()
.filter(entry -> Objects.isNull(pattern) ||
pattern.matchesTable(entry.getKey()))
.iterator();
tableQueryExecutor =
diff --git
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/memory/PipeMemoryWeightUtil.java
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/memory/PipeMemoryWeightUtil.java
index ed1eab78929..20b48b8ad48 100644
---
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/memory/PipeMemoryWeightUtil.java
+++
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/memory/PipeMemoryWeightUtil.java
@@ -29,6 +29,7 @@ import org.apache.tsfile.read.common.BatchData;
import org.apache.tsfile.read.common.Field;
import org.apache.tsfile.read.common.RowRecord;
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.record.Tablet;
@@ -197,9 +198,12 @@ public class PipeMemoryWeightUtil {
return totalSizeInBytes;
}
+ long[] timestamps = tablet.getTimestamps();
+ Object[] tabletValues = tablet.getValues();
+
// timestamps
- if (tablet.timestamps != null) {
- totalSizeInBytes += tablet.timestamps.length * 8L;
+ if (timestamps != null) {
+ totalSizeInBytes += timestamps.length * 8L;
}
// values
@@ -217,10 +221,10 @@ public class PipeMemoryWeightUtil {
}
if (tsDataType.isBinary()) {
- if (tablet.values == null || tablet.values.length <= column) {
+ if (tabletValues == null || tabletValues.length <= column) {
continue;
}
- final Binary[] values = ((Binary[]) tablet.values[column]);
+ final Binary[] values = ((Binary[]) tabletValues[column]);
if (values == null) {
continue;
}
@@ -229,15 +233,16 @@ public class PipeMemoryWeightUtil {
value == null ? 0 : (value.getLength() == -1 ? 0 :
value.getLength());
}
} else {
- totalSizeInBytes += (long) tablet.timestamps.length *
tsDataType.getDataTypeSize();
+ totalSizeInBytes += (long) timestamps.length *
tsDataType.getDataTypeSize();
}
}
}
// bitMaps
- if (tablet.bitMaps != null) {
- for (int i = 0; i < tablet.bitMaps.length; i++) {
- totalSizeInBytes += tablet.bitMaps[i] == null ? 0 :
tablet.bitMaps[i].getSize();
+ BitMap[] bitMaps = tablet.getBitMaps();
+ if (bitMaps != null) {
+ for (int i = 0; i < bitMaps.length; i++) {
+ totalSizeInBytes += bitMaps[i] == null ? 0 : bitMaps[i].getSize();
}
}
diff --git
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/load/LoadTsFileToTableModelAnalyzer.java
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/load/LoadTsFileToTableModelAnalyzer.java
index fd75d2b0623..cd5c7d09b74 100644
---
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/load/LoadTsFileToTableModelAnalyzer.java
+++
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/load/LoadTsFileToTableModelAnalyzer.java
@@ -123,6 +123,8 @@ public class LoadTsFileToTableModelAnalyzer extends
LoadTsFileAnalyzer {
// can be reused when constructing tsfile resource
final TsFileSequenceReaderTimeseriesMetadataIterator
timeseriesMetadataIterator =
new TsFileSequenceReaderTimeseriesMetadataIterator(reader, true, 1);
+ final Map<String, org.apache.tsfile.file.metadata.TableSchema>
tableSchemaMap =
+ reader.getTableSchemaMap();
// check if the tsfile is empty
if (!timeseriesMetadataIterator.hasNext()) {
@@ -130,8 +132,7 @@ public class LoadTsFileToTableModelAnalyzer extends
LoadTsFileAnalyzer {
}
// check whether the tsfile is table-model or not
- if (Objects.isNull(reader.readFileMetadata().getTableSchemaMap())
- || reader.readFileMetadata().getTableSchemaMap().isEmpty()) {
+ if (Objects.isNull(tableSchemaMap) || tableSchemaMap.isEmpty()) {
throw new SemanticException("Attempted to load a tree-model TsFile
into table-model.");
}
@@ -149,7 +150,7 @@ public class LoadTsFileToTableModelAnalyzer extends
LoadTsFileAnalyzer {
schemaCache.setCurrentModificationsAndTimeIndex(tsFileResource, reader);
for (Map.Entry<String, org.apache.tsfile.file.metadata.TableSchema>
name2Schema :
- reader.readFileMetadata().getTableSchemaMap().entrySet()) {
+ tableSchemaMap.entrySet()) {
final TableSchema fileSchema =
TableSchema.fromTsFileTableSchema(name2Schema.getKey(),
name2Schema.getValue());
schemaCache.createTable(fileSchema, context, metadata);
diff --git
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/load/LoadTsFileToTreeModelAnalyzer.java
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/load/LoadTsFileToTreeModelAnalyzer.java
index 84d684733dc..d212aec6ea5 100644
---
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/load/LoadTsFileToTreeModelAnalyzer.java
+++
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/load/LoadTsFileToTreeModelAnalyzer.java
@@ -40,6 +40,7 @@ import org.apache.commons.io.FileUtils;
import org.apache.tsfile.encrypt.EncryptParameter;
import org.apache.tsfile.encrypt.EncryptUtils;
import org.apache.tsfile.file.metadata.IDeviceID;
+import org.apache.tsfile.file.metadata.TableSchema;
import org.apache.tsfile.file.metadata.TimeseriesMetadata;
import org.apache.tsfile.read.TsFileSequenceReader;
import org.apache.tsfile.read.TsFileSequenceReaderTimeseriesMetadataIterator;
@@ -132,8 +133,8 @@ public class LoadTsFileToTreeModelAnalyzer extends
LoadTsFileAnalyzer {
// check whether the tsfile is tree-model or not
// TODO: currently, loading a file with both tree-model and table-model
data is not supported.
// May need to support this and remove this check in the future.
- if (Objects.nonNull(reader.readFileMetadata().getTableSchemaMap())
- && reader.readFileMetadata().getTableSchemaMap().size() != 0) {
+ Map<String, TableSchema> tableSchemaMap = reader.getTableSchemaMap();
+ if (Objects.nonNull(tableSchemaMap) && !tableSchemaMap.isEmpty()) {
throw new SemanticException("Attempted to load a table-model TsFile
into tree-model.");
}
diff --git
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/utils/CompactionTableSchemaCollector.java
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/utils/CompactionTableSchemaCollector.java
index a842b46a048..002858d93cf 100644
---
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/utils/CompactionTableSchemaCollector.java
+++
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/utils/CompactionTableSchemaCollector.java
@@ -70,7 +70,7 @@ public class CompactionTableSchemaCollector {
Map<String, TableSchema> targetTableSchemaMap = new HashMap<>();
for (TsFileResource resource : sourceFiles) {
TsFileSequenceReader reader = readerMap.get(resource);
- Map<String, TableSchema> tableSchemaMap =
reader.readFileMetadata().getTableSchemaMap();
+ Map<String, TableSchema> tableSchemaMap = reader.getTableSchemaMap();
if (tableSchemaMap == null) {
// v3 tsfile
continue;
diff --git
a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/connector/PipeTabletEventSorterTest.java
b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/connector/PipeTabletEventSorterTest.java
index e6f2d44a589..72d20dab050 100644
---
a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/connector/PipeTabletEventSorterTest.java
+++
b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/connector/PipeTabletEventSorterTest.java
@@ -43,15 +43,6 @@ import java.util.Set;
public class PipeTabletEventSorterTest {
- private static boolean checkSorted(final Tablet tablet) {
- for (int i = 1; i < tablet.getRowSize(); i++) {
- if (tablet.timestamps[i] < tablet.timestamps[i - 1]) {
- return false;
- }
- }
- return true;
- }
-
@Test
public void testTreeModelDeduplicateAndSort() {
List<IMeasurementSchema> schemaList = new ArrayList<>();
@@ -84,27 +75,27 @@ public class PipeTabletEventSorterTest {
Set<Integer> indices = new HashSet<>();
for (int i = 0; i < 30; i++) {
- indices.add((int) tablet.timestamps[i]);
+ indices.add((int) tablet.getTimestamp(i));
}
- Assert.assertFalse(checkSorted(tablet));
+ Assert.assertFalse(tablet.isSorted());
new
PipeTreeModelTabletEventSorter(tablet).deduplicateAndSortTimestampsIfNecessary();
- Assert.assertTrue(checkSorted(tablet));
+ Assert.assertTrue(tablet.isSorted());
Assert.assertEquals(indices.size(), tablet.getRowSize());
- final long[] timestamps = Arrays.copyOfRange(tablet.timestamps, 0,
tablet.getRowSize());
+ final long[] timestamps = Arrays.copyOfRange(tablet.getTimestamps(), 0,
tablet.getRowSize());
for (int i = 0; i < 3; ++i) {
Assert.assertArrayEquals(
- timestamps, Arrays.copyOfRange((long[]) tablet.values[0], 0,
tablet.getRowSize()));
+ timestamps, Arrays.copyOfRange((long[]) tablet.getValues()[0], 0,
tablet.getRowSize()));
}
for (int i = 1; i < tablet.getRowSize(); ++i) {
Assert.assertTrue(timestamps[i] > timestamps[i - 1]);
for (int j = 0; j < 3; ++j) {
- Assert.assertTrue(((long[]) tablet.values[j])[i] > ((long[])
tablet.values[j])[i - 1]);
+ Assert.assertTrue((long) tablet.getValue(i, j) > (long)
tablet.getValue(i - 1, j));
}
}
}
@@ -129,27 +120,27 @@ public class PipeTabletEventSorterTest {
Set<Integer> indices = new HashSet<>();
for (int i = 0; i < 10; i++) {
- indices.add((int) tablet.timestamps[i]);
+ indices.add((int) tablet.getTimestamp(i));
}
- Assert.assertTrue(checkSorted(tablet));
+ Assert.assertTrue(tablet.isSorted());
new
PipeTreeModelTabletEventSorter(tablet).deduplicateAndSortTimestampsIfNecessary();
- Assert.assertTrue(checkSorted(tablet));
+ Assert.assertTrue(tablet.isSorted());
Assert.assertEquals(indices.size(), tablet.getRowSize());
- final long[] timestamps = Arrays.copyOfRange(tablet.timestamps, 0,
tablet.getRowSize());
+ final long[] timestamps = Arrays.copyOfRange(tablet.getTimestamps(), 0,
tablet.getRowSize());
for (int i = 0; i < 3; ++i) {
Assert.assertArrayEquals(
- timestamps, Arrays.copyOfRange((long[]) tablet.values[0], 0,
tablet.getRowSize()));
+ timestamps, Arrays.copyOfRange((long[]) tablet.getValues()[0], 0,
tablet.getRowSize()));
}
for (int i = 1; i < tablet.getRowSize(); ++i) {
Assert.assertTrue(timestamps[i] > timestamps[i - 1]);
for (int j = 0; j < 3; ++j) {
- Assert.assertTrue(((long[]) tablet.values[j])[i] > ((long[])
tablet.values[j])[i - 1]);
+ Assert.assertTrue((long) tablet.getValue(i, j) > (long)
tablet.getValue(i - 1, j));
}
}
}
@@ -185,40 +176,40 @@ public class PipeTabletEventSorterTest {
Set<Integer> indices = new HashSet<>();
for (int i = 0; i < 30; i++) {
- indices.add((int) tablet.timestamps[i]);
+ indices.add((int) tablet.getTimestamp(i));
}
- Assert.assertFalse(checkSorted(tablet));
+ Assert.assertFalse(tablet.isSorted());
- long[] timestamps = Arrays.copyOfRange(tablet.timestamps, 0,
tablet.getRowSize());
+ long[] timestamps = Arrays.copyOfRange(tablet.getTimestamps(), 0,
tablet.getRowSize());
for (int i = 0; i < 3; ++i) {
Assert.assertArrayEquals(
- timestamps, Arrays.copyOfRange((long[]) tablet.values[0], 0,
tablet.getRowSize()));
+ timestamps, Arrays.copyOfRange((long[]) tablet.getValues()[0], 0,
tablet.getRowSize()));
}
for (int i = 1; i < tablet.getRowSize(); ++i) {
Assert.assertTrue(timestamps[i] != timestamps[i - 1]);
for (int j = 0; j < 3; ++j) {
- Assert.assertTrue(((long[]) tablet.values[j])[i] != ((long[])
tablet.values[j])[i - 1]);
+ Assert.assertNotEquals((long) tablet.getValue(i, j), (long)
tablet.getValue(i - 1, j));
}
}
new
PipeTreeModelTabletEventSorter(tablet).deduplicateAndSortTimestampsIfNecessary();
- Assert.assertTrue(checkSorted(tablet));
+ Assert.assertTrue(tablet.isSorted());
Assert.assertEquals(indices.size(), tablet.getRowSize());
- timestamps = Arrays.copyOfRange(tablet.timestamps, 0, tablet.getRowSize());
+ timestamps = Arrays.copyOfRange(tablet.getTimestamps(), 0,
tablet.getRowSize());
for (int i = 0; i < 3; ++i) {
Assert.assertArrayEquals(
- timestamps, Arrays.copyOfRange((long[]) tablet.values[0], 0,
tablet.getRowSize()));
+ timestamps, Arrays.copyOfRange((long[]) tablet.getValues()[0], 0,
tablet.getRowSize()));
}
for (int i = 1; i < tablet.getRowSize(); ++i) {
Assert.assertTrue(timestamps[i] > timestamps[i - 1]);
for (int j = 0; j < 3; ++j) {
- Assert.assertTrue(((long[]) tablet.values[j])[i] > ((long[])
tablet.values[j])[i - 1]);
+ Assert.assertTrue((long) tablet.getValue(i, j) > (long)
tablet.getValue(i - 1, j));
}
}
}
@@ -256,9 +247,10 @@ public class PipeTabletEventSorterTest {
public void doTableModelTest(final boolean hasDuplicates, final boolean
isUnSorted) {
final Tablet tablet = generateTablet("test", 10, hasDuplicates,
isUnSorted);
new
PipeTableModelTabletEventSorter(tablet).sortAndDeduplicateByDevIdTimestamp();
+ long[] timestamps = tablet.getTimestamps();
for (int i = 1; i < tablet.getRowSize(); i++) {
- long time = tablet.timestamps[i];
- Assert.assertTrue(time > tablet.timestamps[i - 1]);
+ long time = timestamps[i];
+ Assert.assertTrue(time > timestamps[i - 1]);
Assert.assertEquals(
tablet.getValue(i, 0),
new Binary(String.valueOf(i /
100).getBytes(StandardCharsets.UTF_8)));
@@ -278,9 +270,10 @@ public class PipeTabletEventSorterTest {
public void doTableModelTest1(final boolean hasDuplicates, final boolean
isUnSorted) {
final Tablet tablet = generateTablet("test", 10, hasDuplicates,
isUnSorted);
new
PipeTableModelTabletEventSorter(tablet).sortAndDeduplicateByTimestampIfNecessary();
+ long[] timestamps = tablet.getTimestamps();
for (int i = 1; i < tablet.getRowSize(); i++) {
- long time = tablet.timestamps[i];
- Assert.assertTrue(time > tablet.timestamps[i - 1]);
+ long time = timestamps[i];
+ Assert.assertTrue(time > timestamps[i - 1]);
Assert.assertEquals(
tablet.getValue(i, 0),
new Binary(String.valueOf(i /
100).getBytes(StandardCharsets.UTF_8)));
diff --git
a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/event/PipeTabletInsertionEventTest.java
b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/event/PipeTabletInsertionEventTest.java
index dd3626b6fbc..8516a9900e6 100644
---
a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/event/PipeTabletInsertionEventTest.java
+++
b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/event/PipeTabletInsertionEventTest.java
@@ -213,11 +213,14 @@ public class PipeTabletInsertionEventTest {
((Binary[]) values[9])[r] = BytesUtils.valueOf("string");
}
- tabletForInsertRowNode = new Tablet(deviceId, Arrays.asList(schemas), 1);
- tabletForInsertRowNode.values = values;
- tabletForInsertRowNode.timestamps = new long[] {times[0]};
- tabletForInsertRowNode.setRowSize(1);
- tabletForInsertRowNode.bitMaps = bitMapsForInsertRowNode;
+ tabletForInsertRowNode =
+ new Tablet(
+ deviceId,
+ Arrays.asList(schemas),
+ new long[] {times[0]},
+ values,
+ bitMapsForInsertRowNode,
+ 1);
// create tablet for insertTabletNode
BitMap[] bitMapsForInsertTabletNode = new BitMap[schemas.length];
@@ -250,10 +253,14 @@ public class PipeTabletInsertionEventTest {
}
tabletForInsertTabletNode = new Tablet(deviceId, Arrays.asList(schemas),
times.length);
- tabletForInsertTabletNode.values = values;
- tabletForInsertTabletNode.timestamps = times;
- tabletForInsertTabletNode.setRowSize(times.length);
- tabletForInsertTabletNode.bitMaps = bitMapsForInsertTabletNode;
+ tabletForInsertTabletNode =
+ new Tablet(
+ deviceId,
+ Arrays.asList(schemas),
+ times,
+ values,
+ bitMapsForInsertTabletNode,
+ times.length);
}
@Test
diff --git
a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/buffer/BloomFilterCacheTest.java
b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/buffer/BloomFilterCacheTest.java
index 83629b5eaba..ee1c2a7df2a 100644
---
a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/buffer/BloomFilterCacheTest.java
+++
b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/buffer/BloomFilterCacheTest.java
@@ -188,16 +188,13 @@ public class BloomFilterCacheTest {
try (TsFileWriter tsFileWriter = new TsFileWriter(f, schema)) {
// construct the tablet
Tablet tablet = new Tablet(device, measurementSchemas);
- long[] timestamps = tablet.timestamps;
- Object[] values = tablet.values;
long timestamp = 1;
long value = 1000000L;
for (int r = 0; r < rowNum; r++, value++) {
int row = tablet.getRowSize();
tablet.addTimestamp(row, timestamp++);
for (int i = 0; i < sensorNum; i++) {
- long[] sensor = (long[]) values[i];
- sensor[row] = value;
+ tablet.addValue(row, i, value);
}
// write Tablet to TsFile
if (tablet.getRowSize() == tablet.getMaxRowNumber()) {
diff --git
a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/CompactionValidationTest.java
b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/CompactionValidationTest.java
index 53469ed80b7..12be8035f28 100644
---
a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/CompactionValidationTest.java
+++
b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/CompactionValidationTest.java
@@ -26,14 +26,12 @@ import
org.apache.iotdb.db.storageengine.dataregion.utils.TsFileResourceUtils;
import org.apache.iotdb.db.utils.constant.TestConstant;
import org.apache.commons.io.FileUtils;
-import org.apache.tsfile.common.conf.TSFileConfig;
import org.apache.tsfile.enums.TSDataType;
import org.apache.tsfile.exception.write.WriteProcessException;
import org.apache.tsfile.file.metadata.IDeviceID;
import org.apache.tsfile.file.metadata.enums.TSEncoding;
import org.apache.tsfile.fileSystem.FSFactoryProducer;
import org.apache.tsfile.read.common.Path;
-import org.apache.tsfile.utils.Binary;
import org.apache.tsfile.write.TsFileWriter;
import org.apache.tsfile.write.record.Tablet;
import org.apache.tsfile.write.schema.IMeasurementSchema;
@@ -100,16 +98,13 @@ public class CompactionValidationTest {
long startValue)
throws IOException, WriteProcessException {
Tablet tablet = new Tablet(deviceId, schemas);
- long[] timestamps = tablet.timestamps;
- Object[] values = tablet.values;
long sensorNum = schemas.size();
for (long r = 0; r < rowNum; r++, startValue++) {
int row = tablet.getRowSize();
tablet.addTimestamp(row, startTime++);
for (int i = 0; i < sensorNum; i++) {
- Binary[] textSensor = (Binary[]) values[i];
- textSensor[row] = new Binary("testString.........",
TSFileConfig.STRING_CHARSET);
+ tablet.addValue(row, i, "testString.........");
}
// write
if (tablet.getRowSize() == tablet.getMaxRowNumber()) {
diff --git
a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/tablemodel/TableModelFastCompactionPerformerTest.java
b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/tablemodel/TableModelFastCompactionPerformerTest.java
index 9b4092745f6..66602eb347e 100644
---
a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/tablemodel/TableModelFastCompactionPerformerTest.java
+++
b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/tablemodel/TableModelFastCompactionPerformerTest.java
@@ -28,7 +28,6 @@ import
org.apache.iotdb.db.storageengine.dataregion.compaction.execute.task.Inne
import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource;
import org.apache.tsfile.exception.write.WriteProcessException;
-import org.apache.tsfile.file.metadata.TsFileMetadata;
import org.apache.tsfile.file.metadata.enums.CompressionType;
import org.apache.tsfile.file.metadata.enums.TSEncoding;
import org.apache.tsfile.read.TsFileSequenceReader;
@@ -147,14 +146,12 @@ public class TableModelFastCompactionPerformerTest
extends AbstractCompactionTes
TsFileResource targetResource0 = tsFileManager.getTsFileList(true).get(0);
try (TsFileSequenceReader reader =
new
TsFileSequenceReader(targetResource0.getTsFile().getAbsolutePath())) {
- TsFileMetadata tsFileMetadata = reader.readFileMetadata();
- Assert.assertEquals(1, tsFileMetadata.getTableSchemaMap().size());
+ Assert.assertEquals(1, reader.getTableSchemaMap().size());
}
TsFileResource targetResource1 = tsFileManager.getTsFileList(true).get(1);
try (TsFileSequenceReader reader =
new
TsFileSequenceReader(targetResource1.getTsFile().getAbsolutePath())) {
- TsFileMetadata tsFileMetadata = reader.readFileMetadata();
- Assert.assertEquals(1, tsFileMetadata.getTableSchemaMap().size());
+ Assert.assertEquals(1, reader.getTableSchemaMap().size());
}
}
@@ -210,8 +207,7 @@ public class TableModelFastCompactionPerformerTest extends
AbstractCompactionTes
try (TsFileSequenceReader reader =
new TsFileSequenceReader(
tsFileManager.getTsFileList(true).get(0).getTsFile().getAbsolutePath())) {
- TsFileMetadata tsFileMetadata = reader.readFileMetadata();
- Assert.assertEquals(1, tsFileMetadata.getTableSchemaMap().size());
+ Assert.assertEquals(1, reader.getTableSchemaMap().size());
}
}
@@ -249,8 +245,7 @@ public class TableModelFastCompactionPerformerTest extends
AbstractCompactionTes
try (TsFileSequenceReader reader =
new TsFileSequenceReader(
tsFileManager.getTsFileList(true).get(0).getTsFile().getAbsolutePath())) {
- TsFileMetadata tsFileMetadata = reader.readFileMetadata();
- Assert.assertEquals(1, tsFileMetadata.getTableSchemaMap().size());
+ Assert.assertEquals(1, reader.getTableSchemaMap().size());
}
}
}
diff --git
a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/tablemodel/TableModelReadChunkCompactionPerformerTest.java
b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/tablemodel/TableModelReadChunkCompactionPerformerTest.java
index 6ca6ed62cce..a294ba01485 100644
---
a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/tablemodel/TableModelReadChunkCompactionPerformerTest.java
+++
b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/tablemodel/TableModelReadChunkCompactionPerformerTest.java
@@ -32,7 +32,6 @@ import
org.apache.tsfile.exception.write.WriteProcessException;
import org.apache.tsfile.file.metadata.IDeviceID;
import org.apache.tsfile.file.metadata.TableSchema;
import org.apache.tsfile.file.metadata.TimeseriesMetadata;
-import org.apache.tsfile.file.metadata.TsFileMetadata;
import org.apache.tsfile.file.metadata.enums.CompressionType;
import org.apache.tsfile.file.metadata.enums.TSEncoding;
import org.apache.tsfile.read.TsFileSequenceReader;
@@ -116,8 +115,7 @@ public class TableModelReadChunkCompactionPerformerTest
extends AbstractCompacti
TsFileResource targetResource = tsFileManager.getTsFileList(true).get(0);
try (TsFileSequenceReader reader =
new
TsFileSequenceReader(targetResource.getTsFile().getAbsolutePath())) {
- TsFileMetadata tsFileMetadata = reader.readFileMetadata();
- TableSchema tableSchema = tsFileMetadata.getTableSchemaMap().get("t1");
+ TableSchema tableSchema = reader.getTableSchemaMap().get("t1");
Assert.assertEquals(5, tableSchema.getColumnTypes().size());
Map<IDeviceID, List<TimeseriesMetadata>> allTimeseriesMetadata =
reader.getAllTimeseriesMetadata(true);
@@ -165,8 +163,7 @@ public class TableModelReadChunkCompactionPerformerTest
extends AbstractCompacti
TsFileResource targetResource = tsFileManager.getTsFileList(true).get(0);
try (TsFileSequenceReader reader =
new
TsFileSequenceReader(targetResource.getTsFile().getAbsolutePath())) {
- TsFileMetadata tsFileMetadata = reader.readFileMetadata();
- Assert.assertTrue(tsFileMetadata.getTableSchemaMap().isEmpty());
+ Assert.assertTrue(reader.getTableSchemaMap().isEmpty());
Map<IDeviceID, List<TimeseriesMetadata>> allTimeseriesMetadata =
reader.getAllTimeseriesMetadata(true);
for (Map.Entry<IDeviceID, List<TimeseriesMetadata>> entry :
@@ -213,8 +210,7 @@ public class TableModelReadChunkCompactionPerformerTest
extends AbstractCompacti
TsFileResource targetResource = tsFileManager.getTsFileList(true).get(0);
try (TsFileSequenceReader reader =
new
TsFileSequenceReader(targetResource.getTsFile().getAbsolutePath())) {
- TsFileMetadata tsFileMetadata = reader.readFileMetadata();
- Assert.assertEquals(1, tsFileMetadata.getTableSchemaMap().size());
+ Assert.assertEquals(1, reader.getTableSchemaMap().size());
}
}
@@ -321,8 +317,7 @@ public class TableModelReadChunkCompactionPerformerTest
extends AbstractCompacti
try (TsFileSequenceReader reader =
new TsFileSequenceReader(
tsFileManager.getTsFileList(true).get(0).getTsFile().getAbsolutePath())) {
- TsFileMetadata tsFileMetadata = reader.readFileMetadata();
- Assert.assertEquals(1, tsFileMetadata.getTableSchemaMap().size());
+ Assert.assertEquals(1, reader.getTableSchemaMap().size());
}
}
}
diff --git
a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/tablemodel/TableModelReadPointCompactionPerformerTest.java
b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/tablemodel/TableModelReadPointCompactionPerformerTest.java
index a68a8c400a6..1cbb803d529 100644
---
a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/tablemodel/TableModelReadPointCompactionPerformerTest.java
+++
b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/tablemodel/TableModelReadPointCompactionPerformerTest.java
@@ -29,7 +29,6 @@ import
org.apache.iotdb.db.storageengine.dataregion.read.control.FileReaderManag
import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource;
import org.apache.tsfile.exception.write.WriteProcessException;
-import org.apache.tsfile.file.metadata.TsFileMetadata;
import org.apache.tsfile.file.metadata.enums.CompressionType;
import org.apache.tsfile.file.metadata.enums.TSEncoding;
import org.apache.tsfile.read.TsFileSequenceReader;
@@ -159,14 +158,12 @@ public class TableModelReadPointCompactionPerformerTest
extends AbstractCompacti
TsFileResource targetResource0 = tsFileManager.getTsFileList(true).get(0);
try (TsFileSequenceReader reader =
new
TsFileSequenceReader(targetResource0.getTsFile().getAbsolutePath())) {
- TsFileMetadata tsFileMetadata = reader.readFileMetadata();
- Assert.assertEquals(1, tsFileMetadata.getTableSchemaMap().size());
+ Assert.assertEquals(1, reader.getTableSchemaMap().size());
}
TsFileResource targetResource1 = tsFileManager.getTsFileList(true).get(1);
try (TsFileSequenceReader reader =
new
TsFileSequenceReader(targetResource1.getTsFile().getAbsolutePath())) {
- TsFileMetadata tsFileMetadata = reader.readFileMetadata();
- Assert.assertEquals(1, tsFileMetadata.getTableSchemaMap().size());
+ Assert.assertEquals(1, reader.getTableSchemaMap().size());
}
}
@@ -222,8 +219,7 @@ public class TableModelReadPointCompactionPerformerTest
extends AbstractCompacti
try (TsFileSequenceReader reader =
new TsFileSequenceReader(
tsFileManager.getTsFileList(true).get(0).getTsFile().getAbsolutePath())) {
- TsFileMetadata tsFileMetadata = reader.readFileMetadata();
- Assert.assertEquals(1, tsFileMetadata.getTableSchemaMap().size());
+ Assert.assertEquals(1, reader.getTableSchemaMap().size());
}
}
@@ -261,8 +257,7 @@ public class TableModelReadPointCompactionPerformerTest
extends AbstractCompacti
try (TsFileSequenceReader reader =
new TsFileSequenceReader(
tsFileManager.getTsFileList(true).get(0).getTsFile().getAbsolutePath())) {
- TsFileMetadata tsFileMetadata = reader.readFileMetadata();
- Assert.assertEquals(1, tsFileMetadata.getTableSchemaMap().size());
+ Assert.assertEquals(1, reader.getTableSchemaMap().size());
}
}
}
diff --git
a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/wal/recover/file/UnsealedTsFileRecoverPerformerTest.java
b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/wal/recover/file/UnsealedTsFileRecoverPerformerTest.java
index 391d27dcd32..4addc6e1be8 100644
---
a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/wal/recover/file/UnsealedTsFileRecoverPerformerTest.java
+++
b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/wal/recover/file/UnsealedTsFileRecoverPerformerTest.java
@@ -48,7 +48,6 @@ import
org.apache.tsfile.exception.write.WriteProcessException;
import org.apache.tsfile.file.metadata.ChunkMetadata;
import org.apache.tsfile.file.metadata.IDeviceID;
import org.apache.tsfile.file.metadata.TableSchema;
-import org.apache.tsfile.file.metadata.TsFileMetadata;
import org.apache.tsfile.file.metadata.enums.CompressionType;
import org.apache.tsfile.file.metadata.enums.TSEncoding;
import org.apache.tsfile.read.TsFileSequenceReader;
@@ -212,8 +211,7 @@ public class UnsealedTsFileRecoverPerformerTest {
}
// check file content
TsFileSequenceReader reader = new TsFileSequenceReader(FILE_NAME);
- TsFileMetadata metadata = reader.readFileMetadata();
- Map<String, TableSchema> tableSchemaMap =
reader.readFileMetadata().getTableSchemaMap();
+ Map<String, TableSchema> tableSchemaMap = reader.getTableSchemaMap();
assertEquals(1, tableSchemaMap.size());
assertTrue(tableSchemaMap.containsKey("table1"));
reader.close();
diff --git
a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/tools/TsFileSelfCheckToolTest.java
b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/tools/TsFileSelfCheckToolTest.java
index 1aca298ba17..192864f1a6f 100644
---
a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/tools/TsFileSelfCheckToolTest.java
+++
b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/tools/TsFileSelfCheckToolTest.java
@@ -102,9 +102,6 @@ public class TsFileSelfCheckToolTest {
// construct the tablet
Tablet tablet = new Tablet(device, measurementSchemas);
- long[] timestamps = tablet.timestamps;
- Object[] values = tablet.values;
-
long timestamp = 1;
long value = 1000000L;
@@ -112,8 +109,7 @@ public class TsFileSelfCheckToolTest {
int row = tablet.getRowSize();
tablet.addTimestamp(row, timestamp++);
for (int i = 0; i < sensorNum; i++) {
- long[] sensor = (long[]) values[i];
- sensor[row] = value;
+ tablet.addValue(row, i, value);
}
// write Tablet to TsFile
if (tablet.getRowSize() == tablet.getMaxRowNumber()) {
diff --git
a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/tools/TsFileSketchToolTest.java
b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/tools/TsFileSketchToolTest.java
index d7b3cd73957..23c056677bc 100644
---
a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/tools/TsFileSketchToolTest.java
+++
b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/tools/TsFileSketchToolTest.java
@@ -103,16 +103,13 @@ public class TsFileSketchToolTest {
// add measurements into TSFileWriter
// construct the tablet
Tablet tablet = new Tablet(device.toString(), measurementSchemas);
- long[] timestamps = tablet.timestamps;
- Object[] values = tablet.values;
long timestamp = 1;
long value = 1000000L;
for (int r = 0; r < rowNum; r++, value++) {
int row = tablet.getRowSize();
tablet.addTimestamp(row, timestamp++);
for (int i = 0; i < sensorNum; i++) {
- long[] sensor = (long[]) values[i];
- sensor[row] = value;
+ tablet.addValue(row, i, value);
}
// write Tablet to TsFile
if (tablet.getRowSize() == tablet.getMaxRowNumber()) {
@@ -129,16 +126,13 @@ public class TsFileSketchToolTest {
// add aligned measurements into TSFileWriter
// construct the tablet
tablet = new Tablet(alignedDevice.toString(),
alignedMeasurementSchemas);
- timestamps = tablet.timestamps;
- values = tablet.values;
timestamp = 1;
value = 1000000L;
for (int r = 0; r < rowNum; r++, value++) {
int row = tablet.getRowSize();
tablet.addTimestamp(row, timestamp++);
for (int i = 0; i < sensorNum; i++) {
- long[] sensor = (long[]) values[i];
- sensor[row] = value;
+ tablet.addValue(row, i, value);
}
// write Tablet to TsFile
if (tablet.getRowSize() == tablet.getMaxRowNumber()) {
diff --git a/pom.xml b/pom.xml
index e278d579b44..35708d0638f 100644
--- a/pom.xml
+++ b/pom.xml
@@ -167,7 +167,7 @@
<thrift.version>0.14.1</thrift.version>
<xz.version>1.9</xz.version>
<zstd-jni.version>1.5.6-3</zstd-jni.version>
- <tsfile.version>2.0.0-250109-SNAPSHOT</tsfile.version>
+ <tsfile.version>2.0.0-250113-SNAPSHOT</tsfile.version>
</properties>
<!--
if we claim dependencies in dependencyManagement, then we do not claim