From 2aea45b14e426ca4ebcb2cc6ead4dff911b5ecac Mon Sep 17 00:00:00 2001 From: yschengzi <87161145+yschengzi@users.noreply.github.com> Date: Fri, 16 Sep 2022 22:12:18 +0800 Subject: [PATCH] [IOTDB-3656] basic mpp load implement (#6764) --- integration-test/import-control.xml | 7 + .../iotdb/it/env/AbstractNodeWrapper.java | 4 +- .../apache/iotdb/itbase/env/BaseConfig.java | 4 + .../apache/iotdb/db/it/IOTDBLoadTsFileIT.java | 412 ++++++++++++++ .../iotdb/db/it/env/StandaloneEnvConfig.java | 5 + .../iotdb/commons/conf/IoTDBConstant.java | 1 + .../commons/partition/DataPartition.java | 8 + .../org/apache/iotdb/db/conf/IoTDBConfig.java | 21 + .../iotdb/db/engine/StorageEngineV2.java | 72 +++ .../db/engine/load/AlignedChunkData.java | 497 ++++++++++++++++ .../iotdb/db/engine/load/ChunkData.java | 72 +++ .../db/engine/load/LoadTsFileManager.java | 246 ++++++++ .../db/engine/load/NonAlignedChunkData.java | 366 ++++++++++++ .../db/engine/storagegroup/DataRegion.java | 10 +- .../db/mpp/plan/analyze/AnalyzeVisitor.java | 62 ++ .../db/mpp/plan/analyze/SchemaValidator.java | 13 + .../db/mpp/plan/execution/QueryExecution.java | 9 + .../iotdb/db/mpp/plan/parser/ASTVisitor.java | 40 ++ .../mpp/plan/planner/LogicalPlanVisitor.java | 8 + .../plan/planner/plan/FragmentInstance.java | 3 + .../plan/planner/plan/node/PlanNodeId.java | 5 + .../plan/planner/plan/node/PlanNodeType.java | 10 +- .../plan/node/load/LoadSingleTsFileNode.java | 534 ++++++++++++++++++ .../plan/node/load/LoadTsFileNode.java | 106 ++++ .../plan/node/load/LoadTsFilePieceNode.java | 176 ++++++ .../load/LoadTsFileDispatcherImpl.java | 239 ++++++++ .../scheduler/load/LoadTsFileScheduler.java | 252 +++++++++ .../mpp/plan/statement/StatementVisitor.java | 5 + .../statement/crud/LoadTsFileStatement.java | 130 +++++ .../impl/DataNodeInternalRPCServiceImpl.java | 43 ++ .../org/apache/iotdb/rpc/TSStatusCode.java | 2 + thrift/src/main/thrift/datanode.thrift | 21 + .../iotdb/tsfile/file/header/ChunkHeader.java | 3 +- .../tsfile/read/TsFileSequenceReader.java | 2 +- .../write/chunk/AlignedChunkWriterImpl.java | 37 ++ .../tsfile/write/chunk/TimeChunkWriter.java | 52 ++ .../tsfile/write/chunk/ValueChunkWriter.java | 56 ++ .../tsfile/write/writer/TsFileIOWriter.java | 13 + 38 files changed, 3535 insertions(+), 11 deletions(-) create mode 100644 integration-test/src/test/java/org/apache/iotdb/db/it/IOTDBLoadTsFileIT.java create mode 100644 server/src/main/java/org/apache/iotdb/db/engine/load/AlignedChunkData.java create mode 100644 server/src/main/java/org/apache/iotdb/db/engine/load/ChunkData.java create mode 100644 server/src/main/java/org/apache/iotdb/db/engine/load/LoadTsFileManager.java create mode 100644 server/src/main/java/org/apache/iotdb/db/engine/load/NonAlignedChunkData.java create mode 100644 server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/load/LoadSingleTsFileNode.java create mode 100644 server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/load/LoadTsFileNode.java create mode 100644 server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/load/LoadTsFilePieceNode.java create mode 100644 server/src/main/java/org/apache/iotdb/db/mpp/plan/scheduler/load/LoadTsFileDispatcherImpl.java create mode 100644 server/src/main/java/org/apache/iotdb/db/mpp/plan/scheduler/load/LoadTsFileScheduler.java create mode 100644 server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/crud/LoadTsFileStatement.java diff --git a/integration-test/import-control.xml b/integration-test/import-control.xml index 4a13fc907801..b0c7d372eecf 100644 --- a/integration-test/import-control.xml +++ b/integration-test/import-control.xml @@ -27,6 +27,7 @@ + @@ -43,4 +44,10 @@ + + + + + + diff --git a/integration-test/src/main/java/org/apache/iotdb/it/env/AbstractNodeWrapper.java b/integration-test/src/main/java/org/apache/iotdb/it/env/AbstractNodeWrapper.java index a0ebb1a75717..a55777c7d511 100644 --- a/integration-test/src/main/java/org/apache/iotdb/it/env/AbstractNodeWrapper.java +++ b/integration-test/src/main/java/org/apache/iotdb/it/env/AbstractNodeWrapper.java @@ -174,7 +174,9 @@ public void start() { "-Xms200m", "-Xmx200m", "-XX:MaxDirectMemorySize=200m", - "-Djdk.nio.maxCachedBufferSize=262144")); + "-Djdk.nio.maxCachedBufferSize=262144", + "-cp", + templateNodeLibPath)); addStartCmdParams(startCmd); FileUtils.write( stdoutFile, String.join(" ", startCmd) + "\n\n", StandardCharsets.UTF_8, true); diff --git a/integration-test/src/main/java/org/apache/iotdb/itbase/env/BaseConfig.java b/integration-test/src/main/java/org/apache/iotdb/itbase/env/BaseConfig.java index 5cf5f09d6c47..f3c2490abddb 100644 --- a/integration-test/src/main/java/org/apache/iotdb/itbase/env/BaseConfig.java +++ b/integration-test/src/main/java/org/apache/iotdb/itbase/env/BaseConfig.java @@ -70,6 +70,10 @@ default BaseConfig setDataRegionNum(int dataRegionNum) { return this; } + default boolean isEnablePartition() { + return true; + } + default BaseConfig setPartitionInterval(long partitionInterval) { return this; } 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 new file mode 100644 index 000000000000..333415527315 --- /dev/null +++ b/integration-test/src/test/java/org/apache/iotdb/db/it/IOTDBLoadTsFileIT.java @@ -0,0 +1,412 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.it; + +import org.apache.iotdb.it.env.ConfigFactory; +import org.apache.iotdb.it.env.EnvFactory; +import org.apache.iotdb.it.framework.IoTDBTestRunner; +import org.apache.iotdb.itbase.category.ClusterIT; +import org.apache.iotdb.itbase.category.LocalStandaloneIT; +import org.apache.iotdb.tsfile.exception.write.WriteProcessException; +import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType; +import org.apache.iotdb.tsfile.read.common.Path; +import org.apache.iotdb.tsfile.utils.Binary; +import org.apache.iotdb.tsfile.write.TsFileWriter; +import org.apache.iotdb.tsfile.write.record.Tablet; +import org.apache.iotdb.tsfile.write.schema.MeasurementSchema; + +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Files; +import java.sql.Connection; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Statement; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Random; + +@RunWith(IoTDBTestRunner.class) +@Category({LocalStandaloneIT.class, ClusterIT.class}) +public class IOTDBLoadTsFileIT { + private static final Logger LOGGER = LoggerFactory.getLogger(IOTDBLoadTsFileIT.class); + private static final long PARTITION_INTERVAL = 10L; + + private boolean originEnablePartition; + private long originPartitionInterval; + + private File tmpDir; + + @Before + public void setUp() throws Exception { + tmpDir = new File(Files.createTempDirectory("load").toUri()); + originEnablePartition = ConfigFactory.getConfig().isEnablePartition(); + originPartitionInterval = ConfigFactory.getConfig().getPartitionInterval(); + ConfigFactory.getConfig().setPartitionInterval(PARTITION_INTERVAL); + EnvFactory.getEnv().initBeforeTest(); + + registerSchema(); + } + + @After + public void tearDown() throws Exception { + deleteSG(); + + EnvFactory.getEnv().cleanAfterTest(); + ConfigFactory.getConfig().setEnablePartition(originEnablePartition); + ConfigFactory.getConfig().setPartitionInterval(originPartitionInterval); + + if (!deleteDir()) { + LOGGER.error("Can not delete tmp dir for loading tsfile."); + } + } + + private void registerSchema() throws SQLException { + try (Connection connection = EnvFactory.getEnv().getConnection(); + Statement statement = connection.createStatement()) { + + statement.execute("set storage group to " + SchemaConfig.STORAGE_GROUP_0); + statement.execute("set storage group to " + SchemaConfig.STORAGE_GROUP_1); + + statement.execute(convert2SQL(SchemaConfig.DEVICE_0, SchemaConfig.MEASUREMENT_00)); + statement.execute(convert2SQL(SchemaConfig.DEVICE_0, SchemaConfig.MEASUREMENT_01)); + statement.execute(convert2SQL(SchemaConfig.DEVICE_0, SchemaConfig.MEASUREMENT_02)); + statement.execute(convert2SQL(SchemaConfig.DEVICE_0, SchemaConfig.MEASUREMENT_03)); + + statement.execute( + convert2AlignedSQL( + SchemaConfig.DEVICE_1, + Arrays.asList( + SchemaConfig.MEASUREMENT_10, + SchemaConfig.MEASUREMENT_11, + SchemaConfig.MEASUREMENT_12, + SchemaConfig.MEASUREMENT_13))); + + statement.execute(convert2SQL(SchemaConfig.DEVICE_2, SchemaConfig.MEASUREMENT_20)); + + statement.execute(convert2SQL(SchemaConfig.DEVICE_3, SchemaConfig.MEASUREMENT_30)); + + statement.execute( + convert2AlignedSQL(SchemaConfig.DEVICE_4, Arrays.asList(SchemaConfig.MEASUREMENT_40))); + } + } + + private String convert2SQL(String device, MeasurementSchema schema) { + String sql = + String.format( + "create timeseries %s %s", + new Path(device, schema.getMeasurementId()).getFullPath(), schema.getType().name()); + LOGGER.info(String.format("schema execute: %s.", sql)); + return sql; + } + + private String convert2AlignedSQL(String device, List schemas) { + String sql = String.format("create aligned timeseries %s(", device); + for (int i = 0; i < schemas.size(); i++) { + MeasurementSchema schema = schemas.get(i); + sql += (String.format("%s %s", schema.getMeasurementId(), schema.getType().name())); + sql += (i == schemas.size() - 1 ? ")" : ","); + } + LOGGER.info(String.format("schema execute: %s.", sql)); + return sql; + } + + private void deleteSG() throws SQLException { + try (Connection connection = EnvFactory.getEnv().getConnection(); + Statement statement = connection.createStatement()) { + + statement.execute(String.format("delete storage group %s", SchemaConfig.STORAGE_GROUP_0)); + statement.execute(String.format("delete storage group %s", SchemaConfig.STORAGE_GROUP_1)); + } + } + + private boolean deleteDir() { + for (File file : tmpDir.listFiles()) { + if (!file.delete()) { + return false; + } + } + return tmpDir.delete(); + } + + @Test + public void testLoad() throws Exception { + long writtenPoint1 = 0; + // device 0, device 1, sg 0 + try (TsFileGenerator generator = new TsFileGenerator(new File(tmpDir, "1-0-0-0.tsfile"))) { + generator.registerTimeseries( + new Path(SchemaConfig.DEVICE_0), + Arrays.asList( + SchemaConfig.MEASUREMENT_00, + SchemaConfig.MEASUREMENT_01, + SchemaConfig.MEASUREMENT_02, + SchemaConfig.MEASUREMENT_03)); + generator.registerAlignedTimeseries( + new Path(SchemaConfig.DEVICE_1), + Arrays.asList( + SchemaConfig.MEASUREMENT_10, + SchemaConfig.MEASUREMENT_11, + SchemaConfig.MEASUREMENT_12, + SchemaConfig.MEASUREMENT_13)); + generator.generateData(new Path(SchemaConfig.DEVICE_0), 100000, false); + generator.generateData(new Path(SchemaConfig.DEVICE_1), 100000, true); + writtenPoint1 = generator.getTotalNumber(); + } + + long writtenPoint2 = 0; + // device 2, device 3, device4, sg 1 + try (TsFileGenerator generator = new TsFileGenerator(new File(tmpDir, "2-0-0-0.tsfile"))) { + generator.registerTimeseries( + new Path(SchemaConfig.DEVICE_2), Arrays.asList(SchemaConfig.MEASUREMENT_20)); + generator.registerTimeseries( + new Path(SchemaConfig.DEVICE_3), Arrays.asList(SchemaConfig.MEASUREMENT_30)); + generator.registerAlignedTimeseries( + new Path(SchemaConfig.DEVICE_4), Arrays.asList(SchemaConfig.MEASUREMENT_40)); + generator.generateData(new Path(SchemaConfig.DEVICE_2), 10000, false); + generator.generateData(new Path(SchemaConfig.DEVICE_3), 10000, false); + generator.generateData(new Path(SchemaConfig.DEVICE_4), 10000, true); + writtenPoint2 = generator.getTotalNumber(); + } + + try (Connection connection = EnvFactory.getEnv().getConnection(); + Statement statement = connection.createStatement()) { + + statement.execute(String.format("load \"%s\"", tmpDir.getAbsolutePath())); + + try (ResultSet resultSet = + statement.executeQuery("select count(*) from root.** group by level=1,2")) { + if (resultSet.next()) { + long sg1Count = resultSet.getLong("count(root.sg.test_0.*.*)"); + Assert.assertEquals(writtenPoint1, sg1Count); + long sg2Count = resultSet.getLong("count(root.sg.test_1.*.*)"); + Assert.assertEquals(writtenPoint2, sg2Count); + } else { + Assert.fail("This ResultSet is empty."); + } + } + } + } + + private static class SchemaConfig { + private static final String STORAGE_GROUP_0 = "root.sg.test_0"; + private static final String STORAGE_GROUP_1 = "root.sg.test_1"; + + // device 0, nonaligned, sg 0 + private static final String DEVICE_0 = "root.sg.test_0.d_0"; + private static final MeasurementSchema MEASUREMENT_00 = + new MeasurementSchema("sensor_00", TSDataType.INT32); + private static final MeasurementSchema MEASUREMENT_01 = + new MeasurementSchema("sensor_01", TSDataType.INT64); + private static final MeasurementSchema MEASUREMENT_02 = + new MeasurementSchema("sensor_02", TSDataType.DOUBLE); + private static final MeasurementSchema MEASUREMENT_03 = + new MeasurementSchema("sensor_03", TSDataType.TEXT); + + // device 1, aligned, sg 0 + private static final String DEVICE_1 = "root.sg.test_0.a_1"; + private static final MeasurementSchema MEASUREMENT_10 = + new MeasurementSchema("sensor_10", TSDataType.INT32); + private static final MeasurementSchema MEASUREMENT_11 = + new MeasurementSchema("sensor_11", TSDataType.INT64); + private static final MeasurementSchema MEASUREMENT_12 = + new MeasurementSchema("sensor_12", TSDataType.DOUBLE); + private static final MeasurementSchema MEASUREMENT_13 = + new MeasurementSchema("sensor_13", TSDataType.TEXT); + + // device 2, non aligned, sg 1 + private static final String DEVICE_2 = "root.sg.test_1.d_2"; + private static final MeasurementSchema MEASUREMENT_20 = + new MeasurementSchema("sensor_20", TSDataType.INT32); + + // device 3, non aligned, sg 1 + private static final String DEVICE_3 = "root.sg.test_1.d_3"; + private static final MeasurementSchema MEASUREMENT_30 = + new MeasurementSchema("sensor_30", TSDataType.INT32); + + // device 4, aligned, sg 1 + private static final String DEVICE_4 = "root.sg.test_1.a_4"; + private static final MeasurementSchema MEASUREMENT_40 = + new MeasurementSchema("sensor_40", TSDataType.INT32); + } + + public class TsFileGenerator implements AutoCloseable { + private final File tsFile; + private final TsFileWriter writer; + private final Map device2Number; + private final Map> device2MeasurementSchema; + private Random random; + private long totalNumber; + + public TsFileGenerator(File tsFile) throws IOException { + this.tsFile = tsFile; + this.writer = new TsFileWriter(tsFile); + this.device2Number = new HashMap<>(); + this.device2MeasurementSchema = new HashMap<>(); + this.random = new Random(); + this.totalNumber = 0; + } + + public void resetRandom() { + random = new Random(); + } + + public void resetRandom(long seed) { + random = new Random(seed); + } + + public void registerTimeseries(Path path, List measurementSchemaList) { + if (device2Number.containsKey(path)) { + LOGGER.error(String.format("Register same device %s.", path)); + return; + } + writer.registerTimeseries(path, measurementSchemaList); + device2Number.put(path, 0); + device2MeasurementSchema.put(path, measurementSchemaList); + } + + public void registerAlignedTimeseries(Path path, List measurementSchemaList) + throws WriteProcessException { + if (device2Number.containsKey(path)) { + LOGGER.error(String.format("Register same device %s.", path)); + return; + } + writer.registerAlignedTimeseries(path, measurementSchemaList); + device2Number.put(path, 0); + device2MeasurementSchema.put(path, measurementSchemaList); + } + + public void generateData(Path path, int number, boolean isAligned) + throws IOException, WriteProcessException { + List schemas = device2MeasurementSchema.get(path); + Tablet tablet = new Tablet(path.getFullPath(), schemas); + long[] timestamps = tablet.timestamps; + Object[] values = tablet.values; + long sensorNum = schemas.size(); + long startTime = device2Number.get(path); + + for (long r = 0; r < number; r++) { + int row = tablet.rowSize++; + timestamps[row] = startTime++; + for (int i = 0; i < sensorNum; i++) { + generateDataPoint(values[i], row, schemas.get(i)); + totalNumber += 1; + } + // write + if (tablet.rowSize == tablet.getMaxRowNumber()) { + if (!isAligned) { + writer.write(tablet); + } else { + writer.writeAligned(tablet); + } + tablet.reset(); + } + } + // write + if (tablet.rowSize != 0) { + if (!isAligned) { + writer.write(tablet); + } else { + writer.writeAligned(tablet); + } + tablet.reset(); + } + device2Number.compute(path, (k, v) -> v + number); + } + + private void generateDataPoint(Object obj, int row, MeasurementSchema schema) { + switch (schema.getType()) { + case INT32: + generateINT32(obj, row); + break; + case INT64: + generateINT64(obj, row); + break; + case FLOAT: + generateFLOAT(obj, row); + break; + case DOUBLE: + generateDOUBLE(obj, row); + break; + case BOOLEAN: + generateBOOLEAN(obj, row); + break; + case TEXT: + generateTEXT(obj, row); + break; + default: + LOGGER.error(String.format("Wrong data type %s.", schema.getType())); + } + } + + private void generateINT32(Object obj, int row) { + int[] ints = (int[]) obj; + ints[row] = random.nextInt(); + } + + private void generateINT64(Object obj, int row) { + long[] longs = (long[]) obj; + longs[row] = random.nextLong(); + } + + private void generateFLOAT(Object obj, int row) { + float[] floats = (float[]) obj; + floats[row] = random.nextFloat(); + } + + private void generateDOUBLE(Object obj, int row) { + double[] doubles = (double[]) obj; + doubles[row] = random.nextDouble(); + } + + private void generateBOOLEAN(Object obj, int row) { + boolean[] booleans = (boolean[]) obj; + booleans[row] = random.nextBoolean(); + } + + private void generateTEXT(Object obj, int row) { + Binary[] binaries = (Binary[]) obj; + binaries[row] = new Binary(String.format("test point %d", random.nextInt())); + } + + public int getNumber(Path path) { + return device2Number.get(path); + } + + public long getTotalNumber() { + return totalNumber; + } + + @Override + public void close() throws Exception { + writer.close(); + } + } +} diff --git a/integration-test/src/test/java/org/apache/iotdb/db/it/env/StandaloneEnvConfig.java b/integration-test/src/test/java/org/apache/iotdb/db/it/env/StandaloneEnvConfig.java index 30d21973a55e..fb2b655d8cc4 100644 --- a/integration-test/src/test/java/org/apache/iotdb/db/it/env/StandaloneEnvConfig.java +++ b/integration-test/src/test/java/org/apache/iotdb/db/it/env/StandaloneEnvConfig.java @@ -60,6 +60,11 @@ public BaseConfig setDataRegionNum(int dataRegionNum) { return this; } + @Override + public boolean isEnablePartition() { + return IoTDBDescriptor.getInstance().getConfig().isEnablePartition(); + } + @Override public BaseConfig setPartitionInterval(long partitionInterval) { IoTDBDescriptor.getInstance().getConfig().setPartitionInterval(partitionInterval); diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/conf/IoTDBConstant.java b/node-commons/src/main/java/org/apache/iotdb/commons/conf/IoTDBConstant.java index 7a17f3e53a3c..627fcbe4dacd 100644 --- a/node-commons/src/main/java/org/apache/iotdb/commons/conf/IoTDBConstant.java +++ b/node-commons/src/main/java/org/apache/iotdb/commons/conf/IoTDBConstant.java @@ -188,6 +188,7 @@ private IoTDBConstant() {} // system folder name public static final String SYSTEM_FOLDER_NAME = "system"; public static final String SCHEMA_FOLDER_NAME = "schema"; + public static final String LOAD_TSFILE_FOLDER_NAME = "load"; public static final String SYNC_FOLDER_NAME = "sync"; public static final String QUERY_FOLDER_NAME = "query"; public static final String TRACING_FOLDER_NAME = "tracing"; diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/partition/DataPartition.java b/node-commons/src/main/java/org/apache/iotdb/commons/partition/DataPartition.java index 35e3a98a6f2e..5008e07cc600 100644 --- a/node-commons/src/main/java/org/apache/iotdb/commons/partition/DataPartition.java +++ b/node-commons/src/main/java/org/apache/iotdb/commons/partition/DataPartition.java @@ -97,6 +97,14 @@ public List getDataRegionReplicaSetForWriting( .collect(Collectors.toList()); } + public List getAllDataRegionReplicaSetForOneDevice(String deviceName) { + String storageGroup = getStorageGroupByDevice(deviceName); + TSeriesPartitionSlot seriesPartitionSlot = calculateDeviceGroupId(deviceName); + return dataPartitionMap.get(storageGroup).get(seriesPartitionSlot).entrySet().stream() + .flatMap(entry -> entry.getValue().stream()) + .collect(Collectors.toList()); + } + public TRegionReplicaSet getDataRegionReplicaSetForWriting( String deviceName, TTimePartitionSlot timePartitionSlot) { // A list of data region replica sets will store data in a same time partition. diff --git a/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java b/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java index 16b44ec6d3df..9e14dc37ed48 100644 --- a/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java +++ b/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java @@ -32,6 +32,8 @@ import org.apache.iotdb.db.engine.storagegroup.timeindex.TimeIndexLevel; import org.apache.iotdb.db.exception.LoadConfigurationException; import org.apache.iotdb.db.metadata.LocalSchemaProcessor; +import org.apache.iotdb.db.mpp.plan.planner.plan.node.PlanNode; +import org.apache.iotdb.db.mpp.plan.planner.plan.node.load.LoadTsFileNode; import org.apache.iotdb.db.service.thrift.impl.InfluxDBServiceImpl; import org.apache.iotdb.db.service.thrift.impl.TSServiceImpl; import org.apache.iotdb.db.wal.utils.WALMode; @@ -292,6 +294,9 @@ public class IoTDBConfig { IoTDBConstant.DEFAULT_BASE_DIR + File.separator + IoTDBConstant.DATA_FOLDER_NAME }; + private String loadTsFileDir = + dataDirs[0] + File.separator + IoTDBConstant.LOAD_TSFILE_FOLDER_NAME; + /** Strategy of multiple directories. */ private String multiDirStrategyClassName = null; @@ -750,6 +755,9 @@ public class IoTDBConfig { */ private long partitionInterval = 86400; + /** Max size of a {@link PlanNode}, mainly used to control memory of {@link LoadTsFileNode}. */ + private long maxPlanNodeSize = 500 * 1048576L; + /** * Level of TimeIndex, which records the start time and end time of TsFileResource. Currently, * DEVICE_TIME_INDEX and FILE_TIME_INDEX are supported, and could not be changed after first set. @@ -1081,6 +1089,10 @@ public void setPartitionInterval(long partitionInterval) { this.partitionInterval = partitionInterval; } + public long getMaxPlanNodeSize() { + return maxPlanNodeSize; + } + public TimeIndexLevel getTimeIndexLevel() { return timeIndexLevel; } @@ -1098,6 +1110,7 @@ void updatePath() { private void formulateFolders() { systemDir = addHomeDir(systemDir); schemaDir = addHomeDir(schemaDir); + loadTsFileDir = addHomeDir(loadTsFileDir); tracingDir = addHomeDir(tracingDir); consensusDir = addHomeDir(consensusDir); dataRegionConsensusDir = addHomeDir(dataRegionConsensusDir); @@ -1259,6 +1272,14 @@ void setSystemDir(String systemDir) { this.systemDir = systemDir; } + public String getLoadTsFileDir() { + return loadTsFileDir; + } + + public void setLoadTsFileDir(String loadTsFileDir) { + this.loadTsFileDir = loadTsFileDir; + } + public String getSchemaDir() { return schemaDir; } diff --git a/server/src/main/java/org/apache/iotdb/db/engine/StorageEngineV2.java b/server/src/main/java/org/apache/iotdb/db/engine/StorageEngineV2.java index cbe9d5b85f29..b81665d4af66 100644 --- a/server/src/main/java/org/apache/iotdb/db/engine/StorageEngineV2.java +++ b/server/src/main/java/org/apache/iotdb/db/engine/StorageEngineV2.java @@ -41,14 +41,18 @@ import org.apache.iotdb.db.engine.flush.FlushListener; import org.apache.iotdb.db.engine.flush.TsFileFlushPolicy; import org.apache.iotdb.db.engine.flush.TsFileFlushPolicy.DirectFlushPolicy; +import org.apache.iotdb.db.engine.load.LoadTsFileManager; import org.apache.iotdb.db.engine.storagegroup.DataRegion; import org.apache.iotdb.db.engine.storagegroup.TsFileProcessor; import org.apache.iotdb.db.exception.DataRegionException; +import org.apache.iotdb.db.exception.LoadFileException; import org.apache.iotdb.db.exception.StorageEngineException; import org.apache.iotdb.db.exception.TsFileProcessorException; import org.apache.iotdb.db.exception.WriteProcessRejectException; import org.apache.iotdb.db.exception.runtime.StorageEngineFailureException; import org.apache.iotdb.db.mpp.plan.planner.plan.node.PlanNode; +import org.apache.iotdb.db.mpp.plan.planner.plan.node.load.LoadTsFilePieceNode; +import org.apache.iotdb.db.mpp.plan.scheduler.load.LoadTsFileScheduler; import org.apache.iotdb.db.rescon.SystemInfo; import org.apache.iotdb.db.sync.SyncService; import org.apache.iotdb.db.utils.ThreadUtils; @@ -58,6 +62,7 @@ import org.apache.iotdb.db.wal.recover.WALRecoverManager; import org.apache.iotdb.rpc.RpcUtils; import org.apache.iotdb.rpc.TSStatusCode; +import org.apache.iotdb.tsfile.exception.write.PageException; import org.apache.iotdb.tsfile.utils.FilePathUtils; import org.apache.commons.io.FileUtils; @@ -133,6 +138,8 @@ public class StorageEngineV2 implements IService { private List customFlushListeners = new ArrayList<>(); private int recoverDataRegionNum = 0; + private LoadTsFileManager loadTsFileManager = new LoadTsFileManager(); + private StorageEngineV2() {} public static StorageEngineV2 getInstance() { @@ -734,6 +741,71 @@ public TsFileFlushPolicy getFileFlushPolicy() { return fileFlushPolicy; } + public TSStatus writeLoadTsFileNode( + DataRegionId dataRegionId, LoadTsFilePieceNode pieceNode, String uuid) { + TSStatus status = new TSStatus(); + + try { + loadTsFileManager.writeToDataRegion(getDataRegion(dataRegionId), pieceNode, uuid); + } catch (PageException e) { + logger.error( + String.format( + "Parse Page error when writing piece node of TsFile %s to DataRegion %s.", + pieceNode.getTsFile(), dataRegionId), + e); + status.setCode(TSStatusCode.TSFILE_RUNTIME_ERROR.getStatusCode()); + status.setMessage(e.getMessage()); + return status; + } catch (IOException e) { + logger.error( + String.format( + "IO error when writing piece node of TsFile %s to DataRegion %s.", + pieceNode.getTsFile(), dataRegionId), + e); + status.setCode(TSStatusCode.DATA_REGION_ERROR.getStatusCode()); + status.setMessage(e.getMessage()); + return status; + } + + return RpcUtils.SUCCESS_STATUS; + } + + public TSStatus executeLoadCommand(LoadTsFileScheduler.LoadCommand loadCommand, String uuid) { + TSStatus status = new TSStatus(); + + try { + switch (loadCommand) { + case EXECUTE: + if (loadTsFileManager.loadAll(uuid)) { + status.setCode(TSStatusCode.SUCCESS_STATUS.getStatusCode()); + } else { + status.setCode(TSStatusCode.LOAD_FILE_ERROR.getStatusCode()); + status.setMessage(String.format("No uuid %s recorded.", uuid)); + } + break; + case ROLLBACK: + if (loadTsFileManager.deleteAll(uuid)) { + status.setCode(TSStatusCode.SUCCESS_STATUS.getStatusCode()); + } else { + status.setCode(TSStatusCode.LOAD_FILE_ERROR.getStatusCode()); + status.setMessage(String.format("No uuid %s recorded.", uuid)); + } + break; + default: + status.setCode(TSStatusCode.ILLEGAL_PARAMETER.getStatusCode()); + status.setMessage(String.format("Wrong load command %s.", loadCommand)); + } + } catch (IOException e) { + status.setCode(TSStatusCode.DATA_REGION_ERROR.getStatusCode()); + status.setMessage(e.getMessage()); + } catch (LoadFileException e) { + status.setCode(TSStatusCode.LOAD_FILE_ERROR.getStatusCode()); + status.setMessage(e.getMessage()); + } + + return RpcUtils.SUCCESS_STATUS; + } + static class InstanceHolder { private static final StorageEngineV2 INSTANCE = new StorageEngineV2(); diff --git a/server/src/main/java/org/apache/iotdb/db/engine/load/AlignedChunkData.java b/server/src/main/java/org/apache/iotdb/db/engine/load/AlignedChunkData.java new file mode 100644 index 000000000000..7a2a3eb77683 --- /dev/null +++ b/server/src/main/java/org/apache/iotdb/db/engine/load/AlignedChunkData.java @@ -0,0 +1,497 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.engine.load; + +import org.apache.iotdb.common.rpc.thrift.TTimePartitionSlot; +import org.apache.iotdb.db.engine.StorageEngineV2; +import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor; +import org.apache.iotdb.tsfile.encoding.decoder.Decoder; +import org.apache.iotdb.tsfile.exception.write.PageException; +import org.apache.iotdb.tsfile.exception.write.UnSupportedDataTypeException; +import org.apache.iotdb.tsfile.file.MetaMarker; +import org.apache.iotdb.tsfile.file.header.ChunkHeader; +import org.apache.iotdb.tsfile.file.header.PageHeader; +import org.apache.iotdb.tsfile.file.metadata.IChunkMetadata; +import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType; +import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding; +import org.apache.iotdb.tsfile.file.metadata.statistics.Statistics; +import org.apache.iotdb.tsfile.read.TsFileSequenceReader; +import org.apache.iotdb.tsfile.read.common.Chunk; +import org.apache.iotdb.tsfile.read.reader.page.TimePageReader; +import org.apache.iotdb.tsfile.read.reader.page.ValuePageReader; +import org.apache.iotdb.tsfile.utils.Binary; +import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils; +import org.apache.iotdb.tsfile.utils.TsPrimitiveType; +import org.apache.iotdb.tsfile.write.chunk.AlignedChunkWriterImpl; +import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema; +import org.apache.iotdb.tsfile.write.schema.MeasurementSchema; +import org.apache.iotdb.tsfile.write.writer.TsFileIOWriter; + +import java.io.DataOutputStream; +import java.io.File; +import java.io.IOException; +import java.io.InputStream; +import java.io.Serializable; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.List; + +public class AlignedChunkData implements ChunkData { + private static final int DEFAULT_INT32 = 0; + private static final long DEFAULT_INT64 = 0L; + private static final float DEFAULT_FLOAT = 0; + private static final double DEFAULT_DOUBLE = 0.0; + private static final boolean DEFAULT_BOOLEAN = false; + private static final Binary DEFAULT_BINARY = null; + + private List offset; + private List dataSize; + private boolean isHeadPageNeedDecode; + private boolean isTailPageNeedDecode; + + private TTimePartitionSlot timePartitionSlot; + private String device; + private List chunkHeaderList; + private List chunkMetadataList; + + private List timeBatch; + private List satisfiedTimeBatchLength; + + private AlignedChunkWriterImpl chunkWriter; + private List chunkList; + + public AlignedChunkData(long timeOffset, String device, ChunkHeader chunkHeader) { + this.offset = new ArrayList<>(); + this.dataSize = new ArrayList<>(); + this.isHeadPageNeedDecode = false; + this.isTailPageNeedDecode = false; + this.device = device; + this.chunkHeaderList = new ArrayList<>(); + + offset.add(timeOffset); + dataSize.add(0L); + chunkHeaderList.add(chunkHeader); + } + + @Override + public String getDevice() { + return device; + } + + @Override + public TTimePartitionSlot getTimePartitionSlot() { + return timePartitionSlot; + } + + @Override + public long getDataSize() { + return dataSize.stream().mapToLong(o -> o).sum(); + } + + @Override + public void addDataSize(long pageSize) { + dataSize.set(0, dataSize.get(0) + pageSize); + } + + @Override + public void setNotDecode(IChunkMetadata chunkMetadata) { + chunkMetadataList = new ArrayList<>(); + chunkMetadataList.add(chunkMetadata); + } + + @Override + public boolean needDecodeChunk() { + return chunkMetadataList == null; + } + + @Override + public void setHeadPageNeedDecode(boolean headPageNeedDecode) { + isHeadPageNeedDecode = headPageNeedDecode; + } + + @Override + public void setTailPageNeedDecode(boolean tailPageNeedDecode) { + isTailPageNeedDecode = tailPageNeedDecode; + } + + @Override + public void setTimePartitionSlot(TTimePartitionSlot timePartitionSlot) { + this.timePartitionSlot = timePartitionSlot; + } + + @Override + public boolean isAligned() { + return true; + } + + @Override + public void writeToFileWriter(TsFileIOWriter writer) throws IOException { + if (chunkList != null) { + for (Chunk chunk : chunkList) { + writer.writeChunk(chunk); + } + } else { + chunkWriter.writeToFileWriter(writer); + } + } + + public void addValueChunk(long offset, ChunkHeader chunkHeader, IChunkMetadata chunkMetadata) { + this.offset.add(offset); + this.dataSize.add(0L); + this.chunkHeaderList.add(chunkHeader); + if (chunkMetadataList != null) { + chunkMetadataList.add(chunkMetadata); + } + } + + public void addValueChunkDataSize(long dataSize) { + int lastIndex = this.dataSize.size() - 1; + this.dataSize.set(lastIndex, this.dataSize.get(lastIndex) + dataSize); + } + + @Override + public void serialize(DataOutputStream stream, File tsFile) throws IOException { + ReadWriteIOUtils.write(isAligned(), stream); + serializeAttr(stream); + serializeTsFileData(stream, tsFile); + } + + private void serializeAttr(DataOutputStream stream) throws IOException { + ReadWriteIOUtils.write(timePartitionSlot.getStartTime(), stream); + ReadWriteIOUtils.write(device, stream); + ReadWriteIOUtils.write(chunkHeaderList.size(), stream); + for (ChunkHeader chunkHeader : chunkHeaderList) { + chunkHeader.serializeTo(stream); // chunk header already serialize chunk type + } + } + + private void serializeTsFileData(DataOutputStream stream, File tsFile) throws IOException { + timeBatch = new ArrayList<>(); + satisfiedTimeBatchLength = new ArrayList<>(); + ReadWriteIOUtils.write(needDecodeChunk(), stream); + try (TsFileSequenceReader reader = new TsFileSequenceReader(tsFile.getAbsolutePath())) { + int chunkSize = offset.size(); + for (int i = 0; i < chunkSize; i++) { + if (needDecodeChunk()) { + serializeDecodeChunk(stream, reader, chunkHeaderList.get(i), i); + } else { + serializeEntireChunk(stream, reader, chunkHeaderList.get(i), chunkMetadataList.get(i)); + } + } + } + timeBatch = null; + satisfiedTimeBatchLength = null; + } + + private void serializeEntireChunk( + DataOutputStream stream, + TsFileSequenceReader reader, + ChunkHeader chunkHeader, + IChunkMetadata chunkMetadata) + throws IOException { + ByteBuffer chunkData = + reader.readChunk( + chunkMetadata.getOffsetOfChunkHeader() + chunkHeader.getSerializedSize(), + chunkHeader.getDataSize()); + ReadWriteIOUtils.write(chunkData, stream); + chunkMetadata.getStatistics().serialize(stream); + } + + private void serializeDecodeChunk( + DataOutputStream stream, TsFileSequenceReader reader, ChunkHeader chunkHeader, int chunkIndex) + throws IOException { + Decoder defaultTimeDecoder = + Decoder.getDecoderByType( + TSEncoding.valueOf(TSFileDescriptor.getInstance().getConfig().getTimeEncoder()), + TSDataType.INT64); + Decoder valueDecoder = + Decoder.getDecoderByType(chunkHeader.getEncodingType(), chunkHeader.getDataType()); + + reader.position(offset.get(chunkIndex)); + int decodePageIndex = 0; // should be 0,1 or 2 + long dataSize = this.dataSize.get(chunkIndex); + while (dataSize > 0) { + boolean hasStatistic = (chunkHeader.getChunkType() & 0x3F) == MetaMarker.CHUNK_HEADER; + PageHeader pageHeader = reader.readPageHeader(chunkHeader.getDataType(), hasStatistic); + long pageDataSize = pageHeader.getSerializedPageSize(); + if ((dataSize == this.dataSize.get(chunkIndex) && isHeadPageNeedDecode) // decode head page + || (dataSize == pageDataSize && isTailPageNeedDecode)) { // decode tail page + ReadWriteIOUtils.write(true, stream); // decode + if (chunkIndex == 0) { + decodeTimePage(reader, chunkHeader, pageHeader, defaultTimeDecoder, valueDecoder, stream); + } else { + decodeValuePage(reader, chunkHeader, pageHeader, decodePageIndex, valueDecoder, stream); + } + decodePageIndex += 1; + } else { // entire page + ReadWriteIOUtils.write(false, stream); // don't decode + pageHeader.serializeTo(stream); + ByteBuffer pageData = reader.readCompressedPage(pageHeader); + ReadWriteIOUtils.write(pageData, stream); + } + dataSize -= pageDataSize; + } + + ReadWriteIOUtils.write(true, stream); // means ending + ReadWriteIOUtils.write(-1, stream); + } + + private void decodeTimePage( + TsFileSequenceReader reader, + ChunkHeader chunkHeader, + PageHeader pageHeader, + Decoder timeDecoder, + Decoder valueDecoder, + DataOutputStream stream) + throws IOException { + valueDecoder.reset(); + ByteBuffer pageData = reader.readPage(pageHeader, chunkHeader.getCompressionType()); + TimePageReader timePageReader = new TimePageReader(pageHeader, pageData, timeDecoder); + long[] decodeTime = timePageReader.getNextTimeBatch(); + int satisfiedLength = 0; + long[] time = new long[decodeTime.length]; + for (int i = 0; i < decodeTime.length; i++) { + if (decodeTime[i] < timePartitionSlot.getStartTime()) { + continue; + } else if (!timePartitionSlot.equals(StorageEngineV2.getTimePartitionSlot(decodeTime[i]))) { + break; + } + time[satisfiedLength++] = decodeTime[i]; + } + ReadWriteIOUtils.write(satisfiedLength, stream); + for (int i = 0; i < satisfiedLength; i++) { + ReadWriteIOUtils.write(time[i], stream); + } + timeBatch.add(decodeTime); + satisfiedTimeBatchLength.add(satisfiedLength); + } + + private void decodeValuePage( + TsFileSequenceReader reader, + ChunkHeader chunkHeader, + PageHeader pageHeader, + int pageIndex, + Decoder valueDecoder, + DataOutputStream stream) + throws IOException { + valueDecoder.reset(); + ByteBuffer pageData = reader.readPage(pageHeader, chunkHeader.getCompressionType()); + ValuePageReader valuePageReader = + new ValuePageReader(pageHeader, pageData, chunkHeader.getDataType(), valueDecoder); + long[] time = timeBatch.get(pageIndex); + TsPrimitiveType[] valueBatch = + valuePageReader.nextValueBatch( + time); // should be origin time, so recording satisfied length is necessary + ReadWriteIOUtils.write(satisfiedTimeBatchLength.get(pageIndex), stream); + for (int i = 0; i < valueBatch.length; i++) { + if (time[i] < timePartitionSlot.getStartTime()) { + continue; + } else if (!timePartitionSlot.equals(StorageEngineV2.getTimePartitionSlot(time[i]))) { + break; + } + if (valueBatch[i] == null) { + ReadWriteIOUtils.write(true, stream); + continue; + } + ReadWriteIOUtils.write(false, stream); + switch (chunkHeader.getDataType()) { + case INT32: + ReadWriteIOUtils.write(valueBatch[i].getInt(), stream); + break; + case INT64: + ReadWriteIOUtils.write(valueBatch[i].getLong(), stream); + break; + case FLOAT: + ReadWriteIOUtils.write(valueBatch[i].getFloat(), stream); + break; + case DOUBLE: + ReadWriteIOUtils.write(valueBatch[i].getDouble(), stream); + break; + case BOOLEAN: + ReadWriteIOUtils.write(valueBatch[i].getBoolean(), stream); + break; + case TEXT: + ReadWriteIOUtils.write(valueBatch[i].getBinary(), stream); + break; + default: + throw new UnSupportedDataTypeException( + String.format("Data type %s is not supported.", chunkHeader.getDataType())); + } + } + } + + private void deserializeTsFileData(InputStream stream) throws IOException, PageException { + boolean needDecodeChunk = ReadWriteIOUtils.readBool(stream); + if (needDecodeChunk) { + buildChunkWriter(stream); + } else { + deserializeEntireChunk(stream); + } + } + + private void deserializeEntireChunk(InputStream stream) throws IOException { + chunkList = new ArrayList<>(); + int chunkSize = chunkHeaderList.size(); + for (int i = 0; i < chunkSize; i++) { + ByteBuffer chunkData = + ByteBuffer.wrap(ReadWriteIOUtils.readBytesWithSelfDescriptionLength(stream)); + Statistics statistics = + Statistics.deserialize(stream, chunkHeaderList.get(i).getDataType()); + chunkList.add(new Chunk(chunkHeaderList.get(i), chunkData, null, statistics)); + } + } + + private void buildChunkWriter(InputStream stream) throws IOException, PageException { + List measurementSchemaList = new ArrayList<>(); + for (ChunkHeader chunkHeader : chunkHeaderList) { + if (TSDataType.VECTOR.equals(chunkHeader.getDataType())) { + continue; + } + measurementSchemaList.add( + new MeasurementSchema( + chunkHeader.getMeasurementID(), + chunkHeader.getDataType(), + chunkHeader.getEncodingType(), + chunkHeader.getCompressionType())); + } + chunkWriter = new AlignedChunkWriterImpl(measurementSchemaList); + timeBatch = new ArrayList<>(); + int chunkHeaderSize = chunkHeaderList.size(); + for (int i = 0; i < chunkHeaderSize; i++) { + buildChunk(stream, chunkHeaderList.get(i), i - 1, i == 0); + } + timeBatch = null; + } + + private void buildChunk( + InputStream stream, ChunkHeader chunkHeader, int valueChunkIndex, boolean isTimeChunk) + throws IOException, PageException { + boolean needDecode; + int decodePageIndex = 0; + while (true) { + needDecode = ReadWriteIOUtils.readBool(stream); + if (needDecode) { + int length = ReadWriteIOUtils.readInt(stream); + if (length == -1) { + break; + } + + long[] timePageBatch = new long[length]; + if (!isTimeChunk) { + timePageBatch = timeBatch.get(decodePageIndex); + } + for (int i = 0; i < length; i++) { + if (isTimeChunk) { + long time = ReadWriteIOUtils.readLong(stream); + timePageBatch[i] = time; + chunkWriter.write(time); + } else { + boolean isNull = ReadWriteIOUtils.readBool(stream); + switch (chunkHeader.getDataType()) { + case INT32: + int int32Value = isNull ? DEFAULT_INT32 : ReadWriteIOUtils.readInt(stream); + chunkWriter.write(timePageBatch[i], int32Value, isNull, valueChunkIndex); + break; + case INT64: + long int64Value = isNull ? DEFAULT_INT64 : ReadWriteIOUtils.readLong(stream); + chunkWriter.write(timePageBatch[i], int64Value, isNull, valueChunkIndex); + break; + case FLOAT: + float floatValue = isNull ? DEFAULT_FLOAT : ReadWriteIOUtils.readFloat(stream); + chunkWriter.write(timePageBatch[i], floatValue, isNull, valueChunkIndex); + break; + case DOUBLE: + double doubleValue = isNull ? DEFAULT_DOUBLE : ReadWriteIOUtils.readDouble(stream); + chunkWriter.write(timePageBatch[i], doubleValue, isNull, valueChunkIndex); + break; + case BOOLEAN: + boolean boolValue = isNull ? DEFAULT_BOOLEAN : ReadWriteIOUtils.readBool(stream); + chunkWriter.write(timePageBatch[i], boolValue, isNull, valueChunkIndex); + break; + case TEXT: + Binary binaryValue = isNull ? DEFAULT_BINARY : ReadWriteIOUtils.readBinary(stream); + chunkWriter.write(timePageBatch[i], binaryValue, isNull, valueChunkIndex); + break; + default: + throw new UnSupportedDataTypeException( + String.format("Data type %s is not supported.", chunkHeader.getDataType())); + } + } + } + if (isTimeChunk) { + timeBatch.add(timePageBatch); + } + decodePageIndex += 1; + } else { + PageHeader pageHeader = PageHeader.deserializeFrom(stream, chunkHeader.getDataType(), true); + if (isTimeChunk) { + chunkWriter.writePageHeaderAndDataIntoTimeBuff( + ByteBuffer.wrap(ReadWriteIOUtils.readBytesWithSelfDescriptionLength(stream)), + pageHeader); + } else { + chunkWriter.writePageHeaderAndDataIntoValueBuff( + ByteBuffer.wrap(ReadWriteIOUtils.readBytesWithSelfDescriptionLength(stream)), + pageHeader, + valueChunkIndex); + } + } + } + } + + public static AlignedChunkData deserialize(InputStream stream) throws IOException, PageException { + long timePartition = ReadWriteIOUtils.readLong(stream); + String device = ReadWriteIOUtils.readString(stream); + int chunkHeaderListSize = ReadWriteIOUtils.readInt(stream); + ChunkHeader[] chunkHeaderList = new ChunkHeader[chunkHeaderListSize]; + for (int i = 0; i < chunkHeaderListSize; i++) { + byte chunkType = ReadWriteIOUtils.readByte(stream); + chunkHeaderList[i] = ChunkHeader.deserializeFrom(stream, chunkType); + } + + AlignedChunkData chunkData = new AlignedChunkData(-1, device, chunkHeaderList[0]); + for (int i = 1; i < chunkHeaderListSize; i++) { + chunkData.addValueChunk(-1, chunkHeaderList[i], null); + } + chunkData.setTimePartitionSlot(StorageEngineV2.getTimePartitionSlot(timePartition)); + chunkData.deserializeTsFileData(stream); + return chunkData; + } + + @Override + public String toString() { + return "AlignedChunkData{" + + "offset=" + + offset + + ", dataSize=" + + dataSize + + ", isHeadPageNeedDecode=" + + isHeadPageNeedDecode + + ", isTailPageNeedDecode=" + + isTailPageNeedDecode + + ", timePartitionSlot=" + + timePartitionSlot + + ", device='" + + device + + '\'' + + ", chunkHeaderList=" + + chunkHeaderList + + '}'; + } +} diff --git a/server/src/main/java/org/apache/iotdb/db/engine/load/ChunkData.java b/server/src/main/java/org/apache/iotdb/db/engine/load/ChunkData.java new file mode 100644 index 000000000000..36ab1e87fa02 --- /dev/null +++ b/server/src/main/java/org/apache/iotdb/db/engine/load/ChunkData.java @@ -0,0 +1,72 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.engine.load; + +import org.apache.iotdb.common.rpc.thrift.TTimePartitionSlot; +import org.apache.iotdb.tsfile.exception.write.PageException; +import org.apache.iotdb.tsfile.file.header.ChunkHeader; +import org.apache.iotdb.tsfile.file.metadata.IChunkMetadata; +import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils; +import org.apache.iotdb.tsfile.write.writer.TsFileIOWriter; + +import java.io.DataOutputStream; +import java.io.File; +import java.io.IOException; +import java.io.InputStream; + +public interface ChunkData { + String getDevice(); + + TTimePartitionSlot getTimePartitionSlot(); + + long getDataSize(); + + void addDataSize(long pageSize); + + void setNotDecode(IChunkMetadata chunkMetadata); + + boolean needDecodeChunk(); + + void setHeadPageNeedDecode(boolean headPageNeedDecode); + + void setTailPageNeedDecode(boolean tailPageNeedDecode); + + void setTimePartitionSlot(TTimePartitionSlot timePartitionSlot); + + boolean isAligned(); + + void writeToFileWriter(TsFileIOWriter writer) throws IOException; + + void serialize(DataOutputStream stream, File tsFile) throws IOException; + + static ChunkData deserialize(InputStream stream) throws PageException, IOException { + boolean isAligned = ReadWriteIOUtils.readBool(stream); + return isAligned + ? AlignedChunkData.deserialize(stream) + : NonAlignedChunkData.deserialize(stream); + } + + static ChunkData createChunkData( + boolean isAligned, long offset, String device, ChunkHeader chunkHeader) { + return isAligned + ? new AlignedChunkData(offset, device, chunkHeader) + : new NonAlignedChunkData(offset, device, chunkHeader); + } +} diff --git a/server/src/main/java/org/apache/iotdb/db/engine/load/LoadTsFileManager.java b/server/src/main/java/org/apache/iotdb/db/engine/load/LoadTsFileManager.java new file mode 100644 index 000000000000..48be7a395e2b --- /dev/null +++ b/server/src/main/java/org/apache/iotdb/db/engine/load/LoadTsFileManager.java @@ -0,0 +1,246 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.engine.load; + +import org.apache.iotdb.common.rpc.thrift.TTimePartitionSlot; +import org.apache.iotdb.commons.conf.IoTDBConstant; +import org.apache.iotdb.commons.file.SystemFileFactory; +import org.apache.iotdb.db.conf.IoTDBConfig; +import org.apache.iotdb.db.conf.IoTDBDescriptor; +import org.apache.iotdb.db.engine.storagegroup.DataRegion; +import org.apache.iotdb.db.engine.storagegroup.TsFileResource; +import org.apache.iotdb.db.engine.storagegroup.TsFileResourceStatus; +import org.apache.iotdb.db.exception.LoadFileException; +import org.apache.iotdb.db.mpp.plan.planner.plan.node.load.LoadTsFilePieceNode; +import org.apache.iotdb.db.mpp.plan.scheduler.load.LoadTsFileScheduler.LoadCommand; +import org.apache.iotdb.tsfile.common.constant.TsFileConstant; +import org.apache.iotdb.tsfile.exception.write.PageException; +import org.apache.iotdb.tsfile.file.metadata.TimeseriesMetadata; +import org.apache.iotdb.tsfile.write.writer.TsFileIOWriter; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.IOException; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.concurrent.locks.ReentrantLock; + +/** + * {@link LoadTsFileManager} is used for dealing with {@link LoadTsFilePieceNode} and {@link + * LoadCommand}. This class turn the content of a piece of loading TsFile into a new TsFile. When + * DataNode finish transfer pieces, this class will flush all TsFile and laod them into IoTDB, or + * delete all. + */ +public class LoadTsFileManager { + private static final Logger logger = LoggerFactory.getLogger(LoadTsFileManager.class); + private static final IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig(); + + private final File loadDir; + + private Map uuid2WriterManager; + private Map dataPartition2NextTsFileIndex; + + private final ReentrantLock lock; + + public LoadTsFileManager() { + this.loadDir = SystemFileFactory.INSTANCE.getFile(config.getLoadTsFileDir()); + this.uuid2WriterManager = new HashMap<>(); + this.dataPartition2NextTsFileIndex = new HashMap<>(); + this.lock = new ReentrantLock(); + + clearDir(loadDir); + } + + private void clearDir(File dir) { + if (dir.delete()) { + logger.info(String.format("Delete origin load TsFile dir %s.", dir.getPath())); + } + if (!dir.mkdirs()) { + logger.warn(String.format("load TsFile dir %s can not be created.", dir.getPath())); + } + } + + public void writeToDataRegion(DataRegion dataRegion, LoadTsFilePieceNode pieceNode, String uuid) + throws PageException, IOException { + TsFileWriterManager writerManager = + uuid2WriterManager.computeIfAbsent( + uuid, o -> new TsFileWriterManager(SystemFileFactory.INSTANCE.getFile(loadDir, uuid))); + for (ChunkData chunkData : pieceNode.getAllChunkData()) { + writerManager.write( + new DataPartitionInfo(dataRegion, chunkData.getTimePartitionSlot()), chunkData); + } + } + + public boolean loadAll(String uuid) throws IOException, LoadFileException { + if (!uuid2WriterManager.containsKey(uuid)) { + return false; + } + uuid2WriterManager.get(uuid).loadAll(); + uuid2WriterManager.get(uuid).close(); + uuid2WriterManager.remove(uuid); + return true; + } + + public boolean deleteAll(String uuid) throws IOException { + if (!uuid2WriterManager.containsKey(uuid)) { + return false; + } + uuid2WriterManager.get(uuid).close(); + uuid2WriterManager.remove(uuid); + return true; + } + + private String getNewTsFileName(String dataPartition) { + lock.lock(); + try { + int nextIndex = dataPartition2NextTsFileIndex.getOrDefault(dataPartition, 0) + 1; + dataPartition2NextTsFileIndex.put(dataPartition, nextIndex); + return dataPartition + + IoTDBConstant.FILE_NAME_SEPARATOR + + nextIndex + + TsFileConstant.TSFILE_SUFFIX; + } finally { + lock.unlock(); + } + } + + private class TsFileWriterManager { + private final File taskDir; + private Map dataPartition2Writer; + private Map dataPartition2LastDevice; + + private TsFileWriterManager(File taskDir) { + this.taskDir = taskDir; + this.dataPartition2Writer = new HashMap<>(); + this.dataPartition2LastDevice = new HashMap<>(); + + clearDir(taskDir); + } + + private void write(DataPartitionInfo partitionInfo, ChunkData chunkData) throws IOException { + if (!dataPartition2Writer.containsKey(partitionInfo)) { + File newTsFile = + SystemFileFactory.INSTANCE.getFile(taskDir, getNewTsFileName(partitionInfo.toString())); + if (!newTsFile.createNewFile()) { + logger.error(String.format("Can not create TsFile %s for writing.", newTsFile.getPath())); + return; + } + + dataPartition2Writer.put(partitionInfo, new TsFileIOWriter(newTsFile)); + } + TsFileIOWriter writer = dataPartition2Writer.get(partitionInfo); + if (!chunkData.getDevice().equals(dataPartition2LastDevice.getOrDefault(partitionInfo, ""))) { + if (dataPartition2LastDevice.containsKey(partitionInfo)) { + writer.endChunkGroup(); + } + writer.startChunkGroup(chunkData.getDevice()); + dataPartition2LastDevice.put(partitionInfo, chunkData.getDevice()); + } + chunkData.writeToFileWriter(writer); + } + + private void loadAll() throws IOException, LoadFileException { + for (Map.Entry entry : dataPartition2Writer.entrySet()) { + TsFileIOWriter writer = entry.getValue(); + if (writer.isWritingChunkGroup()) { + writer.endChunkGroup(); + } + writer.endFile(); + entry.getKey().getDataRegion().loadNewTsFile(generateResource(writer), true); + } + } + + private TsFileResource generateResource(TsFileIOWriter writer) throws IOException { + TsFileResource tsFileResource = new TsFileResource(writer.getFile()); + Map> deviceTimeseriesMetadataMap = + writer.getDeviceTimeseriesMetadataMap(); + for (Map.Entry> entry : + deviceTimeseriesMetadataMap.entrySet()) { + String device = entry.getKey(); + for (TimeseriesMetadata timeseriesMetaData : entry.getValue()) { + tsFileResource.updateStartTime(device, timeseriesMetaData.getStatistics().getStartTime()); + tsFileResource.updateEndTime(device, timeseriesMetaData.getStatistics().getEndTime()); + } + } + tsFileResource.setStatus(TsFileResourceStatus.CLOSED); + tsFileResource.serialize(); + return tsFileResource; + } + + private void close() throws IOException { + for (Map.Entry entry : dataPartition2Writer.entrySet()) { + TsFileIOWriter writer = entry.getValue(); + if (writer.canWrite()) { + entry.getValue().close(); + } + } + if (!taskDir.delete()) { + logger.warn(String.format("Can not delete load uuid dir %s.", taskDir.getPath())); + } + dataPartition2Writer = null; + dataPartition2LastDevice = null; + } + } + + private class DataPartitionInfo { + private final DataRegion dataRegion; + private final TTimePartitionSlot timePartitionSlot; + + private DataPartitionInfo(DataRegion dataRegion, TTimePartitionSlot timePartitionSlot) { + this.dataRegion = dataRegion; + this.timePartitionSlot = timePartitionSlot; + } + + public DataRegion getDataRegion() { + return dataRegion; + } + + public TTimePartitionSlot getTimePartitionSlot() { + return timePartitionSlot; + } + + @Override + public String toString() { + return String.join( + IoTDBConstant.FILE_NAME_SEPARATOR, + dataRegion.getStorageGroupName(), + dataRegion.getDataRegionId(), + Long.toString(timePartitionSlot.getStartTime())); + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + DataPartitionInfo that = (DataPartitionInfo) o; + return Objects.equals(dataRegion, that.dataRegion) + && timePartitionSlot.getStartTime() == that.timePartitionSlot.getStartTime(); + } + + @Override + public int hashCode() { + return Objects.hash(dataRegion, timePartitionSlot.getStartTime()); + } + } +} diff --git a/server/src/main/java/org/apache/iotdb/db/engine/load/NonAlignedChunkData.java b/server/src/main/java/org/apache/iotdb/db/engine/load/NonAlignedChunkData.java new file mode 100644 index 000000000000..5f0e9f084794 --- /dev/null +++ b/server/src/main/java/org/apache/iotdb/db/engine/load/NonAlignedChunkData.java @@ -0,0 +1,366 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.engine.load; + +import org.apache.iotdb.common.rpc.thrift.TTimePartitionSlot; +import org.apache.iotdb.db.engine.StorageEngineV2; +import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor; +import org.apache.iotdb.tsfile.encoding.decoder.Decoder; +import org.apache.iotdb.tsfile.exception.write.PageException; +import org.apache.iotdb.tsfile.exception.write.UnSupportedDataTypeException; +import org.apache.iotdb.tsfile.file.MetaMarker; +import org.apache.iotdb.tsfile.file.header.ChunkHeader; +import org.apache.iotdb.tsfile.file.header.PageHeader; +import org.apache.iotdb.tsfile.file.metadata.IChunkMetadata; +import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType; +import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding; +import org.apache.iotdb.tsfile.file.metadata.statistics.Statistics; +import org.apache.iotdb.tsfile.read.TsFileSequenceReader; +import org.apache.iotdb.tsfile.read.common.BatchData; +import org.apache.iotdb.tsfile.read.common.Chunk; +import org.apache.iotdb.tsfile.read.reader.page.PageReader; +import org.apache.iotdb.tsfile.utils.Binary; +import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils; +import org.apache.iotdb.tsfile.write.chunk.ChunkWriterImpl; +import org.apache.iotdb.tsfile.write.schema.MeasurementSchema; +import org.apache.iotdb.tsfile.write.writer.TsFileIOWriter; + +import java.io.DataOutputStream; +import java.io.File; +import java.io.IOException; +import java.io.InputStream; +import java.io.Serializable; +import java.nio.ByteBuffer; + +public class NonAlignedChunkData implements ChunkData { + private long offset; + private long dataSize; + private boolean isHeadPageNeedDecode; + private boolean isTailPageNeedDecode; + + private TTimePartitionSlot timePartitionSlot; + private String device; + private ChunkHeader chunkHeader; + private IChunkMetadata chunkMetadata; + + private ChunkWriterImpl chunkWriter; + private Chunk chunk; + + public NonAlignedChunkData(long offset, String device, ChunkHeader chunkHeader) { + this.offset = offset; + this.dataSize = 0; + this.isHeadPageNeedDecode = false; + this.isTailPageNeedDecode = false; + this.device = device; + this.chunkHeader = chunkHeader; + } + + @Override + public String getDevice() { + return device; + } + + @Override + public TTimePartitionSlot getTimePartitionSlot() { + return timePartitionSlot; + } + + @Override + public long getDataSize() { + return dataSize; + } + + @Override + public void addDataSize(long pageSize) { + dataSize += pageSize; + } + + @Override + public void setNotDecode(IChunkMetadata chunkMetadata) { + this.chunkMetadata = chunkMetadata; + } + + @Override + public boolean needDecodeChunk() { + return chunkMetadata == null; + } + + @Override + public void setHeadPageNeedDecode(boolean headPageNeedDecode) { + isHeadPageNeedDecode = headPageNeedDecode; + } + + @Override + public void setTailPageNeedDecode(boolean tailPageNeedDecode) { + isTailPageNeedDecode = tailPageNeedDecode; + } + + @Override + public void setTimePartitionSlot(TTimePartitionSlot timePartitionSlot) { + this.timePartitionSlot = timePartitionSlot; + } + + @Override + public boolean isAligned() { + return false; + } + + @Override + public void writeToFileWriter(TsFileIOWriter writer) throws IOException { + if (chunk != null) { + writer.writeChunk(chunk); + } else { + chunkWriter.writeToFileWriter(writer); + } + } + + @Override + public void serialize(DataOutputStream stream, File tsFile) throws IOException { + ReadWriteIOUtils.write(isAligned(), stream); + serializeAttr(stream); + if (needDecodeChunk()) { + ReadWriteIOUtils.write(true, stream); + serializeDecodeChunk(stream, tsFile); + } else { + ReadWriteIOUtils.write(false, stream); + serializeEntireChunk(stream, tsFile); + } + } + + private void serializeAttr(DataOutputStream stream) throws IOException { + ReadWriteIOUtils.write(timePartitionSlot.getStartTime(), stream); + ReadWriteIOUtils.write(device, stream); + chunkHeader.serializeTo(stream); // chunk header already serialize chunk type + } + + private void serializeEntireChunk(DataOutputStream stream, File tsFile) throws IOException { + try (TsFileSequenceReader reader = new TsFileSequenceReader(tsFile.getAbsolutePath())) { + ByteBuffer chunkData = + reader.readChunk( + chunkMetadata.getOffsetOfChunkHeader() + chunkHeader.getSerializedSize(), + chunkHeader.getDataSize()); + ReadWriteIOUtils.write(chunkData, stream); + chunkMetadata.getStatistics().serialize(stream); + } + } + + private void serializeDecodeChunk(DataOutputStream stream, File tsFile) throws IOException { + try (TsFileSequenceReader reader = new TsFileSequenceReader(tsFile.getAbsolutePath())) { + Decoder defaultTimeDecoder = + Decoder.getDecoderByType( + TSEncoding.valueOf(TSFileDescriptor.getInstance().getConfig().getTimeEncoder()), + TSDataType.INT64); + Decoder valueDecoder = + Decoder.getDecoderByType(chunkHeader.getEncodingType(), chunkHeader.getDataType()); + + reader.position(offset); + long dataSize = this.dataSize; + while (dataSize > 0) { + boolean hasStatistic = (chunkHeader.getChunkType() & 0x3F) == MetaMarker.CHUNK_HEADER; + PageHeader pageHeader = reader.readPageHeader(chunkHeader.getDataType(), hasStatistic); + long pageDataSize = pageHeader.getSerializedPageSize(); + if ((dataSize == this.dataSize && isHeadPageNeedDecode) // decode head page + || (dataSize == pageDataSize && isTailPageNeedDecode)) { // decode tail page + ReadWriteIOUtils.write(true, stream); // decode + decodePage(reader, pageHeader, defaultTimeDecoder, valueDecoder, stream); + } else { // entire page + ReadWriteIOUtils.write(false, stream); // don't decode + pageHeader.serializeTo(stream); + ByteBuffer pageData = reader.readCompressedPage(pageHeader); + ReadWriteIOUtils.write(pageData, stream); + } + dataSize -= pageDataSize; + } + } + + ReadWriteIOUtils.write(true, stream); // means ending + ReadWriteIOUtils.write(-1, stream); + } + + private void decodePage( + TsFileSequenceReader reader, + PageHeader pageHeader, + Decoder timeDecoder, + Decoder valueDecoder, + DataOutputStream stream) + throws IOException { + valueDecoder.reset(); + ByteBuffer pageData = reader.readPage(pageHeader, chunkHeader.getCompressionType()); + PageReader pageReader = + new PageReader(pageData, chunkHeader.getDataType(), valueDecoder, timeDecoder, null); + BatchData batchData = pageReader.getAllSatisfiedPageData(); + + int length = 0; + while (batchData.hasCurrent()) { + long time = batchData.currentTime(); + if (time < timePartitionSlot.getStartTime()) { + batchData.next(); + continue; + } else if (!timePartitionSlot.equals(StorageEngineV2.getTimePartitionSlot(time))) { + break; + } + length += 1; + batchData.next(); + } + + ReadWriteIOUtils.write(length, stream); + batchData.resetBatchData(); + while (batchData.hasCurrent()) { + long time = batchData.currentTime(); + if (time < timePartitionSlot.getStartTime()) { + batchData.next(); + continue; + } else if (!timePartitionSlot.equals(StorageEngineV2.getTimePartitionSlot(time))) { + break; + } + + ReadWriteIOUtils.write(time, stream); + Object value = batchData.currentValue(); + switch (chunkHeader.getDataType()) { + case INT32: + ReadWriteIOUtils.write((int) value, stream); + break; + case INT64: + ReadWriteIOUtils.write((long) value, stream); + break; + case FLOAT: + ReadWriteIOUtils.write((float) value, stream); + break; + case DOUBLE: + ReadWriteIOUtils.write((double) value, stream); + break; + case BOOLEAN: + ReadWriteIOUtils.write((boolean) value, stream); + break; + case TEXT: + ReadWriteIOUtils.write((Binary) value, stream); + break; + default: + throw new UnSupportedDataTypeException( + String.format("Data type %s is not supported.", chunkHeader.getDataType())); + } + batchData.next(); + } + } + + private void deserializeTsFileData(InputStream stream) throws IOException, PageException { + boolean needDecodeChunk = ReadWriteIOUtils.readBool(stream); + if (needDecodeChunk) { + buildChunkWriter(stream); + } else { + deserializeEntireChunk(stream); + } + } + + private void buildChunkWriter(InputStream stream) throws IOException, PageException { + chunkWriter = + new ChunkWriterImpl( + new MeasurementSchema( + chunkHeader.getMeasurementID(), + chunkHeader.getDataType(), + chunkHeader.getEncodingType(), + chunkHeader.getCompressionType())); + boolean needDecode; + while (true) { + needDecode = ReadWriteIOUtils.readBool(stream); + if (needDecode) { + int length = ReadWriteIOUtils.readInt(stream); + if (length == -1) { + break; + } + + for (int i = 0; i < length; i++) { + long time = ReadWriteIOUtils.readLong(stream); + switch (chunkHeader.getDataType()) { + case INT32: + chunkWriter.write(time, ReadWriteIOUtils.readInt(stream)); + break; + case INT64: + chunkWriter.write(time, ReadWriteIOUtils.readLong(stream)); + break; + case FLOAT: + chunkWriter.write(time, ReadWriteIOUtils.readFloat(stream)); + break; + case DOUBLE: + chunkWriter.write(time, ReadWriteIOUtils.readDouble(stream)); + break; + case BOOLEAN: + chunkWriter.write(time, ReadWriteIOUtils.readBool(stream)); + break; + case TEXT: + chunkWriter.write(time, ReadWriteIOUtils.readBinary(stream)); + break; + default: + throw new UnSupportedDataTypeException( + String.format("Data type %s is not supported.", chunkHeader.getDataType())); + } + } + + chunkWriter.sealCurrentPage(); + } else { + PageHeader pageHeader = PageHeader.deserializeFrom(stream, chunkHeader.getDataType(), true); + chunkWriter.writePageHeaderAndDataIntoBuff( + ByteBuffer.wrap(ReadWriteIOUtils.readBytesWithSelfDescriptionLength(stream)), + pageHeader); + } + } + } + + private void deserializeEntireChunk(InputStream stream) throws IOException { + ByteBuffer chunkData = + ByteBuffer.wrap(ReadWriteIOUtils.readBytesWithSelfDescriptionLength(stream)); + Statistics statistics = + Statistics.deserialize(stream, chunkHeader.getDataType()); + chunk = new Chunk(chunkHeader, chunkData, null, statistics); + } + + public static NonAlignedChunkData deserialize(InputStream stream) + throws IOException, PageException { + long timePartition = ReadWriteIOUtils.readLong(stream); + String device = ReadWriteIOUtils.readString(stream); + byte chunkType = ReadWriteIOUtils.readByte(stream); + ChunkHeader chunkHeader = ChunkHeader.deserializeFrom(stream, chunkType); + NonAlignedChunkData chunkData = new NonAlignedChunkData(-1, device, chunkHeader); + chunkData.setTimePartitionSlot(StorageEngineV2.getTimePartitionSlot(timePartition)); + chunkData.deserializeTsFileData(stream); + return chunkData; + } + + @Override + public String toString() { + return "NonAlignedChunkData{" + + "offset=" + + offset + + ", dataSize=" + + dataSize + + ", isHeadPageNeedDecode=" + + isHeadPageNeedDecode + + ", isTailPageNeedDecode=" + + isTailPageNeedDecode + + ", timePartitionSlot=" + + timePartitionSlot + + ", device='" + + device + + '\'' + + ", chunkHeader=" + + chunkHeader + + '}'; + } +} diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/DataRegion.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/DataRegion.java index 1780dcb55fc4..bf8286c9a9fa 100755 --- a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/DataRegion.java +++ b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/DataRegion.java @@ -2659,11 +2659,11 @@ private void tryToDeleteLastCacheByDevice(PartialPath deviceId) { if (!IoTDBDescriptor.getInstance().getConfig().isLastCacheEnabled()) { return; } - try { - IoTDB.schemaProcessor.deleteLastCacheByDevice(deviceId); - } catch (MetadataException e) { - // the path doesn't cache in cluster mode now, ignore - } + // try { TODO: support last cache + // IoTDB.schemaProcessor.deleteLastCacheByDevice(deviceId); + // } catch (MetadataException e) { + // // the path doesn't cache in cluster mode now, ignore + // } } /** diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/AnalyzeVisitor.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/AnalyzeVisitor.java index 47ab3a348d15..b39ebbc08762 100644 --- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/AnalyzeVisitor.java +++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/AnalyzeVisitor.java @@ -18,6 +18,7 @@ */ package org.apache.iotdb.db.mpp.plan.analyze; +import org.apache.iotdb.common.rpc.thrift.TTimePartitionSlot; import org.apache.iotdb.commons.conf.IoTDBConstant; import org.apache.iotdb.commons.exception.MetadataException; import org.apache.iotdb.commons.partition.DataPartition; @@ -25,6 +26,8 @@ import org.apache.iotdb.commons.partition.SchemaNodeManagementPartition; import org.apache.iotdb.commons.partition.SchemaPartition; import org.apache.iotdb.commons.path.PartialPath; +import org.apache.iotdb.db.engine.StorageEngineV2; +import org.apache.iotdb.db.engine.storagegroup.TsFileResource; import org.apache.iotdb.db.exception.metadata.template.TemplateImcompatibeException; import org.apache.iotdb.db.exception.sql.MeasurementNotExistException; import org.apache.iotdb.db.exception.sql.SemanticException; @@ -63,6 +66,7 @@ import org.apache.iotdb.db.mpp.plan.statement.crud.InsertRowsStatement; import org.apache.iotdb.db.mpp.plan.statement.crud.InsertStatement; import org.apache.iotdb.db.mpp.plan.statement.crud.InsertTabletStatement; +import org.apache.iotdb.db.mpp.plan.statement.crud.LoadTsFileStatement; import org.apache.iotdb.db.mpp.plan.statement.crud.QueryStatement; import org.apache.iotdb.db.mpp.plan.statement.internal.InternalCreateTimeSeriesStatement; import org.apache.iotdb.db.mpp.plan.statement.internal.SchemaFetchStatement; @@ -92,6 +96,7 @@ import org.apache.iotdb.db.mpp.plan.statement.sys.ExplainStatement; import org.apache.iotdb.db.mpp.plan.statement.sys.ShowVersionStatement; import org.apache.iotdb.db.mpp.plan.statement.sys.sync.ShowPipeSinkTypeStatement; +import org.apache.iotdb.db.utils.FileLoaderUtils; import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType; import org.apache.iotdb.tsfile.read.filter.GroupByFilter; import org.apache.iotdb.tsfile.read.filter.GroupByMonthFilter; @@ -102,6 +107,8 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.File; +import java.io.IOException; import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; @@ -1333,6 +1340,61 @@ public Analysis visitInsertRowsOfOneDevice( return analysis; } + @Override + public Analysis visitLoadFile(LoadTsFileStatement loadTsFileStatement, MPPQueryContext context) { + context.setQueryType(QueryType.WRITE); + + Map device2MinTime = new HashMap<>(); + Map device2MaxTime = new HashMap<>(); + for (File tsFile : loadTsFileStatement.getTsFiles()) { + try { + TsFileResource resource = new TsFileResource(tsFile); + FileLoaderUtils.loadOrGenerateResource(resource); + for (String device : resource.getDevices()) { + device2MinTime.put( + device, + Math.min( + device2MinTime.getOrDefault(device, Long.MAX_VALUE), + resource.getStartTime(device))); + device2MaxTime.put( + device, + Math.max( + device2MaxTime.getOrDefault(device, Long.MIN_VALUE), + resource.getEndTime(device))); + } + } catch (IOException e) { + logger.error(String.format("Parse file %s to resource error.", tsFile.getPath()), e); + throw new SemanticException( + String.format("Parse file %s to resource error", tsFile.getPath())); + } + } + + List params = new ArrayList<>(); + for (Map.Entry entry : device2MinTime.entrySet()) { + List timePartitionSlots = new ArrayList<>(); + String device = entry.getKey(); + long endTime = device2MaxTime.get(device); + long interval = StorageEngineV2.getTimePartitionInterval(); + long time = (entry.getValue() / interval) * interval; + for (; time <= endTime; time += interval) { + timePartitionSlots.add(StorageEngineV2.getTimePartitionSlot(time)); + } + + DataPartitionQueryParam dataPartitionQueryParam = new DataPartitionQueryParam(); + dataPartitionQueryParam.setDevicePath(device); + dataPartitionQueryParam.setTimePartitionSlotList(timePartitionSlots); + params.add(dataPartitionQueryParam); + } + + DataPartition dataPartition = partitionFetcher.getOrCreateDataPartition(params); + + Analysis analysis = new Analysis(); + analysis.setStatement(loadTsFileStatement); + analysis.setDataPartitionInfo(dataPartition); + + return analysis; + } + @Override public Analysis visitShowTimeSeries( ShowTimeSeriesStatement showTimeSeriesStatement, MPPQueryContext context) { diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/SchemaValidator.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/SchemaValidator.java index e14fbf156360..bb34ce306f38 100644 --- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/SchemaValidator.java +++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/SchemaValidator.java @@ -19,11 +19,15 @@ package org.apache.iotdb.db.mpp.plan.analyze; +import org.apache.iotdb.commons.path.PartialPath; import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.exception.sql.SemanticException; import org.apache.iotdb.db.mpp.common.schematree.ISchemaTree; import org.apache.iotdb.db.mpp.plan.planner.plan.node.write.BatchInsertNode; import org.apache.iotdb.db.mpp.plan.planner.plan.node.write.InsertNode; +import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType; + +import java.util.List; public class SchemaValidator { @@ -58,4 +62,13 @@ public static ISchemaTree validate(InsertNode insertNode) { return schemaTree; } + + public static ISchemaTree validate( + List devicePaths, + List measurements, + List dataTypes, + List isAlignedList) { + return SCHEMA_FETCHER.fetchSchemaListWithAutoCreate( + devicePaths, measurements, dataTypes, isAlignedList); + } } diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/QueryExecution.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/QueryExecution.java index 48607b996f08..a471be878034 100644 --- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/QueryExecution.java +++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/QueryExecution.java @@ -52,10 +52,12 @@ import org.apache.iotdb.db.mpp.plan.scheduler.ClusterScheduler; import org.apache.iotdb.db.mpp.plan.scheduler.IScheduler; import org.apache.iotdb.db.mpp.plan.scheduler.StandaloneScheduler; +import org.apache.iotdb.db.mpp.plan.scheduler.load.LoadTsFileScheduler; import org.apache.iotdb.db.mpp.plan.statement.Statement; import org.apache.iotdb.db.mpp.plan.statement.crud.InsertBaseStatement; import org.apache.iotdb.db.mpp.plan.statement.crud.InsertMultiTabletsStatement; import org.apache.iotdb.db.mpp.plan.statement.crud.InsertRowsStatement; +import org.apache.iotdb.db.mpp.plan.statement.crud.LoadTsFileStatement; import org.apache.iotdb.db.utils.SetThreadName; import org.apache.iotdb.rpc.RpcUtils; import org.apache.iotdb.rpc.TSStatusCode; @@ -233,6 +235,13 @@ private Analysis analyze( } private void schedule() { + if (rawStatement instanceof LoadTsFileStatement) { + this.scheduler = + new LoadTsFileScheduler( + distributedPlan, context, stateMachine, internalServiceClientManager); + this.scheduler.start(); + return; + } // TODO: (xingtanzjr) initialize the query scheduler according to configuration this.scheduler = config.isClusterMode() diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/parser/ASTVisitor.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/parser/ASTVisitor.java index bb2c2d8aa196..8e37b8d30d90 100644 --- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/parser/ASTVisitor.java +++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/parser/ASTVisitor.java @@ -79,6 +79,7 @@ import org.apache.iotdb.db.mpp.plan.statement.component.WhereCondition; import org.apache.iotdb.db.mpp.plan.statement.crud.DeleteDataStatement; import org.apache.iotdb.db.mpp.plan.statement.crud.InsertStatement; +import org.apache.iotdb.db.mpp.plan.statement.crud.LoadTsFileStatement; import org.apache.iotdb.db.mpp.plan.statement.crud.QueryStatement; import org.apache.iotdb.db.mpp.plan.statement.literal.BooleanLiteral; import org.apache.iotdb.db.mpp.plan.statement.literal.DoubleLiteral; @@ -1434,6 +1435,45 @@ private void parseInsertValuesSpec( insertStatement.setValuesList(valuesList); } + // Load File + + @Override + public Statement visitLoadFile(IoTDBSqlParser.LoadFileContext ctx) { + LoadTsFileStatement loadTsFileStatement = + new LoadTsFileStatement(parseStringLiteral(ctx.fileName.getText())); + if (ctx.loadFilesClause() != null) { + parseLoadFiles(loadTsFileStatement, ctx.loadFilesClause()); + } + return loadTsFileStatement; + } + + /** + * used for parsing load tsfile, context will be one of "SCHEMA, LEVEL, METADATA", and maybe + * followed by a recursion property statement + * + * @param loadTsFileStatement the result statement, setting by clause context + * @param ctx context of property statement + */ + private void parseLoadFiles( + LoadTsFileStatement loadTsFileStatement, IoTDBSqlParser.LoadFilesClauseContext ctx) { + if (ctx.AUTOREGISTER() != null) { + loadTsFileStatement.setAutoCreateSchema( + Boolean.parseBoolean(ctx.BOOLEAN_LITERAL().getText())); + } else if (ctx.SGLEVEL() != null) { + loadTsFileStatement.setSgLevel(Integer.parseInt(ctx.INTEGER_LITERAL().getText())); + } else if (ctx.VERIFY() != null) { + loadTsFileStatement.setVerifySchema(Boolean.parseBoolean(ctx.BOOLEAN_LITERAL().getText())); + } else { + throw new SQLParserException( + String.format( + "load tsfile format %s error, please input AUTOREGISTER | SGLEVEL | VERIFY.", + ctx.getText())); + } + if (ctx.loadFilesClause() != null) { + parseLoadFiles(loadTsFileStatement, ctx.loadFilesClause()); + } + } + /** Common Parsers */ // IoTDB Objects ======================================================================== diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/LogicalPlanVisitor.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/LogicalPlanVisitor.java index b59e490ebe4f..be44c14eee6f 100644 --- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/LogicalPlanVisitor.java +++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/LogicalPlanVisitor.java @@ -23,6 +23,7 @@ import org.apache.iotdb.db.mpp.plan.analyze.ExpressionAnalyzer; import org.apache.iotdb.db.mpp.plan.expression.Expression; import org.apache.iotdb.db.mpp.plan.planner.plan.node.PlanNode; +import org.apache.iotdb.db.mpp.plan.planner.plan.node.load.LoadTsFileNode; import org.apache.iotdb.db.mpp.plan.planner.plan.node.metedata.write.ActivateTemplateNode; import org.apache.iotdb.db.mpp.plan.planner.plan.node.metedata.write.AlterTimeSeriesNode; import org.apache.iotdb.db.mpp.plan.planner.plan.node.metedata.write.CreateAlignedTimeSeriesNode; @@ -46,6 +47,7 @@ import org.apache.iotdb.db.mpp.plan.statement.crud.InsertRowsOfOneDeviceStatement; import org.apache.iotdb.db.mpp.plan.statement.crud.InsertRowsStatement; import org.apache.iotdb.db.mpp.plan.statement.crud.InsertTabletStatement; +import org.apache.iotdb.db.mpp.plan.statement.crud.LoadTsFileStatement; import org.apache.iotdb.db.mpp.plan.statement.crud.QueryStatement; import org.apache.iotdb.db.mpp.plan.statement.internal.InternalCreateTimeSeriesStatement; import org.apache.iotdb.db.mpp.plan.statement.internal.SchemaFetchStatement; @@ -476,6 +478,12 @@ public PlanNode visitInsertRow(InsertRowStatement insertRowStatement, MPPQueryCo insertRowStatement.isNeedInferType()); } + @Override + public PlanNode visitLoadFile(LoadTsFileStatement loadTsFileStatement, MPPQueryContext context) { + return new LoadTsFileNode( + context.getQueryId().genPlanNodeId(), loadTsFileStatement.getTsFiles()); + } + @Override public PlanNode visitShowTimeSeries( ShowTimeSeriesStatement showTimeSeriesStatement, MPPQueryContext context) { diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/FragmentInstance.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/FragmentInstance.java index 3f43e8245cd2..d6598ae29719 100644 --- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/FragmentInstance.java +++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/FragmentInstance.java @@ -98,6 +98,9 @@ public TRegionReplicaSet getDataRegionId() { } public void setDataRegionAndHost(TRegionReplicaSet regionReplicaSet) { + if (regionReplicaSet == null) { + return; + } this.regionReplicaSet = regionReplicaSet; // TODO: (xingtanzjr) We select the first Endpoint as the default target host for current // instance diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/PlanNodeId.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/PlanNodeId.java index 554a9159455c..fd511b2ba170 100644 --- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/PlanNodeId.java +++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/PlanNodeId.java @@ -22,6 +22,7 @@ import java.io.DataOutputStream; import java.io.IOException; +import java.io.InputStream; import java.nio.ByteBuffer; public class PlanNodeId { @@ -57,6 +58,10 @@ public static PlanNodeId deserialize(ByteBuffer byteBuffer) { return new PlanNodeId(ReadWriteIOUtils.readString(byteBuffer)); } + public static PlanNodeId deserialize(InputStream stream) throws IOException { + return new PlanNodeId(ReadWriteIOUtils.readString(stream)); + } + public void serialize(ByteBuffer byteBuffer) { ReadWriteIOUtils.write(id, byteBuffer); } diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/PlanNodeType.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/PlanNodeType.java index caa84ee5dada..054d4b21deca 100644 --- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/PlanNodeType.java +++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/PlanNodeType.java @@ -18,6 +18,7 @@ */ package org.apache.iotdb.db.mpp.plan.planner.plan.node; +import org.apache.iotdb.db.mpp.plan.planner.plan.node.load.LoadTsFilePieceNode; import org.apache.iotdb.db.mpp.plan.planner.plan.node.metedata.read.CountSchemaMergeNode; import org.apache.iotdb.db.mpp.plan.planner.plan.node.metedata.read.DevicesCountNode; import org.apache.iotdb.db.mpp.plan.planner.plan.node.metedata.read.DevicesSchemaScanNode; @@ -135,8 +136,9 @@ public enum PlanNodeType { INTERNAL_CREATE_TIMESERIES((short) 52), ACTIVATE_TEMPLATE((short) 53), PATHS_USING_TEMPLATE_SCAN((short) 54), - CONSTRUCT_SCHEMA_BLACK_LIST_NODE((short) 55), - ROLLBACK_SCHEMA_BLACK_LIST_NODE((short) 56); + LOAD_TSFILE((short) 55), + CONSTRUCT_SCHEMA_BLACK_LIST_NODE((short) 56), + ROLLBACK_SCHEMA_BLACK_LIST_NODE((short) 57); public static final int BYTES = Short.BYTES; @@ -296,8 +298,10 @@ public static PlanNode deserialize(ByteBuffer buffer) { case 54: return PathsUsingTemplateScanNode.deserialize(buffer); case 55: - return ConstructSchemaBlackListNode.deserialize(buffer); + return LoadTsFilePieceNode.deserialize(buffer); case 56: + return ConstructSchemaBlackListNode.deserialize(buffer); + case 57: return RollbackSchemaBlackListNode.deserialize(buffer); default: throw new IllegalArgumentException("Invalid node type: " + nodeType); diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/load/LoadSingleTsFileNode.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/load/LoadSingleTsFileNode.java new file mode 100644 index 000000000000..91519fe64976 --- /dev/null +++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/load/LoadSingleTsFileNode.java @@ -0,0 +1,534 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.mpp.plan.planner.plan.node.load; + +import org.apache.iotdb.common.rpc.thrift.TDataNodeLocation; +import org.apache.iotdb.common.rpc.thrift.TEndPoint; +import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet; +import org.apache.iotdb.common.rpc.thrift.TTimePartitionSlot; +import org.apache.iotdb.commons.exception.IllegalPathException; +import org.apache.iotdb.commons.partition.DataPartition; +import org.apache.iotdb.commons.path.PartialPath; +import org.apache.iotdb.db.conf.IoTDBDescriptor; +import org.apache.iotdb.db.engine.StorageEngineV2; +import org.apache.iotdb.db.engine.load.AlignedChunkData; +import org.apache.iotdb.db.engine.load.ChunkData; +import org.apache.iotdb.db.engine.storagegroup.TsFileResource; +import org.apache.iotdb.db.mpp.plan.analyze.Analysis; +import org.apache.iotdb.db.mpp.plan.analyze.SchemaValidator; +import org.apache.iotdb.db.mpp.plan.planner.plan.node.PlanNode; +import org.apache.iotdb.db.mpp.plan.planner.plan.node.PlanNodeId; +import org.apache.iotdb.db.mpp.plan.planner.plan.node.WritePlanNode; +import org.apache.iotdb.db.utils.FileLoaderUtils; +import org.apache.iotdb.tsfile.common.conf.TSFileConfig; +import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor; +import org.apache.iotdb.tsfile.common.constant.TsFileConstant; +import org.apache.iotdb.tsfile.encoding.decoder.Decoder; +import org.apache.iotdb.tsfile.exception.NotImplementedException; +import org.apache.iotdb.tsfile.exception.TsFileRuntimeException; +import org.apache.iotdb.tsfile.file.MetaMarker; +import org.apache.iotdb.tsfile.file.header.ChunkGroupHeader; +import org.apache.iotdb.tsfile.file.header.ChunkHeader; +import org.apache.iotdb.tsfile.file.header.PageHeader; +import org.apache.iotdb.tsfile.file.metadata.IChunkMetadata; +import org.apache.iotdb.tsfile.file.metadata.TimeseriesMetadata; +import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType; +import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding; +import org.apache.iotdb.tsfile.read.TsFileSequenceReader; +import org.apache.iotdb.tsfile.read.common.BatchData; +import org.apache.iotdb.tsfile.read.reader.page.PageReader; +import org.apache.iotdb.tsfile.read.reader.page.TimePageReader; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.DataOutputStream; +import java.io.File; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +public class LoadSingleTsFileNode extends WritePlanNode { + private static final Logger logger = LoggerFactory.getLogger(LoadSingleTsFileNode.class); + + private File tsFile; + private boolean needDecodeTsFile; + + private Map> replicaSet2Pieces; + + private TsFileResource resource; + private TRegionReplicaSet localRegionReplicaSet; + + public LoadSingleTsFileNode(PlanNodeId id) { + super(id); + } + + public LoadSingleTsFileNode(PlanNodeId id, File tsFile) throws IOException { + super(id); + this.tsFile = tsFile; + this.resource = new TsFileResource(tsFile); + + FileLoaderUtils.loadOrGenerateResource(resource); + } + + public void checkIfNeedDecodeTsFile(DataPartition dataPartition) { + Set allRegionReplicaSet = new HashSet<>(); + needDecodeTsFile = false; + for (String device : resource.getDevices()) { + if (!StorageEngineV2.getTimePartitionSlot(resource.getStartTime(device)) + .equals(StorageEngineV2.getTimePartitionSlot(resource.getEndTime(device)))) { + needDecodeTsFile = true; + return; + } + allRegionReplicaSet.addAll(dataPartition.getAllDataRegionReplicaSetForOneDevice(device)); + } + needDecodeTsFile = !isDispatchedToLocal(allRegionReplicaSet); + } + + private boolean isDispatchedToLocal(Set replicaSets) { + if (replicaSets.size() > 1) { + return false; + } + for (TRegionReplicaSet replicaSet : replicaSets) { + List dataNodeLocationList = replicaSet.getDataNodeLocations(); + if (dataNodeLocationList.size() > 1) { + return false; + } + localRegionReplicaSet = replicaSet; + return isDispatchedToLocal(dataNodeLocationList.get(0).getInternalEndPoint()); + } + return true; + } + + private boolean isDispatchedToLocal(TEndPoint endPoint) { + return IoTDBDescriptor.getInstance().getConfig().getInternalAddress().equals(endPoint.getIp()) + && IoTDBDescriptor.getInstance().getConfig().getInternalPort() == endPoint.port; + } + + public void autoRegisterSchema() + throws IOException, IllegalPathException { // TODO: only support sg level=1 + try (TsFileSequenceReader reader = new TsFileSequenceReader(tsFile.getAbsolutePath())) { + List deviceList = new ArrayList<>(); + List measurementList = new ArrayList<>(); + List dataTypeList = new ArrayList<>(); + List isAlignedList = new ArrayList<>(); + + Map> device2Metadata = reader.getAllTimeseriesMetadata(true); + for (Map.Entry> entry : device2Metadata.entrySet()) { + deviceList.add(new PartialPath(entry.getKey())); + + List timeseriesMetadataList = entry.getValue(); + boolean isAligned = + timeseriesMetadataList.stream() + .mapToInt(o -> o.getTSDataType().equals(TSDataType.VECTOR) ? 1 : 0) + .sum() + != 0; + int measurementSize = timeseriesMetadataList.size() - (isAligned ? 1 : 0); + String[] measurements = new String[measurementSize]; + TSDataType[] tsDataTypes = new TSDataType[measurementSize]; + + int index = 0; + for (TimeseriesMetadata timeseriesMetadata : timeseriesMetadataList) { + TSDataType dataType = timeseriesMetadata.getTSDataType(); + if (!dataType.equals(TSDataType.VECTOR)) { + measurements[index] = timeseriesMetadata.getMeasurementId(); + tsDataTypes[index++] = dataType; + } + } + measurementList.add(measurements); + dataTypeList.add(tsDataTypes); + isAlignedList.add(isAligned); + } + + SchemaValidator.validate(deviceList, measurementList, dataTypeList, isAlignedList); + } + } + + public boolean needDecodeTsFile() { + return needDecodeTsFile; + } + + /** + * only used for load locally. + * + * @return local TRegionReplicaSet + */ + public TRegionReplicaSet getLocalRegionReplicaSet() { + return localRegionReplicaSet; + } + + public TsFileResource getTsFileResource() { + return resource; + } + + public Map> getReplicaSet2Pieces() { + return replicaSet2Pieces; + } + + @Override + public TRegionReplicaSet getRegionReplicaSet() { + return null; + } + + @Override + public List getChildren() { + return null; + } + + @Override + public void addChild(PlanNode child) {} + + @Override + public PlanNode clone() { + throw new NotImplementedException("clone of load single TsFile is not implemented"); + } + + @Override + public int allowedChildCount() { + return NO_CHILD_ALLOWED; + } + + @Override + public List getOutputColumnNames() { + return null; + } + + @Override + protected void serializeAttributes(ByteBuffer byteBuffer) {} + + @Override + protected void serializeAttributes(DataOutputStream stream) throws IOException {} + + @Override + public List splitByPartition(Analysis analysis) { + throw new NotImplementedException("split load single TsFile is not implemented"); + } + + @Override + public String toString() { + return "LoadSingleTsFileNode{" + + "tsFile=" + + tsFile + + ", needDecodeTsFile=" + + needDecodeTsFile + + '}'; + } + + public void splitTsFileByDataPartition(DataPartition dataPartition) throws IOException { + replicaSet2Pieces = new HashMap<>(); + List chunkDataList = new ArrayList<>(); + + try (TsFileSequenceReader reader = new TsFileSequenceReader(tsFile.getAbsolutePath())) { + if (!checkMagic(reader)) { + throw new TsFileRuntimeException( + String.format("Magic String check error when parsing TsFile %s.", tsFile.getPath())); + } + + reader.position((long) TSFileConfig.MAGIC_STRING.getBytes().length + 1); + String curDevice = null; + boolean isTimeChunkNeedDecode = true; + Map> pageIndex2ChunkData = null; + Map offset2ChunkMetadata = new HashMap<>(); + getChunkMetadata(reader, offset2ChunkMetadata); + byte marker; + while ((marker = reader.readMarker()) != MetaMarker.SEPARATOR) { + switch (marker) { + case MetaMarker.CHUNK_HEADER: + case MetaMarker.TIME_CHUNK_HEADER: + case MetaMarker.ONLY_ONE_PAGE_CHUNK_HEADER: + case MetaMarker.ONLY_ONE_PAGE_TIME_CHUNK_HEADER: + long chunkOffset = reader.position(); + ChunkHeader header = reader.readChunkHeader(marker); + if (header.getDataSize() == 0) { + throw new TsFileRuntimeException( + String.format("Chunk data error when parsing TsFile %s.", tsFile.getPath())); + } + + boolean isAligned = + ((header.getChunkType() & TsFileConstant.TIME_COLUMN_MASK) + == TsFileConstant.TIME_COLUMN_MASK); + IChunkMetadata chunkMetadata = offset2ChunkMetadata.get(chunkOffset - Byte.BYTES); + TTimePartitionSlot timePartitionSlot = + StorageEngineV2.getTimePartitionSlot(chunkMetadata.getStartTime()); + ChunkData chunkData = + ChunkData.createChunkData(isAligned, reader.position(), curDevice, header); + chunkData.setTimePartitionSlot(timePartitionSlot); + if (!needDecodeChunk(chunkMetadata)) { + if (isAligned) { + isTimeChunkNeedDecode = false; + pageIndex2ChunkData = new HashMap<>(); + pageIndex2ChunkData + .computeIfAbsent(1, o -> new ArrayList<>()) + .add((AlignedChunkData) chunkData); + } + chunkData.setNotDecode(chunkMetadata); + chunkDataList.add(chunkData); + reader.position(reader.position() + header.getDataSize()); + break; + } + if (isAligned) { + isTimeChunkNeedDecode = true; + pageIndex2ChunkData = new HashMap<>(); + } + + Decoder defaultTimeDecoder = + Decoder.getDecoderByType( + TSEncoding.valueOf(TSFileDescriptor.getInstance().getConfig().getTimeEncoder()), + TSDataType.INT64); + Decoder valueDecoder = + Decoder.getDecoderByType(header.getEncodingType(), header.getDataType()); + int dataSize = header.getDataSize(); + int pageIndex = 0; + while (dataSize > 0) { + long pageOffset = reader.position(); + PageHeader pageHeader = + reader.readPageHeader( + header.getDataType(), + (header.getChunkType() & 0x3F) == MetaMarker.CHUNK_HEADER); + long pageDataSize = pageHeader.getSerializedPageSize(); + if (!needDecodePage(pageHeader, chunkMetadata)) { // an entire page + long startTime = + pageHeader.getStatistics() == null + ? chunkMetadata.getStartTime() + : pageHeader.getStartTime(); + TTimePartitionSlot pageTimePartitionSlot = + StorageEngineV2.getTimePartitionSlot(startTime); + if (!timePartitionSlot.equals(pageTimePartitionSlot)) { + chunkDataList.add(chunkData); + timePartitionSlot = pageTimePartitionSlot; + chunkData = ChunkData.createChunkData(isAligned, pageOffset, curDevice, header); + chunkData.setTimePartitionSlot(timePartitionSlot); + } + if (isAligned) { + pageIndex2ChunkData + .computeIfAbsent(pageIndex, o -> new ArrayList<>()) + .add((AlignedChunkData) chunkData); + } + chunkData.addDataSize(pageDataSize); + reader.position(pageOffset + pageDataSize); + } else { // split page + ByteBuffer pageData = reader.readPage(pageHeader, header.getCompressionType()); + long[] timeBatch = + decodePage( + isAligned, pageData, pageHeader, defaultTimeDecoder, valueDecoder, header); + boolean isFirstData = true; + for (long currentTime : timeBatch) { + TTimePartitionSlot currentTimePartitionSlot = + StorageEngineV2.getTimePartitionSlot(currentTime); // TODO: can speed up + if (!timePartitionSlot.equals(currentTimePartitionSlot)) { + if (!isFirstData) { + chunkData.setTailPageNeedDecode(true); // close last chunk data + chunkData.addDataSize(pageDataSize); + if (isAligned) { + pageIndex2ChunkData + .computeIfAbsent(pageIndex, o -> new ArrayList<>()) + .add((AlignedChunkData) chunkData); + } + } + chunkDataList.add(chunkData); + + chunkData = + ChunkData.createChunkData( + isAligned, pageOffset, curDevice, header); // open a new chunk data + chunkData.setTimePartitionSlot(currentTimePartitionSlot); + chunkData.setHeadPageNeedDecode(true); + timePartitionSlot = currentTimePartitionSlot; + } + isFirstData = false; + } + chunkData.addDataSize(pageDataSize); + if (isAligned) { + pageIndex2ChunkData + .computeIfAbsent(pageIndex, o -> new ArrayList<>()) + .add((AlignedChunkData) chunkData); + } + } + + pageIndex += 1; + dataSize -= pageDataSize; + } + + chunkDataList.add(chunkData); + break; + case MetaMarker.VALUE_CHUNK_HEADER: + case MetaMarker.ONLY_ONE_PAGE_VALUE_CHUNK_HEADER: + chunkOffset = reader.position(); + chunkMetadata = offset2ChunkMetadata.get(chunkOffset - Byte.BYTES); + header = reader.readChunkHeader(marker); + if (header.getDataSize() == 0) { + handleEmptyValueChunk(chunkOffset, header, chunkMetadata, pageIndex2ChunkData); + break; + } + + Set allChunkData = new HashSet<>(); + if (!isTimeChunkNeedDecode) { + pageIndex2ChunkData.get(1).get(0).addValueChunk(chunkOffset, header, chunkMetadata); + reader.position(reader.position() + header.getDataSize()); + break; + } + + dataSize = header.getDataSize(); + pageIndex = 0; + + while (dataSize > 0) { + long pageOffset = reader.position(); + PageHeader pageHeader = + reader.readPageHeader( + header.getDataType(), + (header.getChunkType() & 0x3F) == MetaMarker.CHUNK_HEADER); + long pageDataSize = pageHeader.getSerializedPageSize(); + for (AlignedChunkData alignedChunkData : pageIndex2ChunkData.get(pageIndex)) { + if (!allChunkData.contains(alignedChunkData)) { + alignedChunkData.addValueChunk(pageOffset, header, chunkMetadata); + allChunkData.add(alignedChunkData); + } + alignedChunkData.addValueChunkDataSize(pageDataSize); + } + reader.position(pageOffset + pageDataSize); + + pageIndex += 1; + dataSize -= pageDataSize; + } + break; + case MetaMarker.CHUNK_GROUP_HEADER: + ChunkGroupHeader chunkGroupHeader = reader.readChunkGroupHeader(); + curDevice = chunkGroupHeader.getDeviceID(); + break; + case MetaMarker.OPERATION_INDEX_RANGE: + reader.readPlanIndex(); + break; + default: + MetaMarker.handleUnexpectedMarker(marker); + } + } + } + + for (ChunkData chunkData : chunkDataList) { + getPieceNode(chunkData.getDevice(), chunkData.getTimePartitionSlot(), dataPartition) + .addChunkData(chunkData); + } + } + + private boolean checkMagic(TsFileSequenceReader reader) throws IOException { + String magic = reader.readHeadMagic(); + if (!magic.equals(TSFileConfig.MAGIC_STRING)) { + logger.error("the file's MAGIC STRING is incorrect, file path: {}", reader.getFileName()); + return false; + } + + byte versionNumber = reader.readVersionNumber(); + if (versionNumber != TSFileConfig.VERSION_NUMBER) { + logger.error("the file's Version Number is incorrect, file path: {}", reader.getFileName()); + return false; + } + + if (!reader.readTailMagic().equals(TSFileConfig.MAGIC_STRING)) { + logger.error("the file is not closed correctly, file path: {}", reader.getFileName()); + return false; + } + return true; + } + + private void getChunkMetadata( + TsFileSequenceReader reader, Map offset2ChunkMetadata) + throws IOException { + Map> device2Metadata = reader.getAllTimeseriesMetadata(true); + for (Map.Entry> entry : device2Metadata.entrySet()) { + for (TimeseriesMetadata timeseriesMetadata : entry.getValue()) { + for (IChunkMetadata chunkMetadata : timeseriesMetadata.getChunkMetadataList()) { + offset2ChunkMetadata.put(chunkMetadata.getOffsetOfChunkHeader(), chunkMetadata); + } + } + } + } + + private boolean needDecodeChunk(IChunkMetadata chunkMetadata) { + return !StorageEngineV2.getTimePartitionSlot(chunkMetadata.getStartTime()) + .equals(StorageEngineV2.getTimePartitionSlot(chunkMetadata.getEndTime())); + } + + private boolean needDecodePage(PageHeader pageHeader, IChunkMetadata chunkMetadata) { + if (pageHeader.getStatistics() == null) { + return !StorageEngineV2.getTimePartitionSlot(chunkMetadata.getStartTime()) + .equals(StorageEngineV2.getTimePartitionSlot(chunkMetadata.getEndTime())); + } + return !StorageEngineV2.getTimePartitionSlot(pageHeader.getStartTime()) + .equals(StorageEngineV2.getTimePartitionSlot(pageHeader.getEndTime())); + } + + private long[] decodePage( + boolean isAligned, + ByteBuffer pageData, + PageHeader pageHeader, + Decoder timeDecoder, + Decoder valueDecoder, + ChunkHeader chunkHeader) + throws IOException { + if (isAligned) { + TimePageReader timePageReader = new TimePageReader(pageHeader, pageData, timeDecoder); + return timePageReader.getNextTimeBatch(); + } + + valueDecoder.reset(); + PageReader pageReader = + new PageReader(pageData, chunkHeader.getDataType(), valueDecoder, timeDecoder, null); + BatchData batchData = pageReader.getAllSatisfiedPageData(); + long[] timeBatch = new long[batchData.length()]; + int index = 0; + while (batchData.hasCurrent()) { + timeBatch[index++] = batchData.currentTime(); + batchData.next(); + } + return timeBatch; + } + + private void handleEmptyValueChunk( + long chunkOffset, + ChunkHeader header, + IChunkMetadata chunkMetadata, + Map> pageIndex2ChunkData) { + Set allChunkData = new HashSet<>(); + for (Map.Entry> entry : pageIndex2ChunkData.entrySet()) { + for (AlignedChunkData alignedChunkData : entry.getValue()) { + if (!allChunkData.contains(alignedChunkData)) { + alignedChunkData.addValueChunk(chunkOffset, header, chunkMetadata); + allChunkData.add(alignedChunkData); + } + } + } + } + + private LoadTsFilePieceNode getPieceNode( + String device, TTimePartitionSlot timePartitionSlot, DataPartition dataPartition) { + TRegionReplicaSet replicaSet = + dataPartition.getDataRegionReplicaSetForWriting(device, timePartitionSlot); + List pieceNodes = + replicaSet2Pieces.computeIfAbsent(replicaSet, o -> new ArrayList<>()); + if (pieceNodes.isEmpty() || pieceNodes.get(pieceNodes.size() - 1).exceedSize()) { + pieceNodes.add(new LoadTsFilePieceNode(getPlanNodeId(), tsFile)); + } + return pieceNodes.get(pieceNodes.size() - 1); + } +} diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/load/LoadTsFileNode.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/load/LoadTsFileNode.java new file mode 100644 index 000000000000..c991a9c4768b --- /dev/null +++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/load/LoadTsFileNode.java @@ -0,0 +1,106 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.mpp.plan.planner.plan.node.load; + +import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet; +import org.apache.iotdb.db.mpp.plan.analyze.Analysis; +import org.apache.iotdb.db.mpp.plan.planner.plan.node.PlanNode; +import org.apache.iotdb.db.mpp.plan.planner.plan.node.PlanNodeId; +import org.apache.iotdb.db.mpp.plan.planner.plan.node.WritePlanNode; +import org.apache.iotdb.tsfile.exception.NotImplementedException; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.DataOutputStream; +import java.io.File; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.List; + +public class LoadTsFileNode extends WritePlanNode { + private static final Logger logger = LoggerFactory.getLogger(LoadTsFileNode.class); + + private final List tsFiles; + + public LoadTsFileNode(PlanNodeId id) { + this(id, new ArrayList<>()); + } + + public LoadTsFileNode(PlanNodeId id, List tsFiles) { + super(id); + this.tsFiles = tsFiles; + } + + @Override + public TRegionReplicaSet getRegionReplicaSet() { + return null; + } + + @Override + public List getChildren() { + return null; + } + + @Override + public void addChild(PlanNode child) {} + + @Override + public PlanNode clone() { + throw new NotImplementedException("clone of load TsFile is not implemented"); + } + + @Override + public int allowedChildCount() { + return NO_CHILD_ALLOWED; + } + + @Override + public List getOutputColumnNames() { + return null; + } + + @Override + protected void serializeAttributes(ByteBuffer byteBuffer) {} + + @Override + protected void serializeAttributes(DataOutputStream stream) throws IOException {} + + @Override + public List splitByPartition(Analysis analysis) { + List res = new ArrayList<>(); + for (File file : tsFiles) { + try { + LoadSingleTsFileNode singleTsFileNode = new LoadSingleTsFileNode(getPlanNodeId(), file); + singleTsFileNode.checkIfNeedDecodeTsFile(analysis.getDataPartitionInfo()); + singleTsFileNode.autoRegisterSchema(); + + if (singleTsFileNode.needDecodeTsFile()) { + singleTsFileNode.splitTsFileByDataPartition(analysis.getDataPartitionInfo()); + } + res.add(singleTsFileNode); + } catch (Exception e) { + logger.error(String.format("Parse TsFile %s error", file.getPath()), e); + } + } + return res; + } +} diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/load/LoadTsFilePieceNode.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/load/LoadTsFilePieceNode.java new file mode 100644 index 000000000000..cb3ae522889d --- /dev/null +++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/load/LoadTsFilePieceNode.java @@ -0,0 +1,176 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.mpp.plan.planner.plan.node.load; + +import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet; +import org.apache.iotdb.db.conf.IoTDBConfig; +import org.apache.iotdb.db.conf.IoTDBDescriptor; +import org.apache.iotdb.db.engine.load.ChunkData; +import org.apache.iotdb.db.mpp.plan.analyze.Analysis; +import org.apache.iotdb.db.mpp.plan.planner.plan.node.PlanNode; +import org.apache.iotdb.db.mpp.plan.planner.plan.node.PlanNodeId; +import org.apache.iotdb.db.mpp.plan.planner.plan.node.PlanNodeType; +import org.apache.iotdb.db.mpp.plan.planner.plan.node.WritePlanNode; +import org.apache.iotdb.tsfile.exception.NotImplementedException; +import org.apache.iotdb.tsfile.exception.write.PageException; +import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.DataOutputStream; +import java.io.File; +import java.io.IOException; +import java.io.InputStream; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.List; + +public class LoadTsFilePieceNode extends WritePlanNode { + private static final Logger logger = LoggerFactory.getLogger(LoadTsFilePieceNode.class); + private static final IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig(); + + private File tsFile; + + private long dataSize; + private List chunkDataList; + + public LoadTsFilePieceNode(PlanNodeId id) { + super(id); + } + + public LoadTsFilePieceNode(PlanNodeId id, File tsFile) { + super(id); + this.tsFile = tsFile; + this.dataSize = 0; + this.chunkDataList = new ArrayList<>(); + } + + public boolean exceedSize() { + return dataSize >= config.getMaxPlanNodeSize(); + } + + public void addChunkData(ChunkData chunkData) { + chunkDataList.add(chunkData); + dataSize += chunkData.getDataSize(); + } + + public List getAllChunkData() { + return chunkDataList; + } + + public File getTsFile() { + return tsFile; + } + + @Override + public TRegionReplicaSet getRegionReplicaSet() { + return null; + } + + @Override + public List getChildren() { + return null; + } + + @Override + public void addChild(PlanNode child) {} + + @Override + public PlanNode clone() { + throw new NotImplementedException("clone of load piece TsFile is not implemented"); + } + + @Override + public int allowedChildCount() { + return NO_CHILD_ALLOWED; + } + + @Override + public List getOutputColumnNames() { + return null; + } + + @Override + protected void serializeAttributes(ByteBuffer byteBuffer) { + try { + ByteArrayOutputStream byteOutputStream = new ByteArrayOutputStream(); + DataOutputStream stream = new DataOutputStream(byteOutputStream); + serializeAttributes(stream); + byteBuffer.put(byteOutputStream.toByteArray()); + } catch (IOException e) { + logger.error("Serialize to ByteBuffer error.", e); + } + } + + @Override + protected void serializeAttributes(DataOutputStream stream) throws IOException { + PlanNodeType.LOAD_TSFILE.serialize(stream); + ReadWriteIOUtils.write(tsFile.getPath(), stream); // TODO: can save this space + ReadWriteIOUtils.write(chunkDataList.size(), stream); + for (ChunkData chunkData : chunkDataList) { + try { + chunkData.serialize(stream, tsFile); + } catch (IOException e) { + logger.error( + String.format( + "Parse page of TsFile %s error, skip chunk %s", tsFile.getPath(), chunkData)); + } + } + } + + @Override + public List splitByPartition(Analysis analysis) { + throw new NotImplementedException("split load piece TsFile is not implemented"); + } + + public static PlanNode deserialize(ByteBuffer buffer) { + InputStream stream = new ByteArrayInputStream(buffer.array()); + try { + ReadWriteIOUtils.readShort(stream); // read PlanNodeType + File tsFile = new File(ReadWriteIOUtils.readString(stream)); + LoadTsFilePieceNode pieceNode = new LoadTsFilePieceNode(new PlanNodeId(""), tsFile); + int chunkDataSize = ReadWriteIOUtils.readInt(stream); + for (int i = 0; i < chunkDataSize; i++) { + ChunkData chunkData = ChunkData.deserialize(stream); + pieceNode.addChunkData(chunkData); + } + pieceNode.setPlanNodeId(PlanNodeId.deserialize(stream)); + return pieceNode; + } catch (IOException | PageException e) { + logger.error(String.format("Deserialize %s error.", LoadTsFilePieceNode.class.getName()), e); + return null; + } + } + + @Override + public String toString() { + return "LoadTsFilePieceNode{" + + "tsFile=" + + tsFile + + ", dataSize=" + + dataSize + + ", chunkDataList=" + + chunkDataList + + '}'; + } +} diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/scheduler/load/LoadTsFileDispatcherImpl.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/scheduler/load/LoadTsFileDispatcherImpl.java new file mode 100644 index 000000000000..69733b2443cb --- /dev/null +++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/scheduler/load/LoadTsFileDispatcherImpl.java @@ -0,0 +1,239 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.mpp.plan.scheduler.load; + +import org.apache.iotdb.common.rpc.thrift.TDataNodeLocation; +import org.apache.iotdb.common.rpc.thrift.TEndPoint; +import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet; +import org.apache.iotdb.common.rpc.thrift.TSStatus; +import org.apache.iotdb.commons.client.IClientManager; +import org.apache.iotdb.commons.client.sync.SyncDataNodeInternalServiceClient; +import org.apache.iotdb.commons.consensus.ConsensusGroupId; +import org.apache.iotdb.commons.consensus.DataRegionId; +import org.apache.iotdb.db.conf.IoTDBDescriptor; +import org.apache.iotdb.db.engine.StorageEngineV2; +import org.apache.iotdb.db.exception.LoadFileException; +import org.apache.iotdb.db.exception.mpp.FragmentInstanceDispatchException; +import org.apache.iotdb.db.mpp.plan.planner.plan.FragmentInstance; +import org.apache.iotdb.db.mpp.plan.planner.plan.node.PlanNode; +import org.apache.iotdb.db.mpp.plan.planner.plan.node.PlanNodeType; +import org.apache.iotdb.db.mpp.plan.planner.plan.node.load.LoadSingleTsFileNode; +import org.apache.iotdb.db.mpp.plan.planner.plan.node.load.LoadTsFilePieceNode; +import org.apache.iotdb.db.mpp.plan.scheduler.FragInstanceDispatchResult; +import org.apache.iotdb.db.mpp.plan.scheduler.IFragInstanceDispatcher; +import org.apache.iotdb.mpp.rpc.thrift.TLoadCommandReq; +import org.apache.iotdb.mpp.rpc.thrift.TLoadResp; +import org.apache.iotdb.mpp.rpc.thrift.TTsFilePieceReq; +import org.apache.iotdb.rpc.RpcUtils; +import org.apache.iotdb.rpc.TSStatusCode; + +import io.airlift.concurrent.SetThreadName; +import org.apache.thrift.TException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.concurrent.Future; + +import static com.google.common.util.concurrent.Futures.immediateFuture; + +public class LoadTsFileDispatcherImpl implements IFragInstanceDispatcher { + private static final Logger logger = LoggerFactory.getLogger(LoadTsFileDispatcherImpl.class); + + private String uuid; + private final String localhostIpAddr; + private final int localhostInternalPort; + private final IClientManager + internalServiceClientManager; + + public LoadTsFileDispatcherImpl( + IClientManager internalServiceClientManager) { + this.internalServiceClientManager = internalServiceClientManager; + this.localhostIpAddr = IoTDBDescriptor.getInstance().getConfig().getInternalAddress(); + this.localhostInternalPort = IoTDBDescriptor.getInstance().getConfig().getInternalPort(); + } + + public void setUuid(String uuid) { + this.uuid = uuid; + } + + @Override + public Future dispatch(List instances) { + for (FragmentInstance instance : instances) { + try (SetThreadName threadName = + new SetThreadName(LoadTsFileScheduler.class.getName() + instance.getId().getFullId())) { + dispatchOneInstance(instance); + } catch (FragmentInstanceDispatchException e) { + return immediateFuture(new FragInstanceDispatchResult(e.getFailureStatus())); + } catch (Throwable t) { + logger.error("cannot dispatch FI for load operation", t); + return immediateFuture( + new FragInstanceDispatchResult( + RpcUtils.getStatus( + TSStatusCode.INTERNAL_SERVER_ERROR, "Unexpected errors: " + t.getMessage()))); + } + } + return immediateFuture(new FragInstanceDispatchResult(true)); + } + + private void dispatchOneInstance(FragmentInstance instance) + throws FragmentInstanceDispatchException { + for (TDataNodeLocation dataNodeLocation : + instance.getRegionReplicaSet().getDataNodeLocations()) { + TEndPoint endPoint = dataNodeLocation.getInternalEndPoint(); + if (isDispatchedToLocal(endPoint)) { + dispatchLocally(instance); + } else { + dispatchRemote(instance, endPoint); // TODO: can read only once + } + } + } + + private boolean isDispatchedToLocal(TEndPoint endPoint) { + return this.localhostIpAddr.equals(endPoint.getIp()) && localhostInternalPort == endPoint.port; + } + + private void dispatchRemote(FragmentInstance instance, TEndPoint endPoint) + throws FragmentInstanceDispatchException { + try (SyncDataNodeInternalServiceClient client = + internalServiceClientManager.borrowClient(endPoint)) { + TTsFilePieceReq loadTsFileReq = + new TTsFilePieceReq( + instance.getFragment().getPlanNodeTree().serializeToByteBuffer(), + uuid, + instance.getRegionReplicaSet().getRegionId()); + TLoadResp loadResp = client.sendTsFilePieceNode(loadTsFileReq); + if (!loadResp.isAccepted()) { + logger.error(loadResp.message); + throw new FragmentInstanceDispatchException(loadResp.status); + } + } catch (IOException | TException e) { + logger.error("can't connect to node {}", endPoint, e); + TSStatus status = new TSStatus(); + status.setCode(TSStatusCode.SYNC_CONNECTION_EXCEPTION.getStatusCode()); + status.setMessage("can't connect to node {}" + endPoint); + throw new FragmentInstanceDispatchException(status); + } + } + + public void dispatchLocally(FragmentInstance instance) throws FragmentInstanceDispatchException { + logger.info(String.format("Receive load node from uuid %s.", uuid)); + + ConsensusGroupId groupId = + ConsensusGroupId.Factory.createFromTConsensusGroupId( + instance.getRegionReplicaSet().getRegionId()); + PlanNode planNode = instance.getFragment().getPlanNodeTree(); + + if (planNode instanceof LoadTsFilePieceNode) { // split + LoadTsFilePieceNode pieceNode = + (LoadTsFilePieceNode) PlanNodeType.deserialize(planNode.serializeToByteBuffer()); + if (pieceNode == null) { + throw new FragmentInstanceDispatchException( + new TSStatus(TSStatusCode.NODE_DESERIALIZE_ERROR.getStatusCode())); + } + TSStatus resultStatus = + StorageEngineV2.getInstance() + .writeLoadTsFileNode((DataRegionId) groupId, pieceNode, uuid); + + if (!RpcUtils.SUCCESS_STATUS.equals(resultStatus)) { + throw new FragmentInstanceDispatchException(resultStatus); + } + } else if (planNode instanceof LoadSingleTsFileNode) { // do not need split + try { + StorageEngineV2.getInstance() + .getDataRegion((DataRegionId) groupId) + .loadNewTsFile(((LoadSingleTsFileNode) planNode).getTsFileResource(), true); + } catch (LoadFileException e) { + logger.error(String.format("Load TsFile Node %s error.", planNode), e); + TSStatus resultStatus = new TSStatus(); + resultStatus.setCode(TSStatusCode.LOAD_FILE_ERROR.getStatusCode()); + resultStatus.setMessage(e.getMessage()); + throw new FragmentInstanceDispatchException(resultStatus); + } + } + } + + public Future dispatchCommand( + TLoadCommandReq loadCommandReq, Set replicaSets) { + Set allEndPoint = new HashSet<>(); + for (TRegionReplicaSet replicaSet : replicaSets) { + for (TDataNodeLocation dataNodeLocation : replicaSet.getDataNodeLocations()) { + allEndPoint.add(dataNodeLocation.getInternalEndPoint()); + } + } + + for (TEndPoint endPoint : allEndPoint) { + try (SetThreadName threadName = + new SetThreadName( + LoadTsFileScheduler.class.getName() + "-" + loadCommandReq.commandType)) { + if (isDispatchedToLocal(endPoint)) { + dispatchLocally(loadCommandReq); + } else { + dispatchRemote(loadCommandReq, endPoint); + } + } catch (FragmentInstanceDispatchException e) { + return immediateFuture(new FragInstanceDispatchResult(e.getFailureStatus())); + } catch (Throwable t) { + logger.error("cannot dispatch LoadCommand for load operation", t); + return immediateFuture( + new FragInstanceDispatchResult( + RpcUtils.getStatus( + TSStatusCode.INTERNAL_SERVER_ERROR, "Unexpected errors: " + t.getMessage()))); + } + } + return immediateFuture(new FragInstanceDispatchResult(true)); + } + + private void dispatchRemote(TLoadCommandReq loadCommandReq, TEndPoint endPoint) + throws FragmentInstanceDispatchException { + try (SyncDataNodeInternalServiceClient client = + internalServiceClientManager.borrowClient(endPoint)) { + TLoadResp loadResp = client.sendLoadCommand(loadCommandReq); + if (!loadResp.isAccepted()) { + logger.error(loadResp.message); + throw new FragmentInstanceDispatchException(loadResp.status); + } + } catch (IOException | TException e) { + logger.error("can't connect to node {}", endPoint, e); + TSStatus status = new TSStatus(); + status.setCode(TSStatusCode.SYNC_CONNECTION_EXCEPTION.getStatusCode()); + status.setMessage("can't connect to node {}" + endPoint); + throw new FragmentInstanceDispatchException(status); + } + } + + private void dispatchLocally(TLoadCommandReq loadCommandReq) + throws FragmentInstanceDispatchException { + TSStatus resultStatus = + StorageEngineV2.getInstance() + .executeLoadCommand( + LoadTsFileScheduler.LoadCommand.values()[loadCommandReq.commandType], + loadCommandReq.uuid); + if (!RpcUtils.SUCCESS_STATUS.equals(resultStatus)) { + throw new FragmentInstanceDispatchException(resultStatus); + } + } + + @Override + public void abort() {} +} diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/scheduler/load/LoadTsFileScheduler.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/scheduler/load/LoadTsFileScheduler.java new file mode 100644 index 000000000000..cad9b8b29764 --- /dev/null +++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/scheduler/load/LoadTsFileScheduler.java @@ -0,0 +1,252 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.mpp.plan.scheduler.load; + +import org.apache.iotdb.common.rpc.thrift.TEndPoint; +import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet; +import org.apache.iotdb.common.rpc.thrift.TSStatus; +import org.apache.iotdb.commons.client.IClientManager; +import org.apache.iotdb.commons.client.sync.SyncDataNodeInternalServiceClient; +import org.apache.iotdb.db.exception.mpp.FragmentInstanceDispatchException; +import org.apache.iotdb.db.mpp.common.FragmentInstanceId; +import org.apache.iotdb.db.mpp.common.MPPQueryContext; +import org.apache.iotdb.db.mpp.common.PlanFragmentId; +import org.apache.iotdb.db.mpp.execution.QueryStateMachine; +import org.apache.iotdb.db.mpp.execution.fragment.FragmentInfo; +import org.apache.iotdb.db.mpp.plan.planner.plan.DistributedQueryPlan; +import org.apache.iotdb.db.mpp.plan.planner.plan.FragmentInstance; +import org.apache.iotdb.db.mpp.plan.planner.plan.PlanFragment; +import org.apache.iotdb.db.mpp.plan.planner.plan.node.load.LoadSingleTsFileNode; +import org.apache.iotdb.db.mpp.plan.planner.plan.node.load.LoadTsFilePieceNode; +import org.apache.iotdb.db.mpp.plan.scheduler.FragInstanceDispatchResult; +import org.apache.iotdb.db.mpp.plan.scheduler.IScheduler; +import org.apache.iotdb.mpp.rpc.thrift.TLoadCommandReq; +import org.apache.iotdb.rpc.TSStatusCode; + +import io.airlift.units.Duration; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; + +/** + * {@link LoadTsFileScheduler} is used for scheduling {@link LoadSingleTsFileNode} and {@link + * LoadTsFilePieceNode}. because these two nodes need two phases to finish transfer. + * + *

for more details please check: ...; + */ +public class LoadTsFileScheduler implements IScheduler { + private static final Logger logger = LoggerFactory.getLogger(LoadTsFileScheduler.class); + + private final MPPQueryContext queryContext; + private QueryStateMachine stateMachine; + private LoadTsFileDispatcherImpl dispatcher; + private List tsFileNodeList; + private PlanFragmentId fragmentId; + + private Set allReplicaSets; + + public LoadTsFileScheduler( + DistributedQueryPlan distributedQueryPlan, + MPPQueryContext queryContext, + QueryStateMachine stateMachine, + IClientManager internalServiceClientManager) { + this.queryContext = queryContext; + this.stateMachine = stateMachine; + this.tsFileNodeList = new ArrayList<>(); + this.fragmentId = distributedQueryPlan.getRootSubPlan().getPlanFragment().getId(); + this.dispatcher = new LoadTsFileDispatcherImpl(internalServiceClientManager); + this.allReplicaSets = new HashSet<>(); + + for (FragmentInstance fragmentInstance : distributedQueryPlan.getInstances()) { + tsFileNodeList.add((LoadSingleTsFileNode) fragmentInstance.getFragment().getPlanNodeTree()); + } + } + + @Override + public void start() { + stateMachine.transitionToRunning(); + for (LoadSingleTsFileNode node : tsFileNodeList) { + if (!node.needDecodeTsFile()) { + boolean isLoadLocallySuccess = loadLocally(node); + if (!isLoadLocallySuccess) { + return; + } + continue; + } + + String uuid = UUID.randomUUID().toString(); + dispatcher.setUuid(uuid); + allReplicaSets.clear(); + + boolean isFirstPhaseSuccess = firstPhase(node); + boolean isSecondPhaseSuccess = secondPhase(isFirstPhaseSuccess, uuid); + if (!isFirstPhaseSuccess || !isSecondPhaseSuccess) { + return; + } + } + stateMachine.transitionToFinished(); + } + + private boolean firstPhase(LoadSingleTsFileNode node) { + if (!dispatchOneTsFile(node)) { + logger.error( + String.format("Dispatch Single TsFile Node error, LoadSingleTsFileNode %s.", node)); + return false; + } + return true; + } + + private boolean dispatchOneTsFile(LoadSingleTsFileNode node) { + for (Map.Entry> entry : + node.getReplicaSet2Pieces().entrySet()) { + allReplicaSets.add(entry.getKey()); + for (LoadTsFilePieceNode pieceNode : entry.getValue()) { + FragmentInstance instance = + new FragmentInstance( + new PlanFragment(fragmentId, pieceNode), + fragmentId.genFragmentInstanceId(), + null, + queryContext.getQueryType(), + queryContext.getTimeOut()); + instance.setDataRegionAndHost(entry.getKey()); + Future dispatchResultFuture = + dispatcher.dispatch(Collections.singletonList(instance)); + + try { + FragInstanceDispatchResult result = dispatchResultFuture.get(); + if (!result.isSuccessful()) { + // TODO: retry. + logger.error( + String.format( + "Dispatch one piece to ReplicaSet %s error, result status code %s.", + entry.getKey(), + TSStatusCode.representOf(result.getFailureStatus().getCode()).name())); + logger.error( + String.format("Result status message %s.", result.getFailureStatus().getMessage())); + if (result.getFailureStatus().getSubStatus() != null) { + for (TSStatus status : result.getFailureStatus().getSubStatus()) { + logger.error( + String.format( + "Sub status code %s.", TSStatusCode.representOf(status.getCode()).name())); + logger.error(String.format("Sub status message %s.", status.getMessage())); + } + } + logger.error(String.format("Dispatch piece node:%n%s", pieceNode)); + stateMachine.transitionToFailed(result.getFailureStatus()); // TODO: record more status + return false; + } + } catch (InterruptedException | ExecutionException e) { + if (e instanceof InterruptedException) { + Thread.currentThread().interrupt(); + } + logger.warn("Interrupt or Execution error.", e); + stateMachine.transitionToFailed(e); + return false; + } + } + } + return true; + } + + private boolean secondPhase(boolean isFirstPhaseSuccess, String uuid) { + TLoadCommandReq loadCommandReq = + new TLoadCommandReq( + (isFirstPhaseSuccess ? LoadCommand.EXECUTE : LoadCommand.ROLLBACK).ordinal(), uuid); + Future dispatchResultFuture = + dispatcher.dispatchCommand(loadCommandReq, allReplicaSets); + + try { + FragInstanceDispatchResult result = dispatchResultFuture.get(); + if (!result.isSuccessful()) { + // TODO: retry. + logger.error( + String.format("Dispatch LoadCommand error to replicaSets %s error.", allReplicaSets)); + logger.error(String.format("Result status code %s.", result.getFailureStatus().getCode())); + logger.error( + String.format("Result status message %s.", result.getFailureStatus().getMessage())); + stateMachine.transitionToFailed(result.getFailureStatus()); + return false; + } + } catch (InterruptedException | ExecutionException e) { + if (e instanceof InterruptedException) { + Thread.currentThread().interrupt(); + } + logger.warn("Interrupt or Execution error.", e); + stateMachine.transitionToFailed(e); + return false; + } + return true; + } + + private boolean loadLocally(LoadSingleTsFileNode node) { + try { + FragmentInstance instance = + new FragmentInstance( + new PlanFragment(fragmentId, node), + fragmentId.genFragmentInstanceId(), + null, + queryContext.getQueryType(), + queryContext.getTimeOut()); + instance.setDataRegionAndHost(node.getLocalRegionReplicaSet()); + dispatcher.dispatchLocally(instance); + } catch (FragmentInstanceDispatchException e) { + logger.error("Dispatch LoadCommand error to local error."); + logger.error(String.format("Result status code %s.", e.getFailureStatus().getCode())); + logger.error(String.format("Result status message %s.", e.getFailureStatus().getMessage())); + stateMachine.transitionToFailed(e.getFailureStatus()); + return false; + } + return true; + } + + @Override + public void stop() {} + + @Override + public Duration getTotalCpuTime() { + return null; + } + + @Override + public FragmentInfo getFragmentInfo() { + return null; + } + + @Override + public void abortFragmentInstance(FragmentInstanceId instanceId, Throwable failureCause) {} + + @Override + public void cancelFragment(PlanFragmentId planFragmentId) {} + + public enum LoadCommand { + EXECUTE, + ROLLBACK + } +} diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/StatementVisitor.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/StatementVisitor.java index de9ad46b779b..fba7acdc9ad5 100644 --- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/StatementVisitor.java +++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/StatementVisitor.java @@ -26,6 +26,7 @@ import org.apache.iotdb.db.mpp.plan.statement.crud.InsertRowsStatement; import org.apache.iotdb.db.mpp.plan.statement.crud.InsertStatement; import org.apache.iotdb.db.mpp.plan.statement.crud.InsertTabletStatement; +import org.apache.iotdb.db.mpp.plan.statement.crud.LoadTsFileStatement; import org.apache.iotdb.db.mpp.plan.statement.crud.QueryStatement; import org.apache.iotdb.db.mpp.plan.statement.internal.InternalCreateTimeSeriesStatement; import org.apache.iotdb.db.mpp.plan.statement.internal.SchemaFetchStatement; @@ -201,6 +202,10 @@ public R visitInsertTablet(InsertTabletStatement insertTabletStatement, C contex return visitStatement(insertTabletStatement, context); } + public R visitLoadFile(LoadTsFileStatement loadTsFileStatement, C context) { + return visitStatement(loadTsFileStatement, context); + } + /** Data Control Language (DCL) */ public R visitAuthor(AuthorStatement authorStatement, C context) { return visitStatement(authorStatement, context); diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/crud/LoadTsFileStatement.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/crud/LoadTsFileStatement.java new file mode 100644 index 000000000000..4fb39cad1a71 --- /dev/null +++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/crud/LoadTsFileStatement.java @@ -0,0 +1,130 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.mpp.plan.statement.crud; + +import org.apache.iotdb.commons.path.PartialPath; +import org.apache.iotdb.db.conf.IoTDBDescriptor; +import org.apache.iotdb.db.mpp.plan.statement.Statement; +import org.apache.iotdb.db.mpp.plan.statement.StatementVisitor; +import org.apache.iotdb.tsfile.common.constant.TsFileConstant; +import org.apache.iotdb.tsfile.utils.FilePathUtils; + +import java.io.File; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +public class LoadTsFileStatement extends Statement { + private File file; + private boolean autoCreateSchema; + private int sgLevel; + private boolean verifySchema; + + private List tsFiles; + + public LoadTsFileStatement(String filePath) { + this.file = new File(filePath); + this.autoCreateSchema = true; + this.sgLevel = IoTDBDescriptor.getInstance().getConfig().getDefaultStorageGroupLevel(); + this.verifySchema = true; + + tsFiles = new ArrayList<>(); + if (file.isFile()) { + tsFiles.add(file); + } else { + findAllTsFile(file); + } + sortTsFiles(tsFiles); + } + + private void findAllTsFile(File file) { + for (File nowFile : file.listFiles()) { + if (nowFile.getName().endsWith(TsFileConstant.TSFILE_SUFFIX)) { + tsFiles.add(nowFile); + } else if (nowFile.isDirectory()) { + findAllTsFile(nowFile); + } + } + } + + private void sortTsFiles(List files) { + Map file2Timestamp = new HashMap<>(); + Map file2Version = new HashMap<>(); + for (File file : files) { + String[] splitStrings = file.getName().split(FilePathUtils.FILE_NAME_SEPARATOR); + file2Timestamp.put(file, Long.parseLong(splitStrings[0])); + file2Version.put(file, Long.parseLong(splitStrings[1])); + } + + Collections.sort( + files, + (o1, o2) -> { + long timestampDiff = file2Timestamp.get(o1) - file2Timestamp.get(o2); + if (timestampDiff != 0) { + return (int) (timestampDiff); + } + return (int) (file2Version.get(o1) - file2Version.get(o2)); + }); + } + + public void setAutoCreateSchema(boolean autoCreateSchema) { + this.autoCreateSchema = autoCreateSchema; + } + + public void setSgLevel(int sgLevel) { + this.sgLevel = sgLevel; + } + + public void setVerifySchema(boolean verifySchema) { + this.verifySchema = verifySchema; + } + + public List getTsFiles() { + return tsFiles; + } + + @Override + public List getPaths() { + return Collections.emptyList(); + } + + @Override + public R accept(StatementVisitor visitor, C context) { + return visitor.visitLoadFile(this, context); + } + + @Override + public String toString() { + return "LoadTsFileStatement{" + + "file=" + + file + + ", autoCreateSchema=" + + autoCreateSchema + + ", sgLevel=" + + sgLevel + + ", verifySchema=" + + verifySchema + + ", tsFiles Size=" + + tsFiles.size() + + '}'; + } +} diff --git a/server/src/main/java/org/apache/iotdb/db/service/thrift/impl/DataNodeInternalRPCServiceImpl.java b/server/src/main/java/org/apache/iotdb/db/service/thrift/impl/DataNodeInternalRPCServiceImpl.java index bc487251b274..a63b68427416 100644 --- a/server/src/main/java/org/apache/iotdb/db/service/thrift/impl/DataNodeInternalRPCServiceImpl.java +++ b/server/src/main/java/org/apache/iotdb/db/service/thrift/impl/DataNodeInternalRPCServiceImpl.java @@ -67,10 +67,12 @@ import org.apache.iotdb.db.mpp.plan.planner.plan.node.PlanNode; import org.apache.iotdb.db.mpp.plan.planner.plan.node.PlanNodeId; import org.apache.iotdb.db.mpp.plan.planner.plan.node.PlanNodeType; +import org.apache.iotdb.db.mpp.plan.planner.plan.node.load.LoadTsFilePieceNode; import org.apache.iotdb.db.mpp.plan.planner.plan.node.metedata.write.ConstructSchemaBlackListNode; import org.apache.iotdb.db.mpp.plan.planner.plan.node.metedata.write.DeleteTimeSeriesNode; import org.apache.iotdb.db.mpp.plan.planner.plan.node.metedata.write.RollbackSchemaBlackListNode; import org.apache.iotdb.db.mpp.plan.planner.plan.node.write.DeleteDataNode; +import org.apache.iotdb.db.mpp.plan.scheduler.load.LoadTsFileScheduler; import org.apache.iotdb.db.service.DataNode; import org.apache.iotdb.db.service.RegionMigrateService; import org.apache.iotdb.db.service.metrics.MetricService; @@ -105,6 +107,8 @@ import org.apache.iotdb.mpp.rpc.thrift.TInvalidateCacheReq; import org.apache.iotdb.mpp.rpc.thrift.TInvalidateMatchedSchemaCacheReq; import org.apache.iotdb.mpp.rpc.thrift.TInvalidatePermissionCacheReq; +import org.apache.iotdb.mpp.rpc.thrift.TLoadCommandReq; +import org.apache.iotdb.mpp.rpc.thrift.TLoadResp; import org.apache.iotdb.mpp.rpc.thrift.TMaintainPeerReq; import org.apache.iotdb.mpp.rpc.thrift.TRegionLeaderChangeReq; import org.apache.iotdb.mpp.rpc.thrift.TRegionRouteReq; @@ -115,6 +119,7 @@ import org.apache.iotdb.mpp.rpc.thrift.TSendFragmentInstanceResp; import org.apache.iotdb.mpp.rpc.thrift.TSendPlanNodeReq; import org.apache.iotdb.mpp.rpc.thrift.TSendPlanNodeResp; +import org.apache.iotdb.mpp.rpc.thrift.TTsFilePieceReq; import org.apache.iotdb.mpp.rpc.thrift.TUpdateConfigNodeGroupReq; import org.apache.iotdb.mpp.rpc.thrift.TUpdateTemplateReq; import org.apache.iotdb.mpp.rpc.thrift.TactiveTriggerInstanceReq; @@ -267,6 +272,44 @@ public TSchemaFetchResponse fetchSchema(TSchemaFetchRequest req) { throw new UnsupportedOperationException(); } + @Override + public TLoadResp sendTsFilePieceNode(TTsFilePieceReq req) throws TException { + LOGGER.info(String.format("Receive load node from uuid %s.", req.uuid)); + + ConsensusGroupId groupId = + ConsensusGroupId.Factory.createFromTConsensusGroupId(req.consensusGroupId); + LoadTsFilePieceNode pieceNode = (LoadTsFilePieceNode) PlanNodeType.deserialize(req.body); + if (pieceNode == null) { + return createTLoadResp(new TSStatus(TSStatusCode.NODE_DESERIALIZE_ERROR.getStatusCode())); + } + + TSStatus resultStatus = + StorageEngineV2.getInstance() + .writeLoadTsFileNode((DataRegionId) groupId, pieceNode, req.uuid); + + return createTLoadResp(resultStatus); + } + + @Override + public TLoadResp sendLoadCommand(TLoadCommandReq req) throws TException { + + TSStatus resultStatus = + StorageEngineV2.getInstance() + .executeLoadCommand( + LoadTsFileScheduler.LoadCommand.values()[req.commandType], req.uuid); + return createTLoadResp(resultStatus); + } + + private TLoadResp createTLoadResp(TSStatus resultStatus) { + boolean isAccepted = RpcUtils.SUCCESS_STATUS.equals(resultStatus); + TLoadResp loadResp = new TLoadResp(isAccepted); + if (!isAccepted) { + loadResp.setMessage(resultStatus.getMessage()); + loadResp.setStatus(resultStatus); + } + return loadResp; + } + @Override public TSStatus createSchemaRegion(TCreateSchemaRegionReq req) { return regionManager.createSchemaRegion(req.getRegionReplicaSet(), req.getStorageGroup()); diff --git a/service-rpc/src/main/java/org/apache/iotdb/rpc/TSStatusCode.java b/service-rpc/src/main/java/org/apache/iotdb/rpc/TSStatusCode.java index 4c11f4b5df7b..8e714f4d1acd 100644 --- a/service-rpc/src/main/java/org/apache/iotdb/rpc/TSStatusCode.java +++ b/service-rpc/src/main/java/org/apache/iotdb/rpc/TSStatusCode.java @@ -91,6 +91,7 @@ public enum TSStatusCode { QUERY_ID_NOT_EXIST(414), SNAPSHOT_DIR_NOT_LEGAL(415), SEMANTIC_ERROR(416), + TSFILE_RUNTIME_ERROR(417), UNSUPPORTED_INDEX_FUNC_ERROR(421), UNSUPPORTED_INDEX_TYPE_ERROR(422), @@ -131,6 +132,7 @@ public enum TSStatusCode { CREATE_REGION_ERROR(711), DELETE_REGION_ERROR(712), CACHE_UPDATE_FAIL(713), + NODE_DESERIALIZE_ERROR(714), // configuration CONFIG_ERROR(800), diff --git a/thrift/src/main/thrift/datanode.thrift b/thrift/src/main/thrift/datanode.thrift index db8ebf20fb95..0f303294c61c 100644 --- a/thrift/src/main/thrift/datanode.thrift +++ b/thrift/src/main/thrift/datanode.thrift @@ -216,6 +216,23 @@ struct TUpdateTemplateReq{ 2: required binary templateInfo } +struct TTsFilePieceReq{ + 1: required binary body + 2: required string uuid + 3: required common.TConsensusGroupId consensusGroupId +} + +struct TLoadCommandReq{ + 1: required i32 commandType + 2: required string uuid +} + +struct TLoadResp{ + 1: required bool accepted + 2: optional string message + 3: optional common.TSStatus status +} + struct TConstructSchemaBlackListReq{ 1: required list schemaRegionIdList 2: required binary pathPatternTree @@ -274,6 +291,10 @@ service IDataNodeRPCService { TSchemaFetchResponse fetchSchema(TSchemaFetchRequest req) + TLoadResp sendTsFilePieceNode(TTsFilePieceReq req); + + TLoadResp sendLoadCommand(TLoadCommandReq req); + // -----------------------------------For Config Node----------------------------------------------- diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/header/ChunkHeader.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/header/ChunkHeader.java index 05b32e81f40f..24a441870221 100644 --- a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/header/ChunkHeader.java +++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/header/ChunkHeader.java @@ -121,7 +121,8 @@ public ChunkHeader( /** the exact serialized size of chunk header */ public static int getSerializedSize(String measurementID, int dataSize) { - int measurementIdLength = measurementID.getBytes(TSFileConfig.STRING_CHARSET).length; + int measurementIdLength = + measurementID == null ? 0 : measurementID.getBytes(TSFileConfig.STRING_CHARSET).length; return Byte.BYTES // chunkType + ReadWriteForEncodingUtils.varIntSize(measurementIdLength) // measurementID length + measurementIdLength // measurementID diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/TsFileSequenceReader.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/TsFileSequenceReader.java index 452ecb938176..2c1020c1bd03 100644 --- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/TsFileSequenceReader.java +++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/TsFileSequenceReader.java @@ -1075,7 +1075,7 @@ private ChunkHeader readChunkHeader(long position, int chunkHeaderSize) throws I * @param position the offset of the chunk data * @return the pages of this chunk */ - private ByteBuffer readChunk(long position, int dataSize) throws IOException { + public ByteBuffer readChunk(long position, int dataSize) throws IOException { try { return readData(position, dataSize); } catch (Throwable t) { diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/write/chunk/AlignedChunkWriterImpl.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/write/chunk/AlignedChunkWriterImpl.java index a1fc973d600b..2737317be36f 100644 --- a/tsfile/src/main/java/org/apache/iotdb/tsfile/write/chunk/AlignedChunkWriterImpl.java +++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/write/chunk/AlignedChunkWriterImpl.java @@ -21,6 +21,8 @@ import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor; import org.apache.iotdb.tsfile.encoding.encoder.Encoder; import org.apache.iotdb.tsfile.encoding.encoder.TSEncodingBuilder; +import org.apache.iotdb.tsfile.exception.write.PageException; +import org.apache.iotdb.tsfile.file.header.PageHeader; import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType; import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding; import org.apache.iotdb.tsfile.read.common.block.column.Column; @@ -32,6 +34,7 @@ import org.apache.iotdb.tsfile.write.writer.TsFileIOWriter; import java.io.IOException; +import java.nio.ByteBuffer; import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.List; @@ -125,6 +128,30 @@ public void write(long time, Binary value, boolean isNull) { valueChunkWriterList.get(valueIndex++).write(time, value, isNull); } + public void write(long time, int value, boolean isNull, int valueIndex) { + valueChunkWriterList.get(valueIndex).write(time, value, isNull); + } + + public void write(long time, long value, boolean isNull, int valueIndex) { + valueChunkWriterList.get(valueIndex).write(time, value, isNull); + } + + public void write(long time, boolean value, boolean isNull, int valueIndex) { + valueChunkWriterList.get(valueIndex).write(time, value, isNull); + } + + public void write(long time, float value, boolean isNull, int valueIndex) { + valueChunkWriterList.get(valueIndex).write(time, value, isNull); + } + + public void write(long time, double value, boolean isNull, int valueIndex) { + valueChunkWriterList.get(valueIndex).write(time, value, isNull); + } + + public void write(long time, Binary value, boolean isNull, int valueIndex) { + valueChunkWriterList.get(valueIndex).write(time, value, isNull); + } + public void write(long time, TsPrimitiveType[] points) { valueIndex = 0; for (TsPrimitiveType point : points) { @@ -270,6 +297,16 @@ private void writePageToPageBuffer() { } } + public void writePageHeaderAndDataIntoTimeBuff(ByteBuffer data, PageHeader header) + throws PageException { + timeChunkWriter.writePageHeaderAndDataIntoBuff(data, header); + } + + public void writePageHeaderAndDataIntoValueBuff( + ByteBuffer data, PageHeader header, int valueIndex) throws PageException { + valueChunkWriterList.get(valueIndex).writePageHeaderAndDataIntoBuff(data, header); + } + @Override public void writeToFileWriter(TsFileIOWriter tsfileWriter) throws IOException { timeChunkWriter.writeToFileWriter(tsfileWriter); diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/write/chunk/TimeChunkWriter.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/write/chunk/TimeChunkWriter.java index 59a1fd465b6a..dc2f2d15f889 100644 --- a/tsfile/src/main/java/org/apache/iotdb/tsfile/write/chunk/TimeChunkWriter.java +++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/write/chunk/TimeChunkWriter.java @@ -22,6 +22,7 @@ import org.apache.iotdb.tsfile.common.constant.TsFileConstant; import org.apache.iotdb.tsfile.compress.ICompressor; import org.apache.iotdb.tsfile.encoding.encoder.Encoder; +import org.apache.iotdb.tsfile.exception.write.PageException; import org.apache.iotdb.tsfile.file.header.ChunkHeader; import org.apache.iotdb.tsfile.file.header.PageHeader; import org.apache.iotdb.tsfile.file.metadata.enums.CompressionType; @@ -30,6 +31,7 @@ import org.apache.iotdb.tsfile.file.metadata.statistics.Statistics; import org.apache.iotdb.tsfile.file.metadata.statistics.TimeStatistics; import org.apache.iotdb.tsfile.utils.PublicBAOS; +import org.apache.iotdb.tsfile.utils.ReadWriteForEncodingUtils; import org.apache.iotdb.tsfile.write.page.TimePageWriter; import org.apache.iotdb.tsfile.write.writer.TsFileIOWriter; @@ -37,6 +39,9 @@ import org.slf4j.LoggerFactory; import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.channels.Channels; +import java.nio.channels.WritableByteChannel; public class TimeChunkWriter { @@ -168,6 +173,53 @@ public void writePageToPageBuffer() { } } + public void writePageHeaderAndDataIntoBuff(ByteBuffer data, PageHeader header) + throws PageException { + // write the page header to pageBuffer + try { + logger.debug( + "start to flush a page header into buffer, buffer position {} ", pageBuffer.size()); + // serialize pageHeader see writePageToPageBuffer method + if (numOfPages == 0) { // record the firstPageStatistics + this.firstPageStatistics = header.getStatistics(); + this.sizeWithoutStatistic += + ReadWriteForEncodingUtils.writeUnsignedVarInt(header.getUncompressedSize(), pageBuffer); + this.sizeWithoutStatistic += + ReadWriteForEncodingUtils.writeUnsignedVarInt(header.getCompressedSize(), pageBuffer); + } else if (numOfPages == 1) { // put the firstPageStatistics into pageBuffer + byte[] b = pageBuffer.toByteArray(); + pageBuffer.reset(); + pageBuffer.write(b, 0, this.sizeWithoutStatistic); + firstPageStatistics.serialize(pageBuffer); + pageBuffer.write(b, this.sizeWithoutStatistic, b.length - this.sizeWithoutStatistic); + ReadWriteForEncodingUtils.writeUnsignedVarInt(header.getUncompressedSize(), pageBuffer); + ReadWriteForEncodingUtils.writeUnsignedVarInt(header.getCompressedSize(), pageBuffer); + header.getStatistics().serialize(pageBuffer); + firstPageStatistics = null; + } else { + ReadWriteForEncodingUtils.writeUnsignedVarInt(header.getUncompressedSize(), pageBuffer); + ReadWriteForEncodingUtils.writeUnsignedVarInt(header.getCompressedSize(), pageBuffer); + header.getStatistics().serialize(pageBuffer); + } + logger.debug( + "finish to flush a page header {} of time page into buffer, buffer position {} ", + header, + pageBuffer.size()); + + statistics.mergeStatistics(header.getStatistics()); + + } catch (IOException e) { + throw new PageException("IO Exception in writeDataPageHeader,ignore this page", e); + } + numOfPages++; + // write page content to temp PBAOS + try (WritableByteChannel channel = Channels.newChannel(pageBuffer)) { + channel.write(data); + } catch (IOException e) { + throw new PageException(e); + } + } + public void writeToFileWriter(TsFileIOWriter tsfileWriter) throws IOException { sealCurrentPage(); writeAllPagesOfChunkToTsFile(tsfileWriter); diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/write/chunk/ValueChunkWriter.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/write/chunk/ValueChunkWriter.java index 8b75269388f6..98d67a695444 100644 --- a/tsfile/src/main/java/org/apache/iotdb/tsfile/write/chunk/ValueChunkWriter.java +++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/write/chunk/ValueChunkWriter.java @@ -22,6 +22,7 @@ import org.apache.iotdb.tsfile.common.constant.TsFileConstant; import org.apache.iotdb.tsfile.compress.ICompressor; import org.apache.iotdb.tsfile.encoding.encoder.Encoder; +import org.apache.iotdb.tsfile.exception.write.PageException; import org.apache.iotdb.tsfile.file.header.ChunkHeader; import org.apache.iotdb.tsfile.file.header.PageHeader; import org.apache.iotdb.tsfile.file.metadata.enums.CompressionType; @@ -30,6 +31,7 @@ import org.apache.iotdb.tsfile.file.metadata.statistics.Statistics; import org.apache.iotdb.tsfile.utils.Binary; import org.apache.iotdb.tsfile.utils.PublicBAOS; +import org.apache.iotdb.tsfile.utils.ReadWriteForEncodingUtils; import org.apache.iotdb.tsfile.write.page.ValuePageWriter; import org.apache.iotdb.tsfile.write.writer.TsFileIOWriter; @@ -38,6 +40,9 @@ import java.io.IOException; import java.io.Serializable; +import java.nio.ByteBuffer; +import java.nio.channels.Channels; +import java.nio.channels.WritableByteChannel; public class ValueChunkWriter { @@ -187,6 +192,57 @@ public void writePageToPageBuffer() { } } + public void writePageHeaderAndDataIntoBuff(ByteBuffer data, PageHeader header) + throws PageException { + // write the page header to pageBuffer + try { + logger.debug( + "start to flush a page header into buffer, buffer position {} ", pageBuffer.size()); + // serialize pageHeader see writePageToPageBuffer method + if (numOfPages == 0) { // record the firstPageStatistics + if (header.getStatistics() != null) { + this.firstPageStatistics = header.getStatistics(); + } + this.sizeWithoutStatistic += + ReadWriteForEncodingUtils.writeUnsignedVarInt(header.getUncompressedSize(), pageBuffer); + this.sizeWithoutStatistic += + ReadWriteForEncodingUtils.writeUnsignedVarInt(header.getCompressedSize(), pageBuffer); + } else if (numOfPages == 1) { // put the firstPageStatistics into pageBuffer + if (firstPageStatistics != null) { + byte[] b = pageBuffer.toByteArray(); + pageBuffer.reset(); + pageBuffer.write(b, 0, this.sizeWithoutStatistic); + firstPageStatistics.serialize(pageBuffer); + pageBuffer.write(b, this.sizeWithoutStatistic, b.length - this.sizeWithoutStatistic); + } + ReadWriteForEncodingUtils.writeUnsignedVarInt(header.getUncompressedSize(), pageBuffer); + ReadWriteForEncodingUtils.writeUnsignedVarInt(header.getCompressedSize(), pageBuffer); + header.getStatistics().serialize(pageBuffer); + firstPageStatistics = null; + } else { + ReadWriteForEncodingUtils.writeUnsignedVarInt(header.getUncompressedSize(), pageBuffer); + ReadWriteForEncodingUtils.writeUnsignedVarInt(header.getCompressedSize(), pageBuffer); + header.getStatistics().serialize(pageBuffer); + } + logger.debug( + "finish to flush a page header {} of time page into buffer, buffer position {} ", + header, + pageBuffer.size()); + + statistics.mergeStatistics(header.getStatistics()); + + } catch (IOException e) { + throw new PageException("IO Exception in writeDataPageHeader,ignore this page", e); + } + numOfPages++; + // write page content to temp PBAOS + try (WritableByteChannel channel = Channels.newChannel(pageBuffer)) { + channel.write(data); + } catch (IOException e) { + throw new PageException(e); + } + } + public void writeToFileWriter(TsFileIOWriter tsfileWriter) throws IOException { sealCurrentPage(); writeAllPagesOfChunkToTsFile(tsfileWriter); diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/write/writer/TsFileIOWriter.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/write/writer/TsFileIOWriter.java index 2f865f297f08..22a68bdfd761 100644 --- a/tsfile/src/main/java/org/apache/iotdb/tsfile/write/writer/TsFileIOWriter.java +++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/write/writer/TsFileIOWriter.java @@ -234,6 +234,19 @@ public void writeChunk(Chunk chunk, ChunkMetadata chunkMetadata) throws IOExcept } } + public void writeChunk(Chunk chunk) throws IOException { + ChunkHeader chunkHeader = chunk.getHeader(); + currentChunkMetadata = + new ChunkMetadata( + chunkHeader.getMeasurementID(), + chunkHeader.getDataType(), + out.getPosition(), + chunk.getChunkStatistic()); + chunkHeader.serializeTo(out.wrapAsStream()); + out.write(chunk.getData()); + endCurrentChunk(); + } + /** end chunk and write some log. */ public void endCurrentChunk() { chunkMetadataList.add(currentChunkMetadata);