diff --git a/integration-test/src/test/java/org/apache/iotdb/relational/it/db/it/udf/IoTDBSQLFunctionManagementIT.java b/integration-test/src/test/java/org/apache/iotdb/relational/it/db/it/udf/IoTDBSQLFunctionManagementIT.java index f414171cf469d..f56da20b05176 100644 --- a/integration-test/src/test/java/org/apache/iotdb/relational/it/db/it/udf/IoTDBSQLFunctionManagementIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/relational/it/db/it/udf/IoTDBSQLFunctionManagementIT.java @@ -21,6 +21,7 @@ import org.apache.iotdb.commons.queryengine.plan.relational.function.TableBuiltinTableFunction; import org.apache.iotdb.commons.udf.builtin.relational.TableBuiltinAggregationFunction; import org.apache.iotdb.commons.udf.builtin.relational.TableBuiltinScalarFunction; +import org.apache.iotdb.db.queryengine.plan.relational.function.DataNodeTableBuiltinTableFunction; import org.apache.iotdb.it.env.EnvFactory; import org.apache.iotdb.it.framework.IoTDBTestRunner; import org.apache.iotdb.itbase.category.TableClusterIT; @@ -55,7 +56,7 @@ public class IoTDBSQLFunctionManagementIT { private static final int BUILTIN_AGGREGATE_FUNCTIONS_COUNT = TableBuiltinAggregationFunction.values().length; private static final int BUILTIN_TABLE_FUNCTIONS_COUNT = - TableBuiltinTableFunction.values().length; + TableBuiltinTableFunction.values().length + DataNodeTableBuiltinTableFunction.values().length; private static final int BUILTIN_FUNCTIONS_COUNT = BUILTIN_SCALAR_FUNCTIONS_COUNT + BUILTIN_AGGREGATE_FUNCTIONS_COUNT diff --git a/integration-test/src/test/java/org/apache/iotdb/relational/it/query/recent/IoTDBReadTsFileTableFunctionIT.java b/integration-test/src/test/java/org/apache/iotdb/relational/it/query/recent/IoTDBReadTsFileTableFunctionIT.java new file mode 100644 index 0000000000000..0b5abc38a6291 --- /dev/null +++ b/integration-test/src/test/java/org/apache/iotdb/relational/it/query/recent/IoTDBReadTsFileTableFunctionIT.java @@ -0,0 +1,622 @@ +/* + * 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.relational.it.query.recent; + +import org.apache.iotdb.it.env.EnvFactory; +import org.apache.iotdb.it.env.cluster.node.DataNodeWrapper; +import org.apache.iotdb.it.framework.IoTDBTestRunner; +import org.apache.iotdb.itbase.category.TableClusterIT; +import org.apache.iotdb.itbase.category.TableLocalStandaloneIT; +import org.apache.iotdb.itbase.env.BaseEnv; + +import org.apache.tsfile.enums.ColumnCategory; +import org.apache.tsfile.enums.TSDataType; +import org.apache.tsfile.exception.write.WriteProcessException; +import org.apache.tsfile.file.metadata.TableSchema; +import org.apache.tsfile.write.TsFileWriter; +import org.apache.tsfile.write.record.Tablet; +import org.junit.After; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Files; +import java.sql.Connection; +import java.sql.SQLException; +import java.sql.Statement; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +import static org.apache.iotdb.db.it.utils.TestUtils.tableAssertTestFail; +import static org.apache.iotdb.db.it.utils.TestUtils.tableResultSetEqualTest; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +@RunWith(IoTDBTestRunner.class) +@Category({TableLocalStandaloneIT.class, TableClusterIT.class}) +public class IoTDBReadTsFileTableFunctionIT { + private static final String DATABASE_NAME = "test_read_tsfile"; + + private static File tmpDir; + + @BeforeClass + public static void setUp() throws Exception { + EnvFactory.getEnv().initClusterEnvironment(); + try (Connection connection = EnvFactory.getEnv().getTableConnection(); + Statement statement = connection.createStatement()) { + statement.execute("CREATE DATABASE " + DATABASE_NAME); + } + } + + @Before + public void setUpBeforeTest() throws IOException { + tmpDir = new File(Files.createTempDirectory("read-tsfile").toUri()); + } + + @After + public void tearDownAfterTest() { + deleteTmpDir(); + } + + @AfterClass + public static void tearDown() { + EnvFactory.getEnv().cleanClusterEnvironment(); + } + + @Test + public void testReadSingleTsFile() throws Exception { + File tsFile = new File(tmpDir, "single.tsfile"); + try (TsFileWriter writer = new TsFileWriter(tsFile)) { + generateTable( + writer, "table1", Arrays.asList("tag1", "tag2"), Arrays.asList("s1", "s2"), 1, 2); + generateTable(writer, "table2", Arrays.asList("tag1"), Arrays.asList("s1"), 1, 2); + } + + String[] expectedHeader = new String[] {"time", "tag1", "tag2", "s1", "s2"}; + String[] retArray = + new String[] { + "1970-01-01T00:00:00.001Z,tag1_1,tag2_1,1,1,", + "1970-01-01T00:00:00.002Z,tag1_1,tag2_1,2,2,", + "1970-01-01T00:00:00.001Z,tag1_2,tag2_2,1,1,", + "1970-01-01T00:00:00.002Z,tag1_2,tag2_2,2,2,", + }; + tableResultSetEqualTest( + "SELECT time, tag1, tag2, s1, s2 FROM read_tsfile(PATHS => '" + + toSqlPath(tsFile) + + "', TABLE_NAME => 'table1') ORDER BY tag1, tag2, time", + expectedHeader, + retArray, + DATABASE_NAME); + } + + @Test + public void testReadTsFileWithNoMatchedDevice() throws Exception { + File tsFile = new File(tmpDir, "empty-device.tsfile"); + try (TsFileWriter writer = new TsFileWriter(tsFile)) { + generateTable( + writer, "table1", Arrays.asList("tag1", "tag2"), Arrays.asList("s1", "s2"), 1, 2); + } + + tableResultSetEqualTest( + "SELECT time, tag1, tag2, s1, s2 FROM read_tsfile(PATHS => '" + + toSqlPath(tsFile) + + "', TABLE_NAME => 'table1') WHERE tag1 = 'not_exists'", + new String[] {"time", "tag1", "tag2", "s1", "s2"}, + new String[] {}, + DATABASE_NAME); + } + + @Test + public void testReadTsFileAggregationWithNoMatchedDevice() throws Exception { + File tsFile = new File(tmpDir, "empty-device-aggregation.tsfile"); + try (TsFileWriter writer = new TsFileWriter(tsFile)) { + generateTable( + writer, "table1", Arrays.asList("tag1", "tag2"), Arrays.asList("s1", "s2"), 1, 2); + } + + tableResultSetEqualTest( + "SELECT count(*) AS count_star, count(s1) AS count_s1, sum(s1) AS sum_s1" + + " FROM read_tsfile(PATHS => '" + + toSqlPath(tsFile) + + "', TABLE_NAME => 'table1') WHERE tag1 = 'not_exists'", + new String[] {"count_star", "count_s1", "sum_s1"}, + new String[] {"0,0,null,"}, + DATABASE_NAME); + } + + @Test + public void testReadTsFileGroupedAggregationAcrossDevices() throws Exception { + File tsFile1 = new File(tmpDir, "grouped-aggregation-1.tsfile"); + try (TsFileWriter writer = new TsFileWriter(tsFile1)) { + generateTable( + writer, "table1", Arrays.asList("tag1", "tag2"), Arrays.asList("s1", "s2"), 1, 2); + } + File tsFile2 = new File(tmpDir, "grouped-aggregation-2.tsfile"); + try (TsFileWriter writer = new TsFileWriter(tsFile2)) { + generateTable( + writer, "table1", Arrays.asList("tag1", "tag2"), Arrays.asList("s1", "s2"), 3, 4); + } + + tableResultSetEqualTest( + "SELECT tag1, count(s1) AS count_s1, sum(s1) AS sum_s1" + + " FROM read_tsfile(PATHS => '" + + toSqlPath(tsFile1) + + "," + + toSqlPath(tsFile2) + + "', TABLE_NAME => 'table1') GROUP BY tag1 ORDER BY tag1", + new String[] {"tag1", "count_s1", "sum_s1"}, + new String[] {"tag1_1,4,10.0,", "tag1_2,4,10.0,"}, + DATABASE_NAME); + } + + @Test + public void testReadMultipleTsFilesWithDeviceFilter() throws Exception { + File tsFile1 = new File(tmpDir, "multi-1.tsfile"); + try (TsFileWriter writer = new TsFileWriter(tsFile1)) { + generateTable( + writer, "table1", Arrays.asList("tag1", "tag2"), Arrays.asList("s1", "s2"), 1, 2); + } + File tsFile2 = new File(tmpDir, "multi-2.tsfile"); + try (TsFileWriter writer = new TsFileWriter(tsFile2)) { + generateTable( + writer, "table1", Arrays.asList("tag1", "tag2"), Arrays.asList("s1", "s2"), 3, 4); + } + + String[] expectedHeader = new String[] {"time", "tag1", "tag2", "s1", "s2"}; + String[] retArray = + new String[] { + "1970-01-01T00:00:00.001Z,tag1_1,tag2_1,1,1,", + "1970-01-01T00:00:00.002Z,tag1_1,tag2_1,2,2,", + "1970-01-01T00:00:00.003Z,tag1_1,tag2_1,3,3,", + "1970-01-01T00:00:00.004Z,tag1_1,tag2_1,4,4,", + }; + tableResultSetEqualTest( + "SELECT time, tag1, tag2, s1, s2 FROM read_tsfile(PATHS => '" + + toSqlPath(tsFile1) + + "," + + toSqlPath(tsFile2) + + "', TABLE_NAME => 'table1') WHERE tag1 = 'tag1_1' ORDER BY time", + expectedHeader, + retArray, + DATABASE_NAME); + } + + @Test + public void testReadMultipleTsFilesWithSchemaMerge() throws Exception { + File tsFile1 = new File(tmpDir, "schema-merge-1.tsfile"); + try (TsFileWriter writer = new TsFileWriter(tsFile1)) { + generateTable(writer, "table1", Arrays.asList("tag1"), Arrays.asList("s1"), 1, 2); + } + File tsFile2 = new File(tmpDir, "schema-merge-2.tsfile"); + try (TsFileWriter writer = new TsFileWriter(tsFile2)) { + generateTable(writer, "table1", Arrays.asList("tag1"), Arrays.asList("s1", "s2"), 3, 4); + } + + String[] expectedHeader = new String[] {"time", "tag1", "s1", "s2"}; + String[] retArray = + new String[] { + "1970-01-01T00:00:00.001Z,tag1_1,1,null,", + "1970-01-01T00:00:00.002Z,tag1_1,2,null,", + "1970-01-01T00:00:00.003Z,tag1_1,3,3,", + "1970-01-01T00:00:00.004Z,tag1_1,4,4,", + "1970-01-01T00:00:00.001Z,tag1_2,1,null,", + "1970-01-01T00:00:00.002Z,tag1_2,2,null,", + "1970-01-01T00:00:00.003Z,tag1_2,3,3,", + "1970-01-01T00:00:00.004Z,tag1_2,4,4,", + }; + tableResultSetEqualTest( + "SELECT time, tag1, s1, s2 FROM read_tsfile(PATHS => '" + + toSqlPath(tsFile1) + + "," + + toSqlPath(tsFile2) + + "', TABLE_NAME => 'table1') ORDER BY tag1, time", + expectedHeader, + retArray, + DATABASE_NAME); + } + + @Test + public void testTimeJoinReadTsFileWithSameTableNameFromDifferentTsFiles() throws Exception { + File leftTsFile = new File(tmpDir, "time-join-left.tsfile"); + try (TsFileWriter writer = new TsFileWriter(leftTsFile)) { + generateTable(writer, "table1", Arrays.asList("left_tag"), Arrays.asList("left_value"), 1, 3); + } + File rightTsFile = new File(tmpDir, "time-join-right.tsfile"); + try (TsFileWriter writer = new TsFileWriter(rightTsFile)) { + generateTable( + writer, "table1", Arrays.asList("right_tag"), Arrays.asList("right_value"), 2, 4); + } + + String[] expectedHeader = + new String[] {"time", "left_tag", "left_value", "right_tag", "right_value"}; + String[] retArray = + new String[] { + "1970-01-01T00:00:00.002Z,left_tag_1,2,right_tag_1,2,", + "1970-01-01T00:00:00.002Z,left_tag_1,2,right_tag_2,2,", + "1970-01-01T00:00:00.002Z,left_tag_2,2,right_tag_1,2,", + "1970-01-01T00:00:00.002Z,left_tag_2,2,right_tag_2,2,", + "1970-01-01T00:00:00.003Z,left_tag_1,3,right_tag_1,3,", + "1970-01-01T00:00:00.003Z,left_tag_1,3,right_tag_2,3,", + "1970-01-01T00:00:00.003Z,left_tag_2,3,right_tag_1,3,", + "1970-01-01T00:00:00.003Z,left_tag_2,3,right_tag_2,3,", + }; + tableResultSetEqualTest( + "SELECT l.time, l.left_tag, l.left_value, r.right_tag, r.right_value" + + " FROM read_tsfile(PATHS => '" + + toSqlPath(leftTsFile) + + "', TABLE_NAME => 'table1') l" + + " JOIN read_tsfile(PATHS => '" + + toSqlPath(rightTsFile) + + "', TABLE_NAME => 'table1') r ON l.time = r.time" + + " ORDER BY l.time, l.left_tag, r.right_tag", + expectedHeader, + retArray, + DATABASE_NAME); + } + + @Test + public void testReadMultipleTsFilesWithTagSchemaMerge() throws Exception { + File tsFile1 = new File(tmpDir, "tag-schema-merge-1.tsfile"); + try (TsFileWriter writer = new TsFileWriter(tsFile1)) { + generateTable(writer, "table1", Arrays.asList("tag1"), Arrays.asList("s1"), 1, 2); + } + File tsFile2 = new File(tmpDir, "tag-schema-merge-2.tsfile"); + try (TsFileWriter writer = new TsFileWriter(tsFile2)) { + generateTable(writer, "table1", Arrays.asList("tag1", "tag2"), Arrays.asList("s1"), 3, 4); + } + + String[] expectedHeader = new String[] {"time", "tag1", "tag2", "s1"}; + String[] retArray = + new String[] { + "1970-01-01T00:00:00.001Z,tag1_1,null,1,", + "1970-01-01T00:00:00.002Z,tag1_1,null,2,", + "1970-01-01T00:00:00.003Z,tag1_1,tag2_1,3,", + "1970-01-01T00:00:00.004Z,tag1_1,tag2_1,4,", + "1970-01-01T00:00:00.001Z,tag1_2,null,1,", + "1970-01-01T00:00:00.002Z,tag1_2,null,2,", + "1970-01-01T00:00:00.003Z,tag1_2,tag2_2,3,", + "1970-01-01T00:00:00.004Z,tag1_2,tag2_2,4,", + }; + tableResultSetEqualTest( + "SELECT time, tag1, tag2, s1 FROM read_tsfile(PATHS => '" + + toSqlPath(tsFile1) + + "," + + toSqlPath(tsFile2) + + "', TABLE_NAME => 'table1') ORDER BY tag1, time", + expectedHeader, + retArray, + DATABASE_NAME); + } + + @Test + public void testReadMultipleTsFilesWithConflictingFieldType() throws Exception { + File tsFile1 = new File(tmpDir, "conflict-1.tsfile"); + try (TsFileWriter writer = new TsFileWriter(tsFile1)) { + generateTable(writer, "table1", Arrays.asList("tag1"), Arrays.asList("s1"), 1, 2); + } + File tsFile2 = new File(tmpDir, "conflict-2.tsfile"); + try (TsFileWriter writer = new TsFileWriter(tsFile2)) { + generateTable( + writer, "table1", Arrays.asList("tag1"), Arrays.asList("s1"), TSDataType.DOUBLE, 3, 4); + } + + tableAssertTestFail( + "SELECT * FROM read_tsfile(PATHS => '" + + toSqlPath(tsFile1) + + "," + + toSqlPath(tsFile2) + + "', TABLE_NAME => 'table1')", + "has conflicting data types when merging table schema", + DATABASE_NAME); + } + + @Test + public void testReadMultipleTsFilesWithConflictingTagColumns() throws Exception { + File tsFile1 = new File(tmpDir, "tag-conflict-1.tsfile"); + try (TsFileWriter writer = new TsFileWriter(tsFile1)) { + generateTable(writer, "table1", Arrays.asList("tag1"), Arrays.asList("s1"), 1, 2); + } + File tsFile2 = new File(tmpDir, "tag-conflict-2.tsfile"); + try (TsFileWriter writer = new TsFileWriter(tsFile2)) { + generateTable(writer, "table1", Arrays.asList("tag2"), Arrays.asList("s1"), 3, 4); + } + + tableAssertTestFail( + "SELECT * FROM read_tsfile(PATHS => '" + + toSqlPath(tsFile1) + + "," + + toSqlPath(tsFile2) + + "', TABLE_NAME => 'table1')", + "Tag columns conflict when merging table schema", + DATABASE_NAME); + } + + @Test + public void testReadMultipleTsFilesWithConflictingTagAndFieldColumns() throws Exception { + File tsFile1 = new File(tmpDir, "tag-field-conflict-1.tsfile"); + try (TsFileWriter writer = new TsFileWriter(tsFile1)) { + registerTableSchema(writer, "table1", Arrays.asList("shared"), Arrays.asList("s1")); + } + File tsFile2 = new File(tmpDir, "tag-field-conflict-2.tsfile"); + try (TsFileWriter writer = new TsFileWriter(tsFile2)) { + registerTableSchema(writer, "table1", new ArrayList<>(), Arrays.asList("shared")); + } + + tableAssertTestFail( + "SELECT * FROM read_tsfile(PATHS => '" + + toSqlPath(tsFile1) + + "," + + toSqlPath(tsFile2) + + "', TABLE_NAME => 'table1')", + "conflicting categories when merging table schema", + DATABASE_NAME); + } + + @Test + public void testReadMultipleTsFilesWithConflictingFieldAndTagColumns() throws Exception { + File tsFile1 = new File(tmpDir, "field-tag-conflict-1.tsfile"); + try (TsFileWriter writer = new TsFileWriter(tsFile1)) { + registerTableSchema(writer, "table1", new ArrayList<>(), Arrays.asList("shared")); + } + File tsFile2 = new File(tmpDir, "field-tag-conflict-2.tsfile"); + try (TsFileWriter writer = new TsFileWriter(tsFile2)) { + registerTableSchema(writer, "table1", Arrays.asList("shared"), Arrays.asList("s1")); + } + + tableAssertTestFail( + "SELECT * FROM read_tsfile(PATHS => '" + + toSqlPath(tsFile1) + + "," + + toSqlPath(tsFile2) + + "', TABLE_NAME => 'table1')", + "conflicting categories when merging table schema", + DATABASE_NAME); + } + + @Test + public void testReadTsFileWithoutTableNameWhenMultipleTablesExist() throws Exception { + File tsFile = new File(tmpDir, "multiple-tables.tsfile"); + try (TsFileWriter writer = new TsFileWriter(tsFile)) { + generateTable(writer, "table1", Arrays.asList("tag1"), Arrays.asList("s1"), 1, 2); + generateTable(writer, "table2", Arrays.asList("tag1"), Arrays.asList("s1"), 1, 2); + } + + tableAssertTestFail( + "SELECT * FROM read_tsfile(PATHS => '" + toSqlPath(tsFile) + "')", + "Cannot infer table name from TsFile because multiple tables are found", + DATABASE_NAME); + } + + @Test + public void testReadSpecifiedInvalidFileFails() throws IOException { + File invalidFile = new File(tmpDir, "invalid-file.txt"); + Files.write(invalidFile.toPath(), new byte[] {1, 2, 3}); + + tableAssertTestFail( + "SELECT * FROM read_tsfile(PATHS => '" + toSqlPath(invalidFile) + "')", + "not a valid TsFile", + DATABASE_NAME); + } + + @Test + public void testReadDirectoryOnlyScansValidTsFileSuffixFiles() throws Exception { + File scanDir = new File(tmpDir, "scan-dir"); + File nestedDir = new File(scanDir, "nested"); + Files.createDirectories(nestedDir.toPath()); + + File validTsFile = new File(nestedDir, "valid.tsfile"); + try (TsFileWriter writer = new TsFileWriter(validTsFile)) { + generateTable(writer, "table1", Arrays.asList("tag1"), Arrays.asList("s1"), 1, 2); + } + + File invalidTsFile = new File(scanDir, "invalid.tsfile"); + Files.write(invalidTsFile.toPath(), new byte[] {1, 2, 3}); + + File validFileWithoutTsFileSuffix = new File(scanDir, "valid.data"); + try (TsFileWriter writer = new TsFileWriter(validFileWithoutTsFileSuffix)) { + generateTable(writer, "table1", Arrays.asList("tag1"), Arrays.asList("s1"), 3, 4); + } + + String[] expectedHeader = new String[] {"time", "tag1", "s1"}; + String[] retArray = + new String[] { + "1970-01-01T00:00:00.001Z,tag1_1,1,", + "1970-01-01T00:00:00.002Z,tag1_1,2,", + "1970-01-01T00:00:00.001Z,tag1_2,1,", + "1970-01-01T00:00:00.002Z,tag1_2,2,", + }; + tableResultSetEqualTest( + "SELECT time, tag1, s1 FROM read_tsfile(PATHS => '" + + toSqlPath(scanDir) + + "', TABLE_NAME => 'table1') ORDER BY tag1, time", + expectedHeader, + retArray, + DATABASE_NAME); + } + + @Test + public void testReadTsFileWithInvalidPaths() throws IOException { + File missingFile = new File(tmpDir, "missing.tsfile"); + tableAssertTestFail( + "SELECT * FROM read_tsfile(PATHS => '" + toSqlPath(missingFile) + "')", + "TsFile path does not exist", + DATABASE_NAME); + + DataNodeWrapper dataNodeWrapper = EnvFactory.getEnv().getDataNodeWrapper(0); + File dataDir = + new File(dataNodeWrapper.getDataNodeDir() + File.separator + "data", "forbidden.tsfile"); + Files.createDirectories(dataDir.getParentFile().toPath()); + Files.write(dataDir.toPath(), new byte[0]); + try (Connection connection = + EnvFactory.getEnv() + .getWriteOnlyConnectionWithSpecifiedDataNode( + dataNodeWrapper, BaseEnv.TABLE_SQL_DIALECT); + Statement statement = connection.createStatement()) { + statement.execute("USE " + DATABASE_NAME); + statement.execute("SELECT * FROM read_tsfile(PATHS => '" + toSqlPath(dataDir) + "')"); + fail("Expected read_tsfile to reject paths under the current DataNode data directory"); + } catch (SQLException e) { + assertTrue( + e.getMessage(), + e.getMessage().contains("is not allowed because it may access IoTDB data directory")); + } + } + + private static void generateTable( + TsFileWriter writer, + String tableName, + List tagColumns, + List fieldColumns, + long startTime, + long endTime) + throws IOException, WriteProcessException { + generateTable( + writer, tableName, tagColumns, fieldColumns, TSDataType.INT64, startTime, endTime); + } + + private static void generateTable( + TsFileWriter writer, + String tableName, + List tagColumns, + List fieldColumns, + TSDataType fieldType, + long startTime, + long endTime) + throws IOException, WriteProcessException { + List columnNames = new ArrayList<>(tagColumns.size() + fieldColumns.size()); + List columnTypes = new ArrayList<>(tagColumns.size() + fieldColumns.size()); + List columnCategories = + new ArrayList<>(tagColumns.size() + fieldColumns.size()); + for (String tagColumn : tagColumns) { + columnNames.add(tagColumn); + columnTypes.add(TSDataType.STRING); + columnCategories.add(ColumnCategory.TAG); + } + for (String fieldColumn : fieldColumns) { + columnNames.add(fieldColumn); + columnTypes.add(fieldType); + columnCategories.add(ColumnCategory.FIELD); + } + + writer.registerTableSchema( + new TableSchema(tableName, columnNames, columnTypes, columnCategories)); + Tablet tablet = new Tablet(tableName, columnNames, columnTypes, columnCategories); + for (int deviceIndex = 1; deviceIndex <= 2; deviceIndex++) { + for (long time = startTime; time <= endTime; time++) { + int row = tablet.getRowSize(); + tablet.addTimestamp(row, time); + for (int i = 0; i < tagColumns.size(); i++) { + tablet.addValue(row, i, tagColumns.get(i) + "_" + deviceIndex); + } + for (int i = 0; i < fieldColumns.size(); i++) { + addFieldValue(tablet, row, tagColumns.size() + i, fieldType, time); + } + if (tablet.getRowSize() == tablet.getMaxRowNumber()) { + writer.writeTable(tablet); + tablet.reset(); + } + } + } + if (tablet.getRowSize() != 0) { + writer.writeTable(tablet); + } + } + + private static void registerTableSchema( + TsFileWriter writer, String tableName, List tagColumns, List fieldColumns) + throws IOException { + List columnNames = new ArrayList<>(tagColumns.size() + fieldColumns.size()); + List columnTypes = new ArrayList<>(tagColumns.size() + fieldColumns.size()); + List columnCategories = + new ArrayList<>(tagColumns.size() + fieldColumns.size()); + for (String tagColumn : tagColumns) { + columnNames.add(tagColumn); + columnTypes.add(TSDataType.STRING); + columnCategories.add(ColumnCategory.TAG); + } + for (String fieldColumn : fieldColumns) { + columnNames.add(fieldColumn); + columnTypes.add(TSDataType.INT64); + columnCategories.add(ColumnCategory.FIELD); + } + + writer.registerTableSchema( + new TableSchema(tableName, columnNames, columnTypes, columnCategories)); + } + + private static void addFieldValue( + Tablet tablet, int row, int column, TSDataType fieldType, long time) { + if (fieldType == TSDataType.DOUBLE) { + tablet.addValue(row, column, (double) time); + return; + } + tablet.addValue(row, column, time); + } + + private static String toSqlPath(File file) { + return file.getAbsolutePath().replace("\\", "\\\\").replace("'", "''"); + } + + private static void clearTmpDir() { + if (tmpDir == null || !tmpDir.exists()) { + return; + } + File[] files = tmpDir.listFiles(); + if (files != null) { + for (File file : files) { + deleteRecursively(file); + } + } + } + + private static void deleteRecursively(File file) { + if (file.isDirectory()) { + File[] children = file.listFiles(); + if (children != null) { + for (File child : children) { + deleteRecursively(child); + } + } + } + try { + Files.delete(file.toPath()); + } catch (IOException ignored) { + // ignore + } + } + + private static void deleteTmpDir() { + clearTmpDir(); + if (tmpDir == null || !tmpDir.exists()) { + return; + } + try { + Files.delete(tmpDir.toPath()); + } catch (IOException ignored) { + // ignore + } + } +} diff --git a/iotdb-core/datanode/src/main/i18n/en/org/apache/iotdb/db/i18n/DataNodeQueryMessages.java b/iotdb-core/datanode/src/main/i18n/en/org/apache/iotdb/db/i18n/DataNodeQueryMessages.java index b63739fef9b06..462b68f9358e6 100644 --- a/iotdb-core/datanode/src/main/i18n/en/org/apache/iotdb/db/i18n/DataNodeQueryMessages.java +++ b/iotdb-core/datanode/src/main/i18n/en/org/apache/iotdb/db/i18n/DataNodeQueryMessages.java @@ -197,6 +197,21 @@ public final class DataNodeQueryMessages { "Failed to close reader in TableDiskUsageSupplier"; public static final String UNSUPPORTED_CATEGORY = "Unsupported category: "; + public static final String CURRENT_DEVICE_ENTRY_IN_TABLESCANOPERATOR_IS_EMPTY = + "Current device entry in TableScanOperator is empty"; + public static final String UNEXPECTED_END_OF_EXTERNAL_TSFILE_DEVICE_TASK_READER_AT_DEVICE_INDEX = + "Unexpected end of external TsFile device task reader at device index "; + public static final String + EXTERNAL_TSFILE_DEVICE_TASK_READER_IS_NOT_ALIGNED_WITH_DEVICE_ENTRIES = + "External TsFile device task reader is not aligned with device entries at index %d:" + + " expected %s but got %s"; + public static final String FAILED_TO_UPDATE_EXTERNAL_TSFILE_DEVICE_RESOURCES = + "Failed to update external TsFile device resources"; + public static final String SCHEMA_FILTER_TYPE_IS_NOT_SUPPORTED = + "The schema filter type %s is not supported"; + public static final String + ATTRIBUTE_FILTER_IS_NOT_SUPPORTED_FOR_EXTERNAL_TSFILE_DEVICE_FILTERING = + "Attribute filter is not supported for external TsFile device filtering"; // --- Execution / Operator / Window --- @@ -761,6 +776,81 @@ public final class DataNodeQueryMessages { "Tag column only support data type STRING."; public static final String ATTRIBUTE_COLUMN_ONLY_SUPPORT_DATA_TYPE_STRING = "Attribute column only support data type STRING."; + public static final String UNSUPPORTED_EXTERNAL_TSFILE_DEVICE_FILTER = + "Unsupported external TsFile device filter: "; + + // --- Plan / Relational / Table Function --- + + public static final String NO_TABLE_SCHEMA_FOUND_IN_TSFILES = + "No table schema found in TsFiles"; + public static final String NO_TABLE_SCHEMA_FOUND_FOR_TABLE_IN_TSFILES = + "No table schema found for table %s in TsFiles"; + public static final String READ_TSFILE_MUST_BE_PLANNED_AS_EXTERNAL_TSFILE_SCAN_NODE = + "readTsFile must be planned as an ExternalTsFileScanNode"; + public static final String MISSING_SCALAR_ARGUMENT = + "Missing scalar argument: "; + public static final String ARGUMENT_SHOULD_NOT_BE_EMPTY = + "Argument %s should not be empty"; + public static final String INVALID_SCALAR_ARGUMENT = + "Invalid scalar argument: "; + public static final String ARGUMENT_SHOULD_BE_A_STRING = + "Argument %s should be a string"; + public static final String ARGUMENT_SHOULD_CONTAIN_AT_LEAST_ONE_PATH = + "Argument %s should contain at least one path"; + public static final String READ_TSFILE_PATH_IS_NOT_ALLOWED = + "readTsFile path %s is not allowed because it may access IoTDB data directory %s"; + public static final String OUTPUT_COLUMN_NAMES_AND_TYPES_SIZE_MISMATCH = + "Output column names and types size mismatch"; + public static final String OUTPUT_COLUMN_NAMES_AND_CATEGORIES_SIZE_MISMATCH = + "Output column names and categories size mismatch"; + public static final String READ_TSFILE_TABLE_FUNCTION_HANDLE_DOES_NOT_SUPPORT_SERIALIZATION = + "ReadTsFileTableFunctionHandle does not support serialization"; + public static final String READ_TSFILE_TABLE_FUNCTION_HANDLE_DOES_NOT_SUPPORT_DESERIALIZATION = + "ReadTsFileTableFunctionHandle does not support deserialization"; + public static final String TSFILE_PATH_DOES_NOT_EXIST = + "TsFile path does not exist: "; + public static final String TSFILE_PATH_IS_NEITHER_A_FILE_NOR_A_DIRECTORY = + "TsFile path is neither a file nor a directory: "; + public static final String NO_VALID_TSFILES_FOUND = + "No valid TsFiles found"; + public static final String FAILED_TO_SCAN_TSFILE_PATH = + "Failed to scan TsFile path: "; + public static final String CANNOT_INFER_TABLE_NAME_FROM_TSFILES_MULTIPLE_TABLES = + "Cannot infer table name from TsFiles because multiple tables are found: %s and %s"; + public static final String CANNOT_INFER_TABLE_NAME_FROM_TSFILE_NO_TABLE_SCHEMA = + "Cannot infer table name from TsFile because no table schema is found in "; + public static final String CANNOT_INFER_TABLE_NAME_FROM_TSFILE_MULTIPLE_TABLES = + "Cannot infer table name from TsFile because multiple tables are found in "; + public static final String FILE_IS_NOT_A_VALID_TSFILE = + "File is not a valid TsFile: "; + public static final String FAILED_TO_READ_TABLE_SCHEMA_FROM_TSFILE = + "Failed to read table schema from TsFile: "; + public static final String MULTIPLE_TIME_COLUMNS_FOUND_WHEN_MERGING_TABLE_SCHEMA = + "Multiple time columns found when merging table schema for table "; + public static final String TIME_COLUMN_CONFLICTS_WHEN_MERGING_TABLE_SCHEMA = + "Time column conflicts when merging table schema for table "; + public static final String TAG_COLUMNS_CONFLICT_WHEN_MERGING_TABLE_SCHEMA = + "Tag columns conflict when merging table schema for table "; + public static final String FIELD_COLUMN_HAS_CONFLICTING_DATA_TYPES_WHEN_MERGING_TABLE_SCHEMA = + "Field column %s has conflicting data types when merging table schema for table %s"; + public static final String COLUMN_HAS_CONFLICTING_CATEGORIES_WHEN_MERGING_TABLE_SCHEMA = + "Column %s has conflicting categories when merging table schema for table %s"; + public static final String FAILED_TO_CREATE_EXTERNAL_TSFILE_DEVICE_TASK_RUN_READER = + "Failed to create external TsFile device task run reader"; + public static final String UNKNOWN_EXTERNAL_TSFILE_DEVICE_TASK_PARTITION = + "Unknown external TsFile device task partition: "; + public static final String EXTERNAL_TSFILE_QUERY_RESOURCE_HAS_BEEN_CLOSED = + "External TsFile query resource has been closed: "; + public static final String EXTERNAL_TSFILE_FRAGMENT_INSTANCE_USAGE_COUNT_CANNOT_BE_NEGATIVE = + "External TsFile fragment instance usage count cannot be negative"; + public static final String FAILED_TO_DESERIALIZE_EXTERNAL_TSFILE_RESOURCE = + "Failed to deserialize external TsFile resource: %s, %s"; + public static final String FAILED_TO_FLUSH_EXTERNAL_TSFILE_DEVICE_TASK_PARTITION = + "Failed to flush external TsFile device task partition"; + public static final String EXTERNAL_TSFILE_DEVICE_TASK_PARTITION_COUNT_MUST_BE_POSITIVE = + "External TsFile device task partition count must be positive"; + public static final String FAILED_TO_CREATE_EXTERNAL_TSFILE_DEVICE_COLLECTOR = + "Failed to create external TsFile device collector"; // --- Plan / Relational / Planner --- @@ -804,6 +894,8 @@ public final class DataNodeQueryMessages { "Values is not supported in current version."; public static final String SUBSCRIPT_IS_NOT_SUPPORTED_IN_CURRENT_VERSION = "Subscript is not supported in current version"; + public static final String READ_TSFILE_TABLE_FUNCTION_HANDLE_IS_INVALID = + "readTsFile table function handle is invalid"; // --- Plan / Relational / Planner / IR --- @@ -833,6 +925,16 @@ public final class DataNodeQueryMessages { "Not supported yet."; public static final String COPYTONODE_SHOULD_NOT_BE_SERIALIZED = "CopyToNode should not be serialized"; + public static final String + EXTERNAL_TSFILE_AGGREGATION_SCAN_NODE_DEVICE_ENTRIES_MUST_BE_SET_BY_DEVICE_ENTRY_INDEXES = + "ExternalTsFileAggregationScanNode device entries must be set by device entry indexes"; + public static final String EXTERNAL_TSFILE_AGGREGATION_SCAN_NODE_CANNOT_BE_SERIALIZED = + "ExternalTsFileAggregationScanNode cannot be serialized because it reads local external TsFiles"; + public static final String + EXTERNAL_TSFILE_SCAN_NODE_DEVICE_ENTRIES_MUST_BE_SET_BY_DEVICE_ENTRY_INDEXES = + "ExternalTsFileScanNode device entries must be set by device entry indexes"; + public static final String EXTERNAL_TSFILE_SCAN_NODE_CANNOT_BE_SERIALIZED = + "ExternalTsFileScanNode cannot be serialized because it reads local external TsFiles"; // --- Plan / Relational / Planner / Optimizations --- diff --git a/iotdb-core/datanode/src/main/i18n/zh/org/apache/iotdb/db/i18n/DataNodeQueryMessages.java b/iotdb-core/datanode/src/main/i18n/zh/org/apache/iotdb/db/i18n/DataNodeQueryMessages.java index e70f00f2db909..dc7d5a59d9e5a 100644 --- a/iotdb-core/datanode/src/main/i18n/zh/org/apache/iotdb/db/i18n/DataNodeQueryMessages.java +++ b/iotdb-core/datanode/src/main/i18n/zh/org/apache/iotdb/db/i18n/DataNodeQueryMessages.java @@ -197,6 +197,20 @@ public final class DataNodeQueryMessages { "在 TableDiskUsageSupplier 中关闭 reader 失败"; public static final String UNSUPPORTED_CATEGORY = "不支持的列类别:"; + public static final String CURRENT_DEVICE_ENTRY_IN_TABLESCANOPERATOR_IS_EMPTY = + "TableScanOperator 中当前设备条目为空"; + public static final String UNEXPECTED_END_OF_EXTERNAL_TSFILE_DEVICE_TASK_READER_AT_DEVICE_INDEX = + "外部 TsFile 设备任务读取器在设备索引处意外结束:"; + public static final String + EXTERNAL_TSFILE_DEVICE_TASK_READER_IS_NOT_ALIGNED_WITH_DEVICE_ENTRIES = + "外部 TsFile 设备任务读取器与设备条目不匹配,索引 %d:期望 %s,实际 %s"; + public static final String FAILED_TO_UPDATE_EXTERNAL_TSFILE_DEVICE_RESOURCES = + "更新外部 TsFile 设备资源失败"; + public static final String SCHEMA_FILTER_TYPE_IS_NOT_SUPPORTED = + "不支持 SchemaFilter 类型 %s"; + public static final String + ATTRIBUTE_FILTER_IS_NOT_SUPPORTED_FOR_EXTERNAL_TSFILE_DEVICE_FILTERING = + "外部 TsFile 设备过滤暂不支持属性过滤"; // --- Execution / Operator / Window --- @@ -760,6 +774,81 @@ public final class DataNodeQueryMessages { "标签列仅支持 STRING 数据类型。"; public static final String ATTRIBUTE_COLUMN_ONLY_SUPPORT_DATA_TYPE_STRING = "属性列仅支持 STRING 数据类型。"; + public static final String UNSUPPORTED_EXTERNAL_TSFILE_DEVICE_FILTER = + "不支持的外部 TsFile 设备过滤器:"; + + // --- Plan / Relational / Table Function --- + + public static final String NO_TABLE_SCHEMA_FOUND_IN_TSFILES = + "TsFile 中未找到表结构"; + public static final String NO_TABLE_SCHEMA_FOUND_FOR_TABLE_IN_TSFILES = + "TsFile 中未找到表 %s 的表结构"; + public static final String READ_TSFILE_MUST_BE_PLANNED_AS_EXTERNAL_TSFILE_SCAN_NODE = + "readTsFile 必须规划为 ExternalTsFileScanNode"; + public static final String MISSING_SCALAR_ARGUMENT = + "缺少标量参数:"; + public static final String ARGUMENT_SHOULD_NOT_BE_EMPTY = + "参数 %s 不应为空"; + public static final String INVALID_SCALAR_ARGUMENT = + "无效的标量参数:"; + public static final String ARGUMENT_SHOULD_BE_A_STRING = + "参数 %s 应为字符串"; + public static final String ARGUMENT_SHOULD_CONTAIN_AT_LEAST_ONE_PATH = + "参数 %s 应至少包含一个路径"; + public static final String READ_TSFILE_PATH_IS_NOT_ALLOWED = + "不允许 readTsFile 路径 %s,因为它可能访问 IoTDB 数据目录 %s"; + public static final String OUTPUT_COLUMN_NAMES_AND_TYPES_SIZE_MISMATCH = + "输出列名和类型数量不匹配"; + public static final String OUTPUT_COLUMN_NAMES_AND_CATEGORIES_SIZE_MISMATCH = + "输出列名和类别数量不匹配"; + public static final String READ_TSFILE_TABLE_FUNCTION_HANDLE_DOES_NOT_SUPPORT_SERIALIZATION = + "ReadTsFileTableFunctionHandle 不支持序列化"; + public static final String READ_TSFILE_TABLE_FUNCTION_HANDLE_DOES_NOT_SUPPORT_DESERIALIZATION = + "ReadTsFileTableFunctionHandle 不支持反序列化"; + public static final String TSFILE_PATH_DOES_NOT_EXIST = + "TsFile 路径不存在:"; + public static final String TSFILE_PATH_IS_NEITHER_A_FILE_NOR_A_DIRECTORY = + "TsFile 路径既不是文件也不是目录:"; + public static final String NO_VALID_TSFILES_FOUND = + "未找到有效的 TsFile"; + public static final String FAILED_TO_SCAN_TSFILE_PATH = + "扫描 TsFile 路径失败:"; + public static final String CANNOT_INFER_TABLE_NAME_FROM_TSFILES_MULTIPLE_TABLES = + "无法从 TsFile 推断表名,因为发现了多个表:%s 和 %s"; + public static final String CANNOT_INFER_TABLE_NAME_FROM_TSFILE_NO_TABLE_SCHEMA = + "无法从 TsFile 推断表名,因为未找到表结构,文件:"; + public static final String CANNOT_INFER_TABLE_NAME_FROM_TSFILE_MULTIPLE_TABLES = + "无法从 TsFile 推断表名,因为发现了多个表,文件:"; + public static final String FILE_IS_NOT_A_VALID_TSFILE = + "文件不是有效的 TsFile:"; + public static final String FAILED_TO_READ_TABLE_SCHEMA_FROM_TSFILE = + "从 TsFile 读取表结构失败:"; + public static final String MULTIPLE_TIME_COLUMNS_FOUND_WHEN_MERGING_TABLE_SCHEMA = + "合并表结构时发现多个时间列,表:"; + public static final String TIME_COLUMN_CONFLICTS_WHEN_MERGING_TABLE_SCHEMA = + "合并表结构时时间列冲突,表:"; + public static final String TAG_COLUMNS_CONFLICT_WHEN_MERGING_TABLE_SCHEMA = + "合并表结构时标签列冲突,表:"; + public static final String FIELD_COLUMN_HAS_CONFLICTING_DATA_TYPES_WHEN_MERGING_TABLE_SCHEMA = + "字段列 %s 在合并表 %s 的结构时存在冲突的数据类型"; + public static final String COLUMN_HAS_CONFLICTING_CATEGORIES_WHEN_MERGING_TABLE_SCHEMA = + "列 %s 在合并表 %s 的结构时存在冲突的类别"; + public static final String FAILED_TO_CREATE_EXTERNAL_TSFILE_DEVICE_TASK_RUN_READER = + "创建外部 TsFile 设备任务运行读取器失败"; + public static final String UNKNOWN_EXTERNAL_TSFILE_DEVICE_TASK_PARTITION = + "未知的外部 TsFile 设备任务分区:"; + public static final String EXTERNAL_TSFILE_QUERY_RESOURCE_HAS_BEEN_CLOSED = + "外部 TsFile 查询资源已关闭:"; + public static final String EXTERNAL_TSFILE_FRAGMENT_INSTANCE_USAGE_COUNT_CANNOT_BE_NEGATIVE = + "外部 TsFile FragmentInstance 使用计数不能为负数"; + public static final String FAILED_TO_DESERIALIZE_EXTERNAL_TSFILE_RESOURCE = + "反序列化外部 TsFile 资源失败:%s,%s"; + public static final String FAILED_TO_FLUSH_EXTERNAL_TSFILE_DEVICE_TASK_PARTITION = + "刷写外部 TsFile 设备任务分区失败"; + public static final String EXTERNAL_TSFILE_DEVICE_TASK_PARTITION_COUNT_MUST_BE_POSITIVE = + "外部 TsFile 设备任务分区数量必须为正数"; + public static final String FAILED_TO_CREATE_EXTERNAL_TSFILE_DEVICE_COLLECTOR = + "创建外部 TsFile 设备收集器失败"; // --- Plan / Relational / Planner --- @@ -803,6 +892,8 @@ public final class DataNodeQueryMessages { "当前版本不支持 Values。"; public static final String SUBSCRIPT_IS_NOT_SUPPORTED_IN_CURRENT_VERSION = "当前版本不支持下标操作"; + public static final String READ_TSFILE_TABLE_FUNCTION_HANDLE_IS_INVALID = + "readTsFile 表函数句柄无效"; // --- Plan / Relational / Planner / IR --- @@ -832,6 +923,16 @@ public final class DataNodeQueryMessages { "暂不支持。"; public static final String COPYTONODE_SHOULD_NOT_BE_SERIALIZED = "CopyToNode 不应被序列化"; + public static final String + EXTERNAL_TSFILE_AGGREGATION_SCAN_NODE_DEVICE_ENTRIES_MUST_BE_SET_BY_DEVICE_ENTRY_INDEXES = + "ExternalTsFileAggregationScanNode 的设备条目必须通过设备条目索引设置"; + public static final String EXTERNAL_TSFILE_AGGREGATION_SCAN_NODE_CANNOT_BE_SERIALIZED = + "ExternalTsFileAggregationScanNode 读取本地外部 TsFile,因此不能被序列化"; + public static final String + EXTERNAL_TSFILE_SCAN_NODE_DEVICE_ENTRIES_MUST_BE_SET_BY_DEVICE_ENTRY_INDEXES = + "ExternalTsFileScanNode 的设备条目必须通过设备条目索引设置"; + public static final String EXTERNAL_TSFILE_SCAN_NODE_CANNOT_BE_SERIALIZED = + "ExternalTsFileScanNode 读取本地外部 TsFile,因此不能被序列化"; // --- Plan / Relational / Planner / Optimizations --- diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/common/MPPQueryContext.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/common/MPPQueryContext.java index 6ef5804fac898..852028e6a2bbb 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/common/MPPQueryContext.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/common/MPPQueryContext.java @@ -28,24 +28,33 @@ import org.apache.iotdb.commons.auth.entity.PrivilegeType; import org.apache.iotdb.commons.queryengine.common.SessionInfo; import org.apache.iotdb.commons.queryengine.plan.relational.analyzer.NodeRef; +import org.apache.iotdb.commons.queryengine.plan.relational.metadata.ColumnSchema; +import org.apache.iotdb.commons.queryengine.plan.relational.planner.Symbol; import org.apache.iotdb.commons.queryengine.plan.relational.sql.ast.Identifier; import org.apache.iotdb.commons.queryengine.plan.relational.sql.ast.Query; import org.apache.iotdb.commons.queryengine.plan.relational.sql.ast.Table; import org.apache.iotdb.commons.queryengine.utils.cte.CteDataStore; +import org.apache.iotdb.commons.utils.FileUtils; import org.apache.iotdb.commons.utils.TestOnly; +import org.apache.iotdb.db.conf.IoTDBDescriptor; +import org.apache.iotdb.db.exception.query.QueryTimeoutRuntimeException; import org.apache.iotdb.db.queryengine.plan.analyze.Analysis; import org.apache.iotdb.db.queryengine.plan.analyze.PredicateUtils; import org.apache.iotdb.db.queryengine.plan.analyze.QueryType; import org.apache.iotdb.db.queryengine.plan.analyze.TypeProvider; import org.apache.iotdb.db.queryengine.plan.analyze.lock.SchemaLockType; import org.apache.iotdb.db.queryengine.plan.planner.memory.NotThreadSafeMemoryReservationManager; +import org.apache.iotdb.db.queryengine.plan.relational.function.tvf.read_tsfile.ExternalTsFileQueryResource; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.ExplainOutputFormat; import org.apache.iotdb.db.queryengine.statistics.QueryPlanStatistics; import com.google.common.collect.ImmutableList; import org.apache.tsfile.read.filter.basic.Filter; import org.apache.tsfile.utils.Pair; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import java.nio.file.Paths; import java.time.ZoneId; import java.util.Collections; import java.util.HashMap; @@ -56,6 +65,7 @@ import java.util.Optional; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicInteger; import java.util.function.LongConsumer; /** @@ -63,6 +73,8 @@ * info and so on. */ public class MPPQueryContext implements IAuditEntity { + private static final Logger LOGGER = LoggerFactory.getLogger(MPPQueryContext.class); + private String sql; private final QueryId queryId; @@ -160,6 +172,10 @@ public enum ExplainType { // Tables in the subquery private final Map, List> subQueryTables = new HashMap<>(); + private final Set externalTsFileQueryResources = + ConcurrentHashMap.newKeySet(); + private final AtomicInteger externalTsFileQueryResourceIndex = new AtomicInteger(); + @TestOnly public MPPQueryContext(QueryId queryId) { this.queryId = queryId; @@ -245,6 +261,54 @@ public QueryId getQueryId() { return queryId; } + public ExternalTsFileQueryResource createExternalTsFileQueryResource( + String tableName, List tsFilePaths, Map tableColumnSchema) { + int resourceIndex = externalTsFileQueryResourceIndex.getAndIncrement(); + ExternalTsFileQueryResource externalTsFileQueryResource = + new ExternalTsFileQueryResource( + this, + Paths.get(IoTDBDescriptor.getInstance().getConfig().getSortTmpDir()) + .resolve(ExternalTsFileQueryResource.EXTERNAL_TSFILE_TMP_DIR) + .resolve(queryId.getId()) + .resolve(String.valueOf(resourceIndex)), + tableName, + tsFilePaths, + tableColumnSchema); + externalTsFileQueryResources.add(externalTsFileQueryResource); + return externalTsFileQueryResource; + } + + public void releaseExternalTsFileQueryResources() { + if (externalTsFileQueryResources.isEmpty()) { + return; + } + for (ExternalTsFileQueryResource externalTsFileQueryResource : externalTsFileQueryResources) { + try { + // QueryExecution may finish before all FragmentInstances stop. Close only resources that + // have not been retained by runtime FragmentInstances. + externalTsFileQueryResource.closeByQueryExecution(); + } catch (Exception e) { + LOGGER.warn("Failed to release external TsFile query resource", e); + } + } + } + + public void removeExternalTsFileQueryResource(ExternalTsFileQueryResource resource) { + externalTsFileQueryResources.remove(resource); + deleteExternalTsFileQueryTmpRootIfEmpty(); + } + + private void deleteExternalTsFileQueryTmpRootIfEmpty() { + if (externalTsFileQueryResources.isEmpty()) { + FileUtils.deleteFileOrDirectory( + Paths.get(IoTDBDescriptor.getInstance().getConfig().getSortTmpDir()) + .resolve(ExternalTsFileQueryResource.EXTERNAL_TSFILE_TMP_DIR) + .resolve(queryId.getId()) + .toFile(), + true); + } + } + public long getLocalQueryId() { return localQueryId; } @@ -263,6 +327,13 @@ public void setTimeOut(long timeOut) { this.timeOut = timeOut; } + public void checkTimeOut() { + long currentTime = System.currentTimeMillis(); + if (currentTime - startTime >= timeOut) { + throw new QueryTimeoutRuntimeException(startTime, currentTime, timeOut); + } + } + public void setQueryType(QueryType queryType) { this.queryType = queryType; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceContext.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceContext.java index 6bda6e9c14d6e..3ab88ce261263 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceContext.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceContext.java @@ -46,6 +46,8 @@ import org.apache.iotdb.db.queryengine.metric.SeriesScanCostMetricSet; import org.apache.iotdb.db.queryengine.plan.planner.memory.ThreadSafeMemoryReservationManager; import org.apache.iotdb.db.queryengine.plan.planner.plan.TimePredicate; +import org.apache.iotdb.db.queryengine.plan.relational.function.tvf.read_tsfile.ExternalTsFileQueryDataSource; +import org.apache.iotdb.db.queryengine.plan.relational.function.tvf.read_tsfile.ExternalTsFileQueryResource; import org.apache.iotdb.db.storageengine.StorageEngine; import org.apache.iotdb.db.storageengine.dataregion.DataRegion; import org.apache.iotdb.db.storageengine.dataregion.IDataRegionForQuery; @@ -118,6 +120,9 @@ public class FragmentInstanceContext extends QueryContext { // Used for region scan, relating methods are to be added. private Map devicePathsToContext; + private ExternalTsFileQueryResource externalTsFileQueryResource; + private boolean externalTsFileQueryResourceRetained; + // Shared by all scan operators in this fragment instance to avoid memory problem protected IQueryDataSource sharedQueryDataSource; @@ -231,6 +236,11 @@ public void setQueryDataSourceType(QueryDataSourceType queryDataSourceType) { this.queryDataSourceType = queryDataSourceType; } + @Override + public boolean isExternalTsFileScan() { + return queryDataSourceType == QueryDataSourceType.EXTERNAL_TSFILE_SCAN; + } + @TestOnly public static FragmentInstanceContext createFragmentInstanceContext( FragmentInstanceId id, FragmentInstanceStateMachine stateMachine) { @@ -632,6 +642,48 @@ public void setDevicePathsToContext(Map devicePathsToC this.devicePathsToContext = devicePathsToContext; } + public void addExternalTsFileQueryResource( + ExternalTsFileQueryResource externalTsFileQueryResource) { + this.externalTsFileQueryResource = externalTsFileQueryResource; + } + + public boolean initExternalTsFileQueryDataSource( + ExternalTsFileQueryResource externalTsFileQueryResource) { + long startTime = System.nanoTime(); + try { + if (externalTsFileQueryResource == null) { + this.sharedQueryDataSource = EMPTY_QUERY_DATA_SOURCE; + } else { + if (!externalTsFileQueryResourceRetained) { + externalTsFileQueryResource.retainFragmentInstanceUsage(); + externalTsFileQueryResourceRetained = true; + } + this.sharedQueryDataSource = new ExternalTsFileQueryDataSource(externalTsFileQueryResource); + closedUnseqFileNum = externalTsFileQueryResource.getSharedTsFileResources().size(); + } + } finally { + addInitQueryDataSourceCost(System.nanoTime() - startTime); + } + return true; + } + + private void releaseExternalTsFileQueryResource() { + if (!externalTsFileQueryResourceRetained || externalTsFileQueryResource == null) { + externalTsFileQueryResource = null; + externalTsFileQueryResourceRetained = false; + return; + } + try { + // This FragmentInstance retained the resource during datasource initialization. Releasing the + // last runtime usage closes the shared resource and deletes its temporary run files. + externalTsFileQueryResource.closeByFragmentInstance(); + } catch (Exception e) { + LOGGER.warn("Failed to release external TsFile query resource", e); + } + externalTsFileQueryResource = null; + externalTsFileQueryResourceRetained = false; + } + public MemoryReservationManager getMemoryReservationContext() { return memoryReservationManager; } @@ -824,6 +876,11 @@ public synchronized IQueryDataSource getSharedQueryDataSource() throws QueryProc return getUnfinishedQueryDataSource(); } break; + case EXTERNAL_TSFILE_SCAN: + if (!initExternalTsFileQueryDataSource(externalTsFileQueryResource)) { + return getUnfinishedQueryDataSource(); + } + break; default: throw new QueryProcessException( "Unsupported query data source type: " + queryDataSourceType); @@ -1036,6 +1093,8 @@ public synchronized void releaseResource() { unClosedFilePaths = null; } + releaseExternalTsFileQueryResource(); + // release TVList/AlignedTVList owned by current query releaseTVListOwnedByQuery(); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/QueryContext.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/QueryContext.java index 44a888266967f..26d59dbe034bf 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/QueryContext.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/QueryContext.java @@ -97,6 +97,10 @@ public QueryContext(long queryId, boolean debug, long startTime, long timeout) { this.timeout = timeout; } + public boolean isExternalTsFileScan() { + return false; + } + // Only used for query with table data(Tree view is not included) public boolean collectTable(String table) { // In the current version (2025.08.14), there is only one table under one FI diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/AlignedSeriesScanUtil.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/AlignedSeriesScanUtil.java index f7ddaee472ea9..331a5a5b267d1 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/AlignedSeriesScanUtil.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/AlignedSeriesScanUtil.java @@ -100,7 +100,8 @@ protected AbstractAlignedTimeSeriesMetadata loadTimeSeriesMetadata( context, scanOptions.getGlobalTimeFilter(), isSeq, - ignoreAllNullRows); + ignoreAllNullRows, + null); } @Override diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/FileLoaderUtils.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/FileLoaderUtils.java index 254c061187a43..e5761050509ac 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/FileLoaderUtils.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/FileLoaderUtils.java @@ -198,7 +198,8 @@ public static AbstractAlignedTimeSeriesMetadata loadAlignedTimeSeriesMetadata( FragmentInstanceContext context, Filter globalTimeFilter, boolean isSeq, - boolean ignoreAllNullRows) + boolean ignoreAllNullRows, + long[] rootMeasurementMetadataIndexNodeOffset) throws IOException { final long t1 = System.nanoTime(); boolean loadFromMem = false; @@ -212,7 +213,12 @@ public static AbstractAlignedTimeSeriesMetadata loadAlignedTimeSeriesMetadata( if (resource.isClosed()) { alignedTimeSeriesMetadata = loadAlignedTimeSeriesMetadataFromDisk( - resource, alignedPath, context, globalTimeFilter, ignoreAllNullRows); + resource, + alignedPath, + context, + globalTimeFilter, + ignoreAllNullRows, + rootMeasurementMetadataIndexNodeOffset); } else { // if the tsfile is unclosed, we just get it directly from TsFileResource loadFromMem = true; alignedTimeSeriesMetadata = @@ -285,7 +291,8 @@ private static AbstractAlignedTimeSeriesMetadata loadAlignedTimeSeriesMetadataFr AlignedFullPath alignedPath, FragmentInstanceContext context, Filter globalTimeFilter, - boolean ignoreAllNullRows) + boolean ignoreAllNullRows, + long[] rootMeasurementMetadataIndexNodeOffset) throws IOException { AbstractAlignedTimeSeriesMetadata alignedTimeSeriesMetadata = null; // load all the TimeseriesMetadata of vector, the first one is for time column and the @@ -308,7 +315,8 @@ private static AbstractAlignedTimeSeriesMetadata loadAlignedTimeSeriesMetadataFr context.ignoreNotExistsDevice() || resource.getTimeIndexType() == ITimeIndex.FILE_TIME_INDEX_TYPE, isDebug, - context); + context, + rootMeasurementMetadataIndexNodeOffset); if (timeColumn != null) { // only need time column, like count_time aggregation if (valueMeasurementList.isEmpty()) { @@ -337,7 +345,8 @@ private static AbstractAlignedTimeSeriesMetadata loadAlignedTimeSeriesMetadataFr context.ignoreNotExistsDevice() || resource.getTimeIndexType() == ITimeIndex.FILE_TIME_INDEX_TYPE, isDebug, - context); + context, + rootMeasurementMetadataIndexNodeOffset); exist = (exist || (valueColumn != null)); valueTimeSeriesMetadataList.add(valueColumn); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/SeriesScanUtil.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/SeriesScanUtil.java index 6e6eb76905bc9..0591e1234c30c 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/SeriesScanUtil.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/SeriesScanUtil.java @@ -208,27 +208,7 @@ public void initQueryDataSource(QueryDataSource dataSource) { dataSource.fillOrderIndexes(deviceID, orderUtils.getAscending()); this.dataSource = dataSource; - // updated filter concerning TTL - // IgnoreAllNullRows is false indicating that the current query is a table model query. - // In most cases, We can use this condition to determine from which model to obtain the ttl - // of the current device. However, it should be noted that for tree model data queried using - // table view, ttl also needs to be obtained from the tree model. - if (context.isIgnoreAllNullRows() || scanOptions.isTableViewForTreeModel()) { - if (deviceID != EMPTY_DEVICE_ID) { - long ttl = DataNodeTTLCache.getInstance().getTTLForTree(deviceID); - scanOptions.setTTLForTreeDevice(ttl); - } - } else { - if (scanOptions.timeFilterNeedUpdatedByTtl()) { - String databaseName = dataSource.getDatabaseName(); - long ttl = - databaseName == null - ? Long.MAX_VALUE - : DataNodeTTLCache.getInstance() - .getTTLForTable(databaseName, deviceID.getTableName()); - scanOptions.setTTLForTableDevice(ttl); - } - } + updateFilterUsingTTL(dataSource); // init file index orderUtils.setCurSeqFileIndex(dataSource); @@ -254,6 +234,30 @@ public void initQueryDataSource(QueryDataSource dataSource) { } } + protected void updateFilterUsingTTL(QueryDataSource dataSource) { + // updated filter concerning TTL + // IgnoreAllNullRows is false indicating that the current query is a table model query. + // In most cases, We can use this condition to determine from which model to obtain the ttl + // of the current device. However, it should be noted that for tree model data queried using + // table view, ttl also needs to be obtained from the tree model. + if (context.isIgnoreAllNullRows() || scanOptions.isTableViewForTreeModel()) { + if (deviceID != EMPTY_DEVICE_ID) { + long ttl = DataNodeTTLCache.getInstance().getTTLForTree(deviceID); + scanOptions.setTTLForTreeDevice(ttl); + } + } else { + if (scanOptions.timeFilterNeedUpdatedByTtl()) { + String databaseName = dataSource.getDatabaseName(); + long ttl = + databaseName == null + ? Long.MAX_VALUE + : DataNodeTTLCache.getInstance() + .getTTLForTable(databaseName, deviceID.getTableName()); + scanOptions.setTTLForTableDevice(ttl); + } + } + } + protected PriorityMergeReader getPriorityMergeReader() { return new PriorityMergeReader(); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/AbstractAggTableScanOperator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/AbstractAggTableScanOperator.java index 1f37ce9f10472..031c10dc8a4db 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/AbstractAggTableScanOperator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/AbstractAggTableScanOperator.java @@ -96,7 +96,7 @@ public abstract class AbstractAggTableScanOperator extends AbstractDataSourceOpe protected SeriesScanOptions seriesScanOptions; private final boolean ascending; - private final Ordering scanOrder; + protected final Ordering scanOrder; // Some special data types(like BLOB) cannot use statistics protected final boolean canUseStatistics; private final long cachedRawDataSize; @@ -251,18 +251,9 @@ protected Optional calculateAggregationResultForCurrentTimeRange() thro // all data of current device has been consumed updateResultTsBlock(); timeIterator.resetCurTimeRange(); - nextDevice(); - - if (currentDeviceIndex < deviceCount) { - // construct AlignedSeriesScanUtil for next device - constructAlignedSeriesScanUtil(); - queryDataSource.reset(); - this.seriesScanUtil.initQueryDataSource(queryDataSource); - } + moveToNextDevice(); if (currentDeviceIndex >= deviceCount) { - // all devices have been consumed - timeIterator.setFinished(); return Optional.of(true); } else { return Optional.of(false); @@ -715,22 +706,25 @@ protected void checkIfAllAggregatorHasFinalResult() throws Exception { if (allAggregatorsHasFinalResult && (timeIterator.getType() == ITableTimeRangeIterator.TimeIteratorType.SINGLE_TIME_ITERATOR || tableAggregators.isEmpty())) { - nextDevice(); - inputTsBlock = null; - - if (currentDeviceIndex < deviceCount) { - // construct AlignedSeriesScanUtil for next device - constructAlignedSeriesScanUtil(); - queryDataSource.reset(); - this.seriesScanUtil.initQueryDataSource(queryDataSource); - } + moveToNextDevice(); + allAggregatorsHasFinalResult = false; + } + } - if (currentDeviceIndex >= deviceCount) { - // all devices have been consumed - timeIterator.setFinished(); - } + protected void moveToNextDevice() throws Exception { + nextDevice(); + inputTsBlock = null; - allAggregatorsHasFinalResult = false; + if (currentDeviceIndex < deviceCount) { + // construct AlignedSeriesScanUtil for next device + constructAlignedSeriesScanUtil(); + queryDataSource.reset(); + this.seriesScanUtil.initQueryDataSource(queryDataSource); + } + + if (currentDeviceIndex >= deviceCount) { + // all devices have been consumed + timeIterator.setFinished(); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/AbstractDefaultAggTableScanOperator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/AbstractDefaultAggTableScanOperator.java index f00b0049ef663..797d4491e417c 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/AbstractDefaultAggTableScanOperator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/AbstractDefaultAggTableScanOperator.java @@ -34,7 +34,7 @@ public abstract class AbstractDefaultAggTableScanOperator extends AbstractAggTableScanOperator { - private static final long INSTANCE_SIZE = + protected static final long INSTANCE_SIZE = RamUsageEstimator.shallowSizeOfInstance(AbstractDefaultAggTableScanOperator.class); protected AbstractDefaultAggTableScanOperator(AbstractAggTableScanOperatorParameter parameter) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/AbstractTableScanOperator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/AbstractTableScanOperator.java index 52a21628f2a3b..c759330e96261 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/AbstractTableScanOperator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/AbstractTableScanOperator.java @@ -54,27 +54,27 @@ import static org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanGraphPrinter.DEVICE_NUMBER; public abstract class AbstractTableScanOperator extends AbstractSeriesScanOperator { - private static final long INSTANCE_SIZE = + protected static final long INSTANCE_SIZE = RamUsageEstimator.shallowSizeOfInstance(TableScanOperator.class); private final List columnSchemas; private final int[] columnsIndexArray; - private final List deviceEntries; + protected final List deviceEntries; - private final int deviceCount; + protected final int deviceCount; - private final Ordering scanOrder; - private final SeriesScanOptions seriesScanOptions; + protected final Ordering scanOrder; + protected final SeriesScanOptions seriesScanOptions; - private final List measurementColumnNames; + protected final List measurementColumnNames; - private final Set allSensors; + protected final Set allSensors; - private final List measurementSchemas; + protected final List measurementSchemas; - private final List measurementColumnTSDataTypes; + protected final List measurementColumnTSDataTypes; private TsBlockBuilder measurementDataBuilder; @@ -84,7 +84,7 @@ public abstract class AbstractTableScanOperator extends AbstractSeriesScanOperat private QueryDataSource queryDataSource; - private int currentDeviceIndex; + protected int currentDeviceIndex; public AbstractTableScanOperator(AbstractTableScanOperatorParameter parameter) { this.sourceId = parameter.sourceId; @@ -114,7 +114,6 @@ public AbstractTableScanOperator(AbstractTableScanOperatorParameter parameter) { maxReturnSize, allSensors.size() * TSFileDescriptor.getInstance().getConfig().getPageSizeInByte()); this.maxTsBlockLineNum = parameter.maxTsBlockLineNum; - constructAlignedSeriesScanUtil(); } @@ -160,8 +159,7 @@ public TsBlock next() throws Exception { if (measurementDataBuilder.isEmpty() && measurementDataBlock == null && currentDeviceNoMoreData) { - currentDeviceIndex++; - prepareForNextDevice(); + moveToNextDevice(); } } catch (IOException e) { @@ -252,7 +250,8 @@ public void initQueryDataSource(IQueryDataSource dataSource) { this.measurementDataBuilder.setMaxTsBlockLineNumber(this.maxTsBlockLineNum); } - private void prepareForNextDevice() { + protected void moveToNextDevice() { + currentDeviceIndex++; if (currentDeviceIndex < deviceCount) { // construct AlignedSeriesScanUtil for next device constructAlignedSeriesScanUtil(); @@ -265,8 +264,8 @@ private void prepareForNextDevice() { } } - private void constructAlignedSeriesScanUtil() { - if (this.deviceEntries.isEmpty()) { + protected void constructAlignedSeriesScanUtil() { + if (this.deviceEntries.isEmpty() || currentDeviceIndex >= deviceCount) { // no need to construct SeriesScanUtil, hasNext will return false return; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/ExternalTsFileAggTableScanOperator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/ExternalTsFileAggTableScanOperator.java new file mode 100644 index 0000000000000..022e44fe16856 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/ExternalTsFileAggTableScanOperator.java @@ -0,0 +1,162 @@ +/* + * 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.queryengine.execution.operator.source.relational; + +import org.apache.iotdb.commons.path.AlignedFullPath; +import org.apache.iotdb.db.i18n.DataNodeQueryMessages; +import org.apache.iotdb.db.queryengine.execution.operator.OperatorContext; +import org.apache.iotdb.db.queryengine.execution.operator.source.SeriesScanUtil; +import org.apache.iotdb.db.queryengine.plan.relational.function.tvf.read_tsfile.ExternalTsFileQueryDataSource; +import org.apache.iotdb.db.queryengine.plan.relational.function.tvf.read_tsfile.ExternalTsFileQueryResource; +import org.apache.iotdb.db.queryengine.plan.relational.function.tvf.read_tsfile.ExternalTsFileQueryResource.DeviceTaskRunReader; +import org.apache.iotdb.db.queryengine.plan.relational.metadata.AlignedDeviceEntry; +import org.apache.iotdb.db.queryengine.plan.relational.metadata.DeviceEntry; +import org.apache.iotdb.db.storageengine.dataregion.read.IQueryDataSource; +import org.apache.iotdb.db.storageengine.dataregion.read.QueryDataSource; +import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; + +import org.apache.tsfile.file.metadata.AbstractAlignedTimeSeriesMetadata; +import org.apache.tsfile.utils.Binary; +import org.apache.tsfile.utils.RamUsageEstimator; + +import java.io.IOException; + +import static org.apache.iotdb.db.queryengine.execution.operator.source.relational.TableScanOperator.constructAlignedPath; + +public class ExternalTsFileAggTableScanOperator extends DefaultAggTableScanOperator { + private static final long INSTANCE_SIZE = + RamUsageEstimator.shallowSizeOfInstance(ExternalTsFileAggTableScanOperator.class); + + private final int deviceTaskPartitionIndex; + private DeviceTaskRunReader deviceTaskReader; + + public ExternalTsFileAggTableScanOperator( + AbstractAggTableScanOperatorParameter parameter, int deviceTaskPartitionIndex) { + super(parameter); + this.deviceTaskPartitionIndex = deviceTaskPartitionIndex; + } + + @Override + protected void constructAlignedSeriesScanUtil() { + DeviceEntry deviceEntry = + !hasCurrentRealDeviceEntry() + ? new AlignedDeviceEntry(SeriesScanUtil.EMPTY_DEVICE_ID, new Binary[0]) + : deviceEntries.get(currentDeviceIndex); + this.seriesScanUtil = + new ExternalTsFileSeriesScanUtil( + constructAlignedPath( + deviceEntry, measurementColumnNames, measurementSchemas, allSensors), + scanOrder, + seriesScanOptions, + ((OperatorContext) operatorContext).getInstanceContext(), + true, + measurementColumnTSDataTypes, + this::loadTimeSeriesMetadata); + } + + private AbstractAlignedTimeSeriesMetadata loadTimeSeriesMetadata( + TsFileResource resource, AlignedFullPath alignedPath) throws IOException { + if (!hasCurrentRealDeviceEntry()) { + return null; + } + return ExternalTsFileSeriesScanUtil.loadTimeSeriesMetadata( + resource, + alignedPath, + deviceEntries.get(currentDeviceIndex).getDeviceID(), + deviceTaskReader.getCurrentDeviceOffsetMap().get(resource), + ((OperatorContext) operatorContext).getInstanceContext(), + seriesScanOptions.getGlobalTimeFilter()); + } + + @Override + public void initQueryDataSource(IQueryDataSource dataSource) { + ExternalTsFileQueryResource externalTsFileQueryResource = + ((ExternalTsFileQueryDataSource) dataSource).getExternalTsFileQueryResource(); + if (hasCurrentRealDeviceEntry() && deviceTaskReader == null) { + deviceTaskReader = + externalTsFileQueryResource.getDeviceTaskRunReader(deviceTaskPartitionIndex); + } + super.initQueryDataSource( + hasCurrentRealDeviceEntry() ? updateCurrentDeviceQueryDataSource() : dataSource); + } + + @Override + protected void moveToNextDevice() throws Exception { + nextDevice(); + inputTsBlock = null; + + if (currentDeviceIndex < deviceCount) { + constructAlignedSeriesScanUtil(); + queryDataSource = + hasCurrentRealDeviceEntry() ? updateCurrentDeviceQueryDataSource() : queryDataSource; + seriesScanUtil.initQueryDataSource(queryDataSource); + } + + if (currentDeviceIndex >= deviceCount) { + timeIterator.setFinished(); + } + } + + private boolean hasCurrentRealDeviceEntry() { + return !deviceEntries.isEmpty() + && currentDeviceIndex < deviceEntries.size() + && deviceEntries.get(currentDeviceIndex) != null; + } + + private QueryDataSource updateCurrentDeviceQueryDataSource() { + try { + if (!deviceTaskReader.nextDevice()) { + throw new IllegalStateException( + DataNodeQueryMessages + .UNEXPECTED_END_OF_EXTERNAL_TSFILE_DEVICE_TASK_READER_AT_DEVICE_INDEX + + currentDeviceIndex); + } + DeviceEntry expectedDeviceEntry = deviceEntries.get(currentDeviceIndex); + DeviceEntry currentDeviceEntry = deviceTaskReader.getCurrentDevice(); + if (!expectedDeviceEntry.getDeviceID().equals(currentDeviceEntry.getDeviceID())) { + throw new IllegalStateException( + String.format( + DataNodeQueryMessages + .EXTERNAL_TSFILE_DEVICE_TASK_READER_IS_NOT_ALIGNED_WITH_DEVICE_ENTRIES, + currentDeviceIndex, + expectedDeviceEntry.getDeviceID(), + currentDeviceEntry.getDeviceID())); + } + return deviceTaskReader.getCurrentDeviceQueryDataSource(); + } catch (IOException e) { + throw new RuntimeException( + DataNodeQueryMessages.FAILED_TO_UPDATE_EXTERNAL_TSFILE_DEVICE_RESOURCES, e); + } + } + + @Override + public void close() throws Exception { + if (deviceTaskReader != null) { + deviceTaskReader.close(); + deviceTaskReader = null; + } + super.close(); + } + + @Override + public long ramBytesUsed() { + return super.ramBytesUsed() + INSTANCE_SIZE - AbstractDefaultAggTableScanOperator.INSTANCE_SIZE; + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/ExternalTsFileSeriesScanUtil.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/ExternalTsFileSeriesScanUtil.java new file mode 100644 index 0000000000000..f222d2c820409 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/ExternalTsFileSeriesScanUtil.java @@ -0,0 +1,94 @@ +/* + * 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.queryengine.execution.operator.source.relational; + +import org.apache.iotdb.commons.path.AlignedFullPath; +import org.apache.iotdb.db.queryengine.execution.fragment.FragmentInstanceContext; +import org.apache.iotdb.db.queryengine.execution.operator.source.AlignedSeriesScanUtil; +import org.apache.iotdb.db.queryengine.execution.operator.source.FileLoaderUtils; +import org.apache.iotdb.db.queryengine.plan.planner.plan.parameter.SeriesScanOptions; +import org.apache.iotdb.db.queryengine.plan.relational.function.tvf.read_tsfile.ExternalTsFileQueryResource.DeviceOffset; +import org.apache.iotdb.db.queryengine.plan.statement.component.Ordering; +import org.apache.iotdb.db.storageengine.dataregion.read.QueryDataSource; +import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; + +import org.apache.tsfile.enums.TSDataType; +import org.apache.tsfile.file.metadata.AbstractAlignedTimeSeriesMetadata; +import org.apache.tsfile.file.metadata.IDeviceID; +import org.apache.tsfile.read.filter.basic.Filter; + +import java.io.IOException; +import java.util.List; + +public class ExternalTsFileSeriesScanUtil extends AlignedSeriesScanUtil { + + private final ExternalTsFileMetadataLoader metadataLoader; + + public ExternalTsFileSeriesScanUtil( + AlignedFullPath seriesPath, + Ordering scanOrder, + SeriesScanOptions scanOptions, + FragmentInstanceContext context, + boolean queryAllSensors, + List givenDataTypes, + ExternalTsFileMetadataLoader metadataLoader) { + super(seriesPath, scanOrder, scanOptions, context, queryAllSensors, givenDataTypes); + this.metadataLoader = metadataLoader; + } + + @Override + protected AbstractAlignedTimeSeriesMetadata loadTimeSeriesMetadata( + TsFileResource resource, boolean isSeq) throws IOException { + return metadataLoader.loadTimeSeriesMetadata(resource, (AlignedFullPath) seriesPath); + } + + @Override + protected void updateFilterUsingTTL(QueryDataSource dataSource) { + // External TsFiles are not managed by IoTDB metadata, so no table/tree TTL applies here. + } + + static AbstractAlignedTimeSeriesMetadata loadTimeSeriesMetadata( + TsFileResource resource, + AlignedFullPath alignedPath, + IDeviceID currentDeviceID, + DeviceOffset currentDeviceOffset, + FragmentInstanceContext context, + Filter globalTimeFilter) + throws IOException { + if (currentDeviceOffset == null || !currentDeviceID.equals(alignedPath.getDeviceId())) { + return null; + } + + return FileLoaderUtils.loadAlignedTimeSeriesMetadata( + resource, + alignedPath, + context, + globalTimeFilter, + resource.isSeq(), + context.isIgnoreAllNullRows(), + new long[] {currentDeviceOffset.getStartOffset(), currentDeviceOffset.getEndOffset()}); + } + + @FunctionalInterface + public interface ExternalTsFileMetadataLoader { + AbstractAlignedTimeSeriesMetadata loadTimeSeriesMetadata( + TsFileResource resource, AlignedFullPath alignedFullPath) throws IOException; + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/ExternalTsFileTableScanOperator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/ExternalTsFileTableScanOperator.java new file mode 100644 index 0000000000000..28ef9c1e4d4c6 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/ExternalTsFileTableScanOperator.java @@ -0,0 +1,150 @@ +/* + * 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.queryengine.execution.operator.source.relational; + +import org.apache.iotdb.calc.plan.planner.CommonOperatorUtils; +import org.apache.iotdb.commons.path.AlignedFullPath; +import org.apache.iotdb.db.i18n.DataNodeQueryMessages; +import org.apache.iotdb.db.queryengine.execution.operator.OperatorContext; +import org.apache.iotdb.db.queryengine.plan.relational.function.tvf.read_tsfile.ExternalTsFileQueryDataSource; +import org.apache.iotdb.db.queryengine.plan.relational.function.tvf.read_tsfile.ExternalTsFileQueryResource; +import org.apache.iotdb.db.queryengine.plan.relational.function.tvf.read_tsfile.ExternalTsFileQueryResource.DeviceTaskRunReader; +import org.apache.iotdb.db.queryengine.plan.relational.metadata.DeviceEntry; +import org.apache.iotdb.db.storageengine.dataregion.read.IQueryDataSource; +import org.apache.iotdb.db.storageengine.dataregion.read.QueryDataSource; +import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; + +import org.apache.tsfile.file.metadata.AbstractAlignedTimeSeriesMetadata; +import org.apache.tsfile.utils.RamUsageEstimator; + +import java.io.IOException; + +public class ExternalTsFileTableScanOperator extends TableScanOperator { + private static final long INSTANCE_SIZE = + RamUsageEstimator.shallowSizeOfInstance(ExternalTsFileTableScanOperator.class); + + private final int deviceTaskPartitionIndex; + private DeviceTaskRunReader deviceTaskReader; + + public ExternalTsFileTableScanOperator( + AbstractTableScanOperatorParameter parameter, int deviceTaskPartitionIndex) { + super(parameter); + this.deviceTaskPartitionIndex = deviceTaskPartitionIndex; + } + + @Override + protected void constructAlignedSeriesScanUtil() { + if (currentDeviceIndex >= deviceCount) { + return; + } + + DeviceEntry deviceEntry = deviceEntries.get(currentDeviceIndex); + if (deviceEntry == null) { + throw new IllegalStateException( + DataNodeQueryMessages.CURRENT_DEVICE_ENTRY_IN_TABLESCANOPERATOR_IS_EMPTY); + } + + this.seriesScanUtil = + new ExternalTsFileSeriesScanUtil( + constructAlignedPath( + deviceEntry, measurementColumnNames, measurementSchemas, allSensors), + scanOrder, + seriesScanOptions, + ((OperatorContext) operatorContext).getInstanceContext(), + true, + measurementColumnTSDataTypes, + this::loadTimeSeriesMetadata); + } + + private AbstractAlignedTimeSeriesMetadata loadTimeSeriesMetadata( + TsFileResource resource, AlignedFullPath alignedPath) throws IOException { + return ExternalTsFileSeriesScanUtil.loadTimeSeriesMetadata( + resource, + alignedPath, + deviceEntries.get(currentDeviceIndex).getDeviceID(), + deviceTaskReader.getCurrentDeviceOffsetMap().get(resource), + ((OperatorContext) operatorContext).getInstanceContext(), + seriesScanOptions.getGlobalTimeFilter()); + } + + @Override + public void initQueryDataSource(IQueryDataSource dataSource) { + ExternalTsFileQueryResource externalTsFileQueryResource = + ((ExternalTsFileQueryDataSource) dataSource).getExternalTsFileQueryResource(); + if (deviceTaskReader == null) { + deviceTaskReader = + externalTsFileQueryResource.getDeviceTaskRunReader(deviceTaskPartitionIndex); + } + IQueryDataSource currentDataSource = + currentDeviceIndex < deviceCount ? updateCurrentDeviceQueryDataSource() : dataSource; + super.initQueryDataSource(currentDataSource); + } + + @Override + protected void moveToNextDevice() { + currentDeviceIndex++; + if (currentDeviceIndex < deviceCount) { + constructAlignedSeriesScanUtil(); + seriesScanUtil.initQueryDataSource(updateCurrentDeviceQueryDataSource()); + this.operatorContext.recordSpecifiedInfo( + CommonOperatorUtils.CURRENT_DEVICE_INDEX_STRING, Integer.toString(currentDeviceIndex)); + } + } + + private QueryDataSource updateCurrentDeviceQueryDataSource() { + try { + if (!deviceTaskReader.nextDevice()) { + throw new IllegalStateException( + DataNodeQueryMessages + .UNEXPECTED_END_OF_EXTERNAL_TSFILE_DEVICE_TASK_READER_AT_DEVICE_INDEX + + currentDeviceIndex); + } + DeviceEntry expectedDeviceEntry = deviceEntries.get(currentDeviceIndex); + DeviceEntry currentDeviceEntry = deviceTaskReader.getCurrentDevice(); + if (!expectedDeviceEntry.getDeviceID().equals(currentDeviceEntry.getDeviceID())) { + throw new IllegalStateException( + String.format( + DataNodeQueryMessages + .EXTERNAL_TSFILE_DEVICE_TASK_READER_IS_NOT_ALIGNED_WITH_DEVICE_ENTRIES, + currentDeviceIndex, + expectedDeviceEntry.getDeviceID(), + currentDeviceEntry.getDeviceID())); + } + return deviceTaskReader.getCurrentDeviceQueryDataSource(); + } catch (IOException e) { + throw new RuntimeException( + DataNodeQueryMessages.FAILED_TO_UPDATE_EXTERNAL_TSFILE_DEVICE_RESOURCES, e); + } + } + + @Override + public void close() throws Exception { + if (deviceTaskReader != null) { + deviceTaskReader.close(); + deviceTaskReader = null; + } + super.close(); + } + + @Override + public long ramBytesUsed() { + return super.ramBytesUsed() + INSTANCE_SIZE - AbstractTableScanOperator.INSTANCE_SIZE; + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/InformationSchemaContentSupplierFactory.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/InformationSchemaContentSupplierFactory.java index ca9c09ecb9548..43efe7c7415bb 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/InformationSchemaContentSupplierFactory.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/InformationSchemaContentSupplierFactory.java @@ -86,6 +86,7 @@ import org.apache.iotdb.db.queryengine.plan.execution.IQueryExecution; import org.apache.iotdb.db.queryengine.plan.execution.config.metadata.relational.ShowCreateViewTask; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanGraphPrinter; +import org.apache.iotdb.db.queryengine.plan.relational.function.DataNodeTableBuiltinTableFunction; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.InformationSchemaTableScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.TableDiskUsageInformationSchemaTableScanNode; import org.apache.iotdb.db.queryengine.plan.relational.security.AccessControl; @@ -133,6 +134,7 @@ import java.util.TreeMap; import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; +import java.util.stream.Stream; import static org.apache.iotdb.commons.conf.IoTDBConstant.FUNCTION_STATE_AVAILABLE; import static org.apache.iotdb.commons.conf.IoTDBConstant.FUNCTION_TYPE_BUILTIN_AGG_FUNC; @@ -957,7 +959,11 @@ public boolean hasNext() { TableBuiltinAggregationFunction.getBuiltInAggregateFunctionName().iterator(); } else if (functionType.equals(BINARY_MAP.get(FUNCTION_TYPE_BUILTIN_AGG_FUNC))) { functionType = BINARY_MAP.get(FUNCTION_TYPE_BUILTIN_TABLE_FUNC); - nameIterator = TableBuiltinTableFunction.getBuiltInTableFunctionName().iterator(); + nameIterator = + Stream.concat( + TableBuiltinTableFunction.getBuiltInTableFunctionName().stream(), + DataNodeTableBuiltinTableFunction.getBuiltInTableFunctionName().stream()) + .iterator(); } else { return false; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/QueryExecution.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/QueryExecution.java index a5f96e3c6c0b3..c14293ba380c0 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/QueryExecution.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/QueryExecution.java @@ -29,7 +29,6 @@ import org.apache.iotdb.db.conf.IoTDBConfig; import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.exception.query.KilledByOthersException; -import org.apache.iotdb.db.exception.query.QueryTimeoutRuntimeException; import org.apache.iotdb.db.i18n.DataNodeQueryMessages; import org.apache.iotdb.db.queryengine.common.FragmentInstanceId; import org.apache.iotdb.db.queryengine.common.MPPQueryContext; @@ -223,11 +222,7 @@ private void checkTimeOutForQuery() { // only check query operation's timeout because we will never limit write operation's execution // time if (isQuery()) { - long currentTime = System.currentTimeMillis(); - if (currentTime - context.getStartTime() >= context.getTimeOut()) { - throw new QueryTimeoutRuntimeException( - context.getStartTime(), currentTime, context.getTimeOut()); - } + context.checkTimeOut(); } } @@ -392,6 +387,7 @@ private void releaseResource(Throwable t) { } cleanUpResultHandle(); } + context.releaseExternalTsFileQueryResources(); } /** diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/metadata/ShowFunctionsTask.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/metadata/ShowFunctionsTask.java index d9fcf3a979c76..9866b77cf13ca 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/metadata/ShowFunctionsTask.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/metadata/ShowFunctionsTask.java @@ -36,6 +36,7 @@ import org.apache.iotdb.db.queryengine.plan.execution.config.ConfigTaskResult; import org.apache.iotdb.db.queryengine.plan.execution.config.IConfigTask; import org.apache.iotdb.db.queryengine.plan.execution.config.executor.IConfigTaskExecutor; +import org.apache.iotdb.db.queryengine.plan.relational.function.DataNodeTableBuiltinTableFunction; import org.apache.iotdb.db.queryengine.plan.udf.TreeUDFUtils; import org.apache.iotdb.rpc.TSStatusCode; @@ -158,6 +159,11 @@ private static void buildTableModelTsBlock( TableBuiltinTableFunction.getBuiltInTableFunctionName(), BINARY_MAP.get(FUNCTION_TYPE_BUILTIN_TABLE_FUNC), BINARY_MAP.get(FUNCTION_STATE_AVAILABLE)); + appendFunctions( + builder, + DataNodeTableBuiltinTableFunction.getBuiltInTableFunctionName(), + BINARY_MAP.get(FUNCTION_TYPE_BUILTIN_TABLE_FUNC), + BINARY_MAP.get(FUNCTION_STATE_AVAILABLE)); DatasetHeader datasetHeader = DatasetHeaderFactory.getShowFunctionsHeader(); future.set(new ConfigTaskResult(TSStatusCode.SUCCESS_STATUS, builder.build(), datasetHeader)); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/DataNodeTableOperatorGenerator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/DataNodeTableOperatorGenerator.java index bd66da8daaa53..426531b2e1623 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/DataNodeTableOperatorGenerator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/DataNodeTableOperatorGenerator.java @@ -95,6 +95,8 @@ import org.apache.iotdb.db.queryengine.execution.operator.source.relational.CteScanOperator; import org.apache.iotdb.db.queryengine.execution.operator.source.relational.DefaultAggTableScanOperator; import org.apache.iotdb.db.queryengine.execution.operator.source.relational.DeviceIteratorScanOperator; +import org.apache.iotdb.db.queryengine.execution.operator.source.relational.ExternalTsFileAggTableScanOperator; +import org.apache.iotdb.db.queryengine.execution.operator.source.relational.ExternalTsFileTableScanOperator; import org.apache.iotdb.db.queryengine.execution.operator.source.relational.InformationSchemaTableScanOperator; import org.apache.iotdb.db.queryengine.execution.operator.source.relational.LastQueryAggTableScanOperator; import org.apache.iotdb.db.queryengine.execution.operator.source.relational.TableScanOperator; @@ -122,6 +124,8 @@ import org.apache.iotdb.db.queryengine.plan.relational.planner.node.DeviceTableScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.ExchangeNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.ExplainAnalyzeNode; +import org.apache.iotdb.db.queryengine.plan.relational.planner.node.ExternalTsFileAggregationScanNode; +import org.apache.iotdb.db.queryengine.plan.relational.planner.node.ExternalTsFileScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.InformationSchemaTableScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.IntoNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.NonAlignedAggregationTreeDeviceViewScanNode; @@ -135,6 +139,7 @@ import org.apache.iotdb.db.schemaengine.schemaregion.read.resp.info.IDeviceSchemaInfo; import org.apache.iotdb.db.schemaengine.table.DataNodeTableCache; import org.apache.iotdb.db.schemaengine.table.DataNodeTreeViewSchemaUtils; +import org.apache.iotdb.db.storageengine.dataregion.read.QueryDataSourceType; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; @@ -864,7 +869,7 @@ private CommonTableScanOperatorParameters( symbolInputs = new ArrayList<>(outputColumnCount); columnsIndexArray = new int[outputColumnCount]; columnSchemaMap = node.getAssignments(); - tagAndAttributeColumnsIndexMap = node.getTagAndAttributeIndexMap(); + this.tagAndAttributeColumnsIndexMap = node.getTagAndAttributeIndexMap(); measurementColumnNames = new ArrayList<>(); measurementColumnsIndexMap = new HashMap<>(); measurementSchemas = new ArrayList<>(); @@ -1116,6 +1121,40 @@ public Operator visitDeviceTableScan( return tableScanOperator; } + @Override + public Operator visitExternalTsFileScan( + ExternalTsFileScanNode node, LocalExecutionPlanContext context) { + if (node.getDeviceEntries().isEmpty()) { + OperatorContext operatorContext = + addOperatorContext( + context, node.getPlanNodeId(), EmptyDataOperator.class.getSimpleName()); + return new EmptyDataOperator(operatorContext); + } + + AbstractTableScanOperator.AbstractTableScanOperatorParameter parameter = + constructAbstractTableScanOperatorParameter( + node, + context, + ExternalTsFileTableScanOperator.class.getSimpleName(), + Collections.emptyMap(), + Long.MAX_VALUE); + + AbstractTableScanOperator externalTsFileTableScanOperator = + new ExternalTsFileTableScanOperator(parameter, node.getDeviceTaskPartitionIndex()); + + context.getInstanceContext().collectTable(node.getQualifiedObjectName().getObjectName()); + + DataDriverContext dataDriverContext = (DataDriverContext) context.getDriverContext(); + dataDriverContext.addSourceOperator(externalTsFileTableScanOperator); + dataDriverContext.setQueryDataSourceType(QueryDataSourceType.EXTERNAL_TSFILE_SCAN); + dataDriverContext.setInputDriver(true); + context + .getInstanceContext() + .addExternalTsFileQueryResource(node.getExternalTsFileQueryResource()); + + return externalTsFileTableScanOperator; + } + private SeriesScanOptions.Builder getSeriesScanOptionsBuilder( LocalExecutionPlanContext context, @NotNull Expression timePredicate) { SeriesScanOptions.Builder scanOptionsBuilder = new SeriesScanOptions.Builder(); @@ -1595,6 +1634,34 @@ public Operator visitAggregationTableScan( } } + @Override + public Operator visitExternalTsFileAggregationScan( + ExternalTsFileAggregationScanNode node, LocalExecutionPlanContext context) { + AbstractAggTableScanOperator.AbstractAggTableScanOperatorParameter parameter = + constructAbstractAggTableScanOperatorParameter(node, context); + + ExternalTsFileAggTableScanOperator aggTableScanOperator = + new ExternalTsFileAggTableScanOperator(parameter, node.getDeviceTaskPartitionIndex()); + + context.getInstanceContext().collectTable(node.getQualifiedObjectName().getObjectName()); + addSource( + aggTableScanOperator, + context, + node, + parameter.getMeasurementColumnNames(), + parameter.getMeasurementSchemas(), + parameter.getAllSensors(), + ExternalTsFileAggregationScanNode.class.getSimpleName()); + + DataDriverContext dataDriverContext = (DataDriverContext) context.getDriverContext(); + dataDriverContext.setQueryDataSourceType(QueryDataSourceType.EXTERNAL_TSFILE_SCAN); + context + .getInstanceContext() + .addExternalTsFileQueryResource(node.getExternalTsFileQueryResource()); + + return aggTableScanOperator; + } + private LastQueryAggTableScanOperator constructLastQueryAggTableScanOperator( AggregationTableScanNode node, AbstractAggTableScanOperator.AbstractAggTableScanOperatorParameter parameter, diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/PlanGraphPrinter.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/PlanGraphPrinter.java index f747c18503573..a0f7b2ba9955c 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/PlanGraphPrinter.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/PlanGraphPrinter.java @@ -98,6 +98,7 @@ import org.apache.iotdb.db.queryengine.plan.relational.planner.node.DeviceTableScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.ExchangeNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.ExplainAnalyzeNode; +import org.apache.iotdb.db.queryengine.plan.relational.planner.node.ExternalTsFileScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.TableDiskUsageInformationSchemaTableScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.TreeDeviceViewScanNode; @@ -662,6 +663,10 @@ public List visitTableScan(TableScanNode node, GraphContext context) { if (node instanceof DeviceTableScanNode) { deviceTableScanNode = (DeviceTableScanNode) node; } + ExternalTsFileScanNode externalTsFileScanNode = null; + if (node instanceof ExternalTsFileScanNode) { + externalTsFileScanNode = (ExternalTsFileScanNode) node; + } List boxValue = new ArrayList<>(); boxValue.add(node.toString()); @@ -677,6 +682,10 @@ public List visitTableScan(TableScanNode node, GraphContext context) { String.format("TimePredicate: %s", deviceTableScanNode.getTimePredicate().get())); } } + if (externalTsFileScanNode != null) { + boxValue.add( + String.format("TsFileNumber: %s", externalTsFileScanNode.getTsFilePaths().size())); + } if (node.getPushDownPredicate() != null) { boxValue.add(String.format("PushDownPredicate: %s", node.getPushDownPredicate())); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/PlanVisitor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/PlanVisitor.java index 54ee9d17a9420..81596f2d38c10 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/PlanVisitor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/PlanVisitor.java @@ -124,6 +124,8 @@ import org.apache.iotdb.db.queryengine.plan.relational.planner.node.AlignedAggregationTreeDeviceViewScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.CteScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.DeviceTableScanNode; +import org.apache.iotdb.db.queryengine.plan.relational.planner.node.ExternalTsFileAggregationScanNode; +import org.apache.iotdb.db.queryengine.plan.relational.planner.node.ExternalTsFileScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.InformationSchemaTableScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.NonAlignedAggregationTreeDeviceViewScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.TreeAlignedDeviceViewScanNode; @@ -649,6 +651,14 @@ default R visitDeviceTableScan(DeviceTableScanNode node, C context) { return visitTableScan(node, context); } + default R visitExternalTsFileScan(ExternalTsFileScanNode node, C context) { + return visitTableScan(node, context); + } + + default R visitExternalTsFileAggregationScan(ExternalTsFileAggregationScanNode node, C context) { + return visitAggregationTableScan(node, context); + } + default R visitInformationSchemaTableScan(InformationSchemaTableScanNode node, C context) { return visitTableScan(node, context); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/StatementAnalyzer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/StatementAnalyzer.java index a2d606dbd06e6..28452c5e2bd48 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/StatementAnalyzer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/StatementAnalyzer.java @@ -138,6 +138,7 @@ import org.apache.iotdb.db.queryengine.plan.relational.analyzer.tablefunction.ArgumentsAnalysis; import org.apache.iotdb.db.queryengine.plan.relational.analyzer.tablefunction.TableArgumentAnalysis; import org.apache.iotdb.db.queryengine.plan.relational.analyzer.tablefunction.TableFunctionInvocationAnalysis; +import org.apache.iotdb.db.queryengine.plan.relational.function.tvf.read_tsfile.ReadTsFileTableFunction; import org.apache.iotdb.db.queryengine.plan.relational.metadata.Metadata; import org.apache.iotdb.db.queryengine.plan.relational.planner.PlannerContext; import org.apache.iotdb.db.queryengine.plan.relational.planner.ScopeAware; @@ -5024,6 +5025,9 @@ public Scope visitTableFunctionInvocation(TableFunctionInvocation node, Optional TableFunctionAnalysis functionAnalysis; try { + if (function instanceof ReadTsFileTableFunction) { + ((ReadTsFileTableFunction) function).setMPPQueryContext(queryContext); + } functionAnalysis = function.analyze(argumentsAnalysis.getPassedArguments()); } catch (UDFException e) { throw new SemanticException(e.getMessage()); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/predicate/schema/ConvertSchemaPredicateToFilterVisitor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/predicate/schema/ConvertSchemaPredicateToFilterVisitor.java index 510c748aceea4..a7781bc1e64d9 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/predicate/schema/ConvertSchemaPredicateToFilterVisitor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/predicate/schema/ConvertSchemaPredicateToFilterVisitor.java @@ -55,6 +55,7 @@ import javax.annotation.Nullable; import java.util.ArrayList; +import java.util.Arrays; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -234,8 +235,20 @@ public SchemaFilter visitNullIfExpression(final NullIfExpression node, final Con } @Override - public SchemaFilter visitBetweenPredicate(final BetweenPredicate node, final Context context) { - return visitExpression(node, context); + public @Nullable SchemaFilter visitBetweenPredicate( + final BetweenPredicate node, final Context context) { + final SchemaFilter lowerBoundFilter = + new ComparisonExpression( + ComparisonExpression.Operator.LESS_THAN_OR_EQUAL, node.getMin(), node.getValue()) + .accept(this, context); + final SchemaFilter upperBoundFilter = + new ComparisonExpression( + ComparisonExpression.Operator.LESS_THAN_OR_EQUAL, node.getValue(), node.getMax()) + .accept(this, context); + if (Objects.isNull(lowerBoundFilter) || Objects.isNull(upperBoundFilter)) { + return null; + } + return new AndFilter(Arrays.asList(lowerBoundFilter, upperBoundFilter)); } private SchemaFilter wrapTagOrAttributeFilter( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/DataNodeTableBuiltinTableFunction.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/DataNodeTableBuiltinTableFunction.java new file mode 100644 index 0000000000000..5060fb8f3def4 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/DataNodeTableBuiltinTableFunction.java @@ -0,0 +1,65 @@ +/* + * 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.queryengine.plan.relational.function; + +import org.apache.iotdb.commons.exception.SemanticException; +import org.apache.iotdb.commons.i18n.QueryMessages; +import org.apache.iotdb.db.queryengine.plan.relational.function.tvf.read_tsfile.ReadTsFileTableFunction; +import org.apache.iotdb.udf.api.relational.TableFunction; + +import java.util.Arrays; +import java.util.Set; +import java.util.stream.Collectors; + +public enum DataNodeTableBuiltinTableFunction { + READ_TSFILE("read_tsfile"); + + private final String functionName; + + DataNodeTableBuiltinTableFunction(String functionName) { + this.functionName = functionName; + } + + public String getFunctionName() { + return functionName; + } + + private static final Set BUILT_IN_TABLE_FUNCTION_NAME = + Arrays.stream(DataNodeTableBuiltinTableFunction.values()) + .map(DataNodeTableBuiltinTableFunction::getFunctionName) + .collect(Collectors.toSet()); + + public static Set getBuiltInTableFunctionName() { + return BUILT_IN_TABLE_FUNCTION_NAME; + } + + public static boolean isBuiltInTableFunction(String functionName) { + return BUILT_IN_TABLE_FUNCTION_NAME.contains(functionName.toLowerCase()); + } + + public static TableFunction getBuiltinTableFunction(String functionName) { + switch (functionName.toLowerCase()) { + case "read_tsfile": + return new ReadTsFileTableFunction(); + default: + throw new SemanticException(String.format(QueryMessages.UNKNOWN_FUNCTION, functionName)); + } + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/read_tsfile/ExternalTsFileDeviceFilterVisitor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/read_tsfile/ExternalTsFileDeviceFilterVisitor.java new file mode 100644 index 0000000000000..7deb98827f692 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/read_tsfile/ExternalTsFileDeviceFilterVisitor.java @@ -0,0 +1,54 @@ +/* + * 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.queryengine.plan.relational.function.tvf.read_tsfile; + +import org.apache.iotdb.commons.schema.filter.SchemaFilter; +import org.apache.iotdb.commons.schema.filter.SchemaFilterVisitor; +import org.apache.iotdb.commons.schema.filter.impl.StringValueFilterVisitor; +import org.apache.iotdb.commons.schema.filter.impl.singlechild.AttributeFilter; +import org.apache.iotdb.commons.schema.filter.impl.singlechild.TagFilter; +import org.apache.iotdb.db.i18n.DataNodeQueryMessages; + +import org.apache.tsfile.file.metadata.IDeviceID; + +public class ExternalTsFileDeviceFilterVisitor extends SchemaFilterVisitor { + + @Override + protected Boolean visitNode(final SchemaFilter filter, final IDeviceID deviceID) { + throw new UnsupportedOperationException( + String.format( + DataNodeQueryMessages.SCHEMA_FILTER_TYPE_IS_NOT_SUPPORTED, + filter.getSchemaFilterType())); + } + + @Override + public Boolean visitTagFilter(final TagFilter filter, final IDeviceID deviceID) { + final int index = filter.getIndex() + 1; + final String value = index < deviceID.segmentNum() ? (String) deviceID.segment(index) : null; + return filter.getChild().accept(StringValueFilterVisitor.getInstance(), value); + } + + @Override + public Boolean visitAttributeFilter(final AttributeFilter filter, final IDeviceID deviceID) { + throw new UnsupportedOperationException( + DataNodeQueryMessages + .ATTRIBUTE_FILTER_IS_NOT_SUPPORTED_FOR_EXTERNAL_TSFILE_DEVICE_FILTERING); + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/read_tsfile/ExternalTsFileQueryDataSource.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/read_tsfile/ExternalTsFileQueryDataSource.java new file mode 100644 index 0000000000000..e4ceb4f68b1bf --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/read_tsfile/ExternalTsFileQueryDataSource.java @@ -0,0 +1,49 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.queryengine.plan.relational.function.tvf.read_tsfile; + +import org.apache.iotdb.db.storageengine.dataregion.read.IQueryDataSource; +import org.apache.iotdb.db.storageengine.dataregion.read.QueryDataSource; + +import java.util.Collections; + +public class ExternalTsFileQueryDataSource extends QueryDataSource { + + private final ExternalTsFileQueryResource externalTsFileQueryResource; + + public ExternalTsFileQueryDataSource(ExternalTsFileQueryResource externalTsFileQueryResource) { + super(Collections.emptyList(), Collections.emptyList()); + this.externalTsFileQueryResource = externalTsFileQueryResource; + } + + public ExternalTsFileQueryResource getExternalTsFileQueryResource() { + return externalTsFileQueryResource; + } + + @Override + public IQueryDataSource clone() { + return new ExternalTsFileQueryDataSource(externalTsFileQueryResource); + } + + @Override + public boolean isEmpty() { + return externalTsFileQueryResource.getSharedTsFileResources().isEmpty(); + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/read_tsfile/ExternalTsFileQueryResource.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/read_tsfile/ExternalTsFileQueryResource.java new file mode 100644 index 0000000000000..8f4fb8883db3e --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/read_tsfile/ExternalTsFileQueryResource.java @@ -0,0 +1,895 @@ +/* + * 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.queryengine.plan.relational.function.tvf.read_tsfile; + +import org.apache.iotdb.calc.exception.MemoryNotEnoughException; +import org.apache.iotdb.calc.plan.planner.memory.MemoryReservationManager; +import org.apache.iotdb.commons.queryengine.plan.planner.plan.node.PlanNodeId; +import org.apache.iotdb.commons.queryengine.plan.relational.metadata.ColumnSchema; +import org.apache.iotdb.commons.queryengine.plan.relational.planner.Symbol; +import org.apache.iotdb.commons.schema.filter.SchemaFilter; +import org.apache.iotdb.commons.utils.FileUtils; +import org.apache.iotdb.commons.utils.TestOnly; +import org.apache.iotdb.db.i18n.DataNodeQueryMessages; +import org.apache.iotdb.db.queryengine.common.MPPQueryContext; +import org.apache.iotdb.db.queryengine.common.QueryId; +import org.apache.iotdb.db.queryengine.plan.planner.memory.NotThreadSafeMemoryReservationManager; +import org.apache.iotdb.db.queryengine.plan.relational.metadata.AlignedDeviceEntry; +import org.apache.iotdb.db.queryengine.plan.relational.metadata.DeviceEntry; +import org.apache.iotdb.db.storageengine.dataregion.read.QueryDataSource; +import org.apache.iotdb.db.storageengine.dataregion.read.control.FileReaderManager; +import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; +import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResourceStatus; +import org.apache.iotdb.db.storageengine.dataregion.tsfile.timeindex.FileTimeIndex; + +import org.apache.tsfile.file.metadata.IDeviceID; +import org.apache.tsfile.read.LazyTsFileDeviceIterator; +import org.apache.tsfile.read.TsFileSequenceReader; +import org.apache.tsfile.utils.Accountable; +import org.apache.tsfile.utils.Binary; +import org.apache.tsfile.utils.RamUsageEstimator; +import org.apache.tsfile.utils.ReadWriteIOUtils; + +import java.io.BufferedInputStream; +import java.io.BufferedOutputStream; +import java.io.Closeable; +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.File; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.PriorityQueue; +import java.util.Queue; + +import static java.util.Objects.requireNonNull; + +/** + * Query-scoped resource for external TsFile scans. + * + *

Reader instances are owned by {@link FileReaderManager}; this class only balances reference + * increments/decrements for the query's external TsFile paths and deletes the query temporary + * directory. + */ +public class ExternalTsFileQueryResource { + + public static final String EXTERNAL_TSFILE_TMP_DIR = "external-tsfile"; + + private static final long TSFILE_READER_MEMORY_RESERVE_SIZE_IN_BYTES = 4L * 1024; + + private final MPPQueryContext queryContext; + private final QueryId queryId; + // This resource outlives the frontend planning phase, whose MPPQueryContext memory manager is + // released after dispatch. Keep a dedicated manager and release it when this resource closes. + private final MemoryReservationManager externalTsFileResourceMemoryReservationManager; + private final Path queryTempRoot; + private final String tableName; + private final List tsFilePaths; + private final Map tableColumnSchema; + private final List sharedTsFileResources; + private final List sharedDeviceEntries = new ArrayList<>(); + private final List deviceTaskPartitions = new ArrayList<>(); + private Comparator deviceEntryComparator; + + // Counts FragmentInstances that have entered runtime datasource initialization and may read + // files under queryTempRoot. QueryExecution cleanup only closes this resource when the count is + // zero. Otherwise the last FragmentInstance release closes it, which prevents QueryExecution from + // deleting temporary run files while drivers are still reading them. + private int fragmentInstanceUsageCount; + private boolean closed; + + public ExternalTsFileQueryResource( + MPPQueryContext queryContext, + Path tempRoot, + String tableName, + List tsFilePaths, + Map tableColumnSchema) { + this.queryContext = requireNonNull(queryContext, "queryContext is null"); + this.queryId = queryContext.getQueryId(); + this.externalTsFileResourceMemoryReservationManager = + new NotThreadSafeMemoryReservationManager( + queryId, ExternalTsFileQueryResource.class.getName()); + this.queryTempRoot = requireNonNull(tempRoot, "tempRoot is null"); + this.tableName = tableName; + this.tsFilePaths = requireNonNull(tsFilePaths, "tsFilePaths"); + this.tableColumnSchema = tableColumnSchema; + this.sharedTsFileResources = createTsFileResources(this.tsFilePaths); + for (String tsFilePath : tsFilePaths) { + FileReaderManager.getInstance().increaseExternalFileReaderReference(tsFilePath); + } + } + + public void collectDeviceEntries( + SchemaFilter schemaFilter, Comparator comparator, int partitionCount) { + if (partitionCount <= 0) { + throw new IllegalArgumentException( + DataNodeQueryMessages.EXTERNAL_TSFILE_DEVICE_TASK_PARTITION_COUNT_MUST_BE_POSITIVE); + } + this.deviceEntryComparator = comparator; + acquireMemoryForTsFileReaders(); + ExternalTsFileDeviceFilterVisitor deviceFilterVisitor = new ExternalTsFileDeviceFilterVisitor(); + try (DeviceCollector deviceCollector = new DeviceCollector()) { + while (deviceCollector.hasNextDevice()) { + queryContext.checkTimeOut(); + IDeviceID deviceID = deviceCollector.nextDevice(); + if (schemaFilter != null + && !Boolean.TRUE.equals(schemaFilter.accept(deviceFilterVisitor, deviceID))) { + continue; + } + DeviceEntry deviceEntry = new AlignedDeviceEntry(deviceID, new Binary[0]); + int deviceEntryIndex = sharedDeviceEntries.size(); + externalTsFileResourceMemoryReservationManager.reserveMemoryCumulatively( + deviceEntry.ramBytesUsed()); + sharedDeviceEntries.add(deviceEntry); + DeviceTask deviceTask = + new DeviceTask(deviceEntryIndex, deviceCollector.getCurrentDeviceOffsets()); + DeviceTaskPartition partition = + getOrCreateDeviceTaskPartition(Math.floorMod(deviceID.hashCode(), partitionCount)); + partition.add(deviceTask); + if (partition.shouldFlush()) { + partition.flush(); + } + } + collectDeviceTaskPartitions(); + } + } + + private void acquireMemoryForTsFileReaders() { + externalTsFileResourceMemoryReservationManager.reserveMemoryImmediately( + tsFilePaths.size() * TSFILE_READER_MEMORY_RESERVE_SIZE_IN_BYTES); + } + + public DeviceTaskRunReader getDeviceTaskRunReader(int partitionIndex) { + DeviceTaskPartition partition = getDeviceTaskPartition(partitionIndex); + try { + return deviceEntryComparator == null + ? new SequentialDeviceTaskRunReader(partition) + : new PriorityDeviceTaskRunReader(partition); + } catch (IOException e) { + throw new RuntimeException( + DataNodeQueryMessages.FAILED_TO_CREATE_EXTERNAL_TSFILE_DEVICE_TASK_RUN_READER, e); + } + } + + @TestOnly + public void setDeviceEntryComparator(Comparator deviceEntryComparator) { + this.deviceEntryComparator = deviceEntryComparator; + } + + public List getTsFilePaths() { + return tsFilePaths; + } + + public Map getTableColumnSchema() { + return tableColumnSchema; + } + + public List getSharedTsFileResources() { + return sharedTsFileResources; + } + + public List getSharedDeviceEntries() { + return sharedDeviceEntries; + } + + public List getDeviceTaskPartitions() { + return deviceTaskPartitions; + } + + private DeviceTaskPartition getDeviceTaskPartition(int partitionIndex) { + for (DeviceTaskPartition partition : deviceTaskPartitions) { + if (partition.getPartitionIndex() == partitionIndex) { + return partition; + } + } + throw new IllegalArgumentException( + DataNodeQueryMessages.UNKNOWN_EXTERNAL_TSFILE_DEVICE_TASK_PARTITION + partitionIndex); + } + + public synchronized void retainFragmentInstanceUsage() { + checkNotClosed(); + fragmentInstanceUsageCount++; + } + + public synchronized void closeByFragmentInstance() { + fragmentInstanceUsageCount--; + if (fragmentInstanceUsageCount < 0) { + fragmentInstanceUsageCount++; + throw new IllegalStateException( + DataNodeQueryMessages.EXTERNAL_TSFILE_FRAGMENT_INSTANCE_USAGE_COUNT_CANNOT_BE_NEGATIVE); + } + if (fragmentInstanceUsageCount == 0) { + close(); + } + } + + public synchronized void closeByQueryExecution() { + if (fragmentInstanceUsageCount == 0) { + close(); + } + } + + private void close() { + if (closed) { + return; + } + closed = true; + + try { + releaseFileReaderReferences(); + + if (Files.exists(queryTempRoot)) { + FileUtils.deleteFileOrDirectory(queryTempRoot.toFile(), true); + } + } finally { + try { + externalTsFileResourceMemoryReservationManager.releaseAllReservedMemory(); + } finally { + queryContext.removeExternalTsFileQueryResource(this); + } + } + } + + private void releaseFileReaderReferences() { + for (String tsFilePath : tsFilePaths) { + FileReaderManager.getInstance().decreaseExternalFileReaderReference(tsFilePath); + } + } + + private void checkNotClosed() { + if (closed) { + throw new IllegalStateException( + DataNodeQueryMessages.EXTERNAL_TSFILE_QUERY_RESOURCE_HAS_BEEN_CLOSED + queryId); + } + } + + private static List createTsFileResources(List tsFilePaths) { + List tsFileResources = new ArrayList<>(tsFilePaths.size()); + for (String tsFilePath : tsFilePaths) { + TsFileResource resource = + new TsFileResource(new File(tsFilePath), TsFileResourceStatus.NORMAL); + resource.setTimeIndex(new FileTimeIndex(Long.MIN_VALUE, Long.MAX_VALUE)); + tsFileResources.add(resource); + } + return Collections.unmodifiableList(tsFileResources); + } + + public class DeviceTaskPartition { + + private static final long DEVICE_TASK_BUCKET_TARGET_SIZE_IN_BYTES = 8L * 1024 * 1024; + private static final long MEMORY_RESERVE_BATCH_SIZE_IN_BYTES = 1024 * 1024; + + private final int partitionIndex; + private final PlanNodeId planNodeId; + private final List pendingDeviceTasks = new ArrayList<>(); + private final List deviceEntryIndexes = new ArrayList<>(); + private final List runFiles = new ArrayList<>(); + private long reservedBytes; + private long unreservedBytes; + + DeviceTaskPartition(int partitionIndex) { + this.partitionIndex = partitionIndex; + this.planNodeId = queryId.genPlanNodeId(); + } + + public int getPartitionIndex() { + return partitionIndex; + } + + public PlanNodeId getPlanNodeId() { + return planNodeId; + } + + public List getDeviceEntryIndexes() { + return deviceEntryIndexes; + } + + void add(DeviceTask deviceTask) { + pendingDeviceTasks.add(deviceTask); + unreservedBytes += deviceTask.ramBytesUsed(); + } + + void flush() { + if (pendingDeviceTasks.isEmpty()) { + return; + } + sortPendingDeviceTasks(); + try { + runFiles.add( + writeDeviceTaskRun( + queryTempRoot.resolve(planNodeId.getId()), runFiles.size(), pendingDeviceTasks)); + } catch (IOException e) { + throw new RuntimeException( + DataNodeQueryMessages.FAILED_TO_FLUSH_EXTERNAL_TSFILE_DEVICE_TASK_PARTITION, e); + } + for (DeviceTask deviceTask : pendingDeviceTasks) { + deviceEntryIndexes.add(deviceTask.deviceEntryIndex); + } + pendingDeviceTasks.clear(); + releaseDeviceTaskMemory(); + } + + private void sortPendingDeviceTasks() { + if (deviceEntryComparator != null) { + pendingDeviceTasks.sort( + (left, right) -> + compareDeviceEntryIndexes(left.deviceEntryIndex, right.deviceEntryIndex)); + } else { + pendingDeviceTasks.sort( + (left, right) -> + sharedDeviceEntries + .get(left.deviceEntryIndex) + .getDeviceID() + .compareTo(sharedDeviceEntries.get(right.deviceEntryIndex).getDeviceID())); + } + } + + private boolean shouldFlush() { + if (getPendingMemoryBytes() >= DEVICE_TASK_BUCKET_TARGET_SIZE_IN_BYTES) { + return true; + } + if (unreservedBytes < MEMORY_RESERVE_BATCH_SIZE_IN_BYTES) { + return false; + } + return !reserveUnreservedMemory(); + } + + private boolean reserveUnreservedMemory() { + if (unreservedBytes == 0) { + return true; + } + try { + externalTsFileResourceMemoryReservationManager.reserveMemoryImmediately(unreservedBytes); + } catch (MemoryNotEnoughException e) { + return false; + } + reservedBytes += unreservedBytes; + unreservedBytes = 0; + return true; + } + + private long getPendingMemoryBytes() { + return reservedBytes + unreservedBytes; + } + + private void releaseDeviceTaskMemory() { + if (reservedBytes != 0) { + externalTsFileResourceMemoryReservationManager.releaseMemoryCumulatively(reservedBytes); + reservedBytes = 0; + } + unreservedBytes = 0; + } + + void finish() { + if (pendingDeviceTasks.isEmpty()) { + return; + } + if (!reserveUnreservedMemory()) { + flush(); + return; + } + sortPendingDeviceTasks(); + for (DeviceTask deviceTask : pendingDeviceTasks) { + deviceEntryIndexes.add(deviceTask.deviceEntryIndex); + } + } + + private void sortDeviceEntries() { + if (deviceEntryComparator != null) { + deviceEntryIndexes.sort(ExternalTsFileQueryResource.this::compareDeviceEntryIndexes); + } else { + deviceEntryIndexes.sort( + (left, right) -> + sharedDeviceEntries + .get(left) + .getDeviceID() + .compareTo(sharedDeviceEntries.get(right).getDeviceID())); + } + } + + private List getRunFiles() { + return runFiles; + } + + private List getPendingDeviceTasks() { + return pendingDeviceTasks; + } + } + + private int compareDeviceEntryIndexes(int leftIndex, int rightIndex) { + int result = + deviceEntryComparator.compare( + sharedDeviceEntries.get(leftIndex), sharedDeviceEntries.get(rightIndex)); + // Use the stable device entry index as a tie-breaker so list sorting and run-file merging keep + // the same deterministic order when the pushed-down comparator is not a total order. + return result != 0 ? result : Integer.compare(leftIndex, rightIndex); + } + + private DeviceTaskPartition getOrCreateDeviceTaskPartition(int partitionIndex) { + for (DeviceTaskPartition partition : deviceTaskPartitions) { + if (partition.getPartitionIndex() == partitionIndex) { + return partition; + } + } + DeviceTaskPartition partition = new DeviceTaskPartition(partitionIndex); + deviceTaskPartitions.add(partition); + return partition; + } + + private void collectDeviceTaskPartitions() { + deviceTaskPartitions.sort(Comparator.comparingInt(DeviceTaskPartition::getPartitionIndex)); + for (DeviceTaskPartition partition : deviceTaskPartitions) { + partition.finish(); + partition.sortDeviceEntries(); + } + } + + private Path writeDeviceTaskRun(Path runRoot, int runIndex, List deviceTasks) + throws IOException { + Files.createDirectories(runRoot); + Path runFile = runRoot.resolve("run-" + runIndex + ".bin"); + try (DataOutputStream outputStream = + new DataOutputStream(new BufferedOutputStream(Files.newOutputStream(runFile)))) { + ReadWriteIOUtils.write(deviceTasks.size(), outputStream); + for (DeviceTask deviceTask : deviceTasks) { + deviceTask.serialize(outputStream); + } + } + return runFile; + } + + public abstract class DeviceTaskRunReader implements AutoCloseable { + + private DeviceEntry currentDevice; + private QueryDataSource currentDeviceQueryDataSource; + private Map currentDeviceOffsetMap; + + protected void initialize(DeviceTaskPartition partition) throws IOException { + for (Path runFile : partition.getRunFiles()) { + DeviceTaskRunCursor cursor = new DiskDeviceTaskRunCursor(runFile); + if (cursor.hasCurrentDeviceTask()) { + addCursor(cursor); + } else { + cursor.close(); + } + } + DeviceTaskRunCursor memoryCursor = + new MemoryDeviceTaskRunCursor(partition.getPendingDeviceTasks()); + if (memoryCursor.hasCurrentDeviceTask()) { + addCursor(memoryCursor); + } else { + memoryCursor.close(); + } + } + + public boolean nextDevice() throws IOException { + DeviceTaskRunCursor cursor = pollCursor(); + if (cursor == null) { + return false; + } + DeviceTask result = cursor.getCurrentDeviceTask(); + cursor.advance(); + recycleOrCloseCursor(cursor); + + currentDevice = sharedDeviceEntries.get(result.deviceEntryIndex); + List unseqResources = new ArrayList<>(result.deviceOffsets.size()); + currentDeviceOffsetMap = new HashMap<>(result.deviceOffsets.size()); + for (DeviceOffset deviceOffset : result.deviceOffsets) { + TsFileResource tsFileResource = sharedTsFileResources.get(deviceOffset.getFileIndex()); + unseqResources.add(tsFileResource); + currentDeviceOffsetMap.put(tsFileResource, deviceOffset); + } + currentDeviceQueryDataSource = new QueryDataSource(Collections.emptyList(), unseqResources); + currentDeviceQueryDataSource.setSingleDevice(true); + return true; + } + + protected abstract DeviceTaskRunCursor pollCursor() throws IOException; + + protected abstract void addCursor(DeviceTaskRunCursor cursor); + + protected abstract void recycleOrCloseCursor(DeviceTaskRunCursor cursor) throws IOException; + + protected abstract DeviceTaskRunCursor pollRemainingCursor(); + + public DeviceEntry getCurrentDevice() { + return currentDevice; + } + + public QueryDataSource getCurrentDeviceQueryDataSource() { + return currentDeviceQueryDataSource; + } + + public Map getCurrentDeviceOffsetMap() { + return currentDeviceOffsetMap; + } + + @Override + public void close() throws IOException { + IOException exception = null; + DeviceTaskRunCursor cursor; + while ((cursor = pollRemainingCursor()) != null) { + try { + cursor.close(); + } catch (IOException e) { + if (exception == null) { + exception = e; + } else { + exception.addSuppressed(e); + } + } + } + if (exception != null) { + throw exception; + } + } + } + + private class PriorityDeviceTaskRunReader extends DeviceTaskRunReader { + + private final Queue runCursors = + new PriorityQueue<>( + (left, right) -> + compareDeviceEntryIndexes( + left.getCurrentDeviceTask().deviceEntryIndex, + right.getCurrentDeviceTask().deviceEntryIndex)); + + private PriorityDeviceTaskRunReader(DeviceTaskPartition partition) throws IOException { + initialize(partition); + } + + @Override + protected DeviceTaskRunCursor pollCursor() { + return runCursors.poll(); + } + + @Override + protected void addCursor(DeviceTaskRunCursor cursor) { + runCursors.add(cursor); + } + + @Override + protected void recycleOrCloseCursor(DeviceTaskRunCursor cursor) throws IOException { + if (cursor.hasCurrentDeviceTask()) { + addCursor(cursor); + } else { + cursor.close(); + } + } + + @Override + protected DeviceTaskRunCursor pollRemainingCursor() { + return runCursors.poll(); + } + } + + private class SequentialDeviceTaskRunReader extends DeviceTaskRunReader { + + private final Iterator runFileIterator; + private final List pendingDeviceTasks; + private DeviceTaskRunCursor currentCursor; + private boolean memoryCursorLoaded; + + private SequentialDeviceTaskRunReader(DeviceTaskPartition partition) { + this.runFileIterator = partition.getRunFiles().iterator(); + this.pendingDeviceTasks = partition.getPendingDeviceTasks(); + } + + @Override + protected DeviceTaskRunCursor pollCursor() throws IOException { + if (currentCursor != null) { + DeviceTaskRunCursor cursor = currentCursor; + currentCursor = null; + return cursor; + } + while (runFileIterator.hasNext()) { + DeviceTaskRunCursor cursor = new DiskDeviceTaskRunCursor(runFileIterator.next()); + if (cursor.hasCurrentDeviceTask()) { + return cursor; + } + cursor.close(); + } + if (!memoryCursorLoaded) { + memoryCursorLoaded = true; + DeviceTaskRunCursor cursor = new MemoryDeviceTaskRunCursor(pendingDeviceTasks); + if (cursor.hasCurrentDeviceTask()) { + return cursor; + } + cursor.close(); + } + return null; + } + + @Override + protected void addCursor(DeviceTaskRunCursor cursor) { + currentCursor = cursor; + } + + @Override + protected void recycleOrCloseCursor(DeviceTaskRunCursor cursor) throws IOException { + if (cursor.hasCurrentDeviceTask()) { + currentCursor = cursor; + } else { + cursor.close(); + } + } + + @Override + protected DeviceTaskRunCursor pollRemainingCursor() { + DeviceTaskRunCursor cursor = currentCursor; + currentCursor = null; + return cursor; + } + } + + private interface DeviceTaskRunCursor extends Closeable { + + boolean hasCurrentDeviceTask(); + + DeviceTask getCurrentDeviceTask(); + + void advance() throws IOException; + } + + private static class DiskDeviceTaskRunCursor implements DeviceTaskRunCursor { + + private final DataInputStream inputStream; + private int remainingDeviceTasks; + private DeviceTask currentDeviceTask; + + private DiskDeviceTaskRunCursor(Path runFile) throws IOException { + this.inputStream = + new DataInputStream(new BufferedInputStream(Files.newInputStream(runFile))); + this.remainingDeviceTasks = ReadWriteIOUtils.readInt(inputStream); + advance(); + } + + @Override + public void advance() throws IOException { + if (remainingDeviceTasks <= 0) { + currentDeviceTask = null; + return; + } + remainingDeviceTasks--; + currentDeviceTask = DeviceTask.deserialize(inputStream); + } + + @Override + public boolean hasCurrentDeviceTask() { + return currentDeviceTask != null; + } + + @Override + public DeviceTask getCurrentDeviceTask() { + return currentDeviceTask; + } + + @Override + public void close() throws IOException { + inputStream.close(); + } + } + + private static class MemoryDeviceTaskRunCursor implements DeviceTaskRunCursor { + + private final List deviceTasks; + private int nextIndex; + private DeviceTask currentDeviceTask; + + private MemoryDeviceTaskRunCursor(List deviceTasks) { + this.deviceTasks = deviceTasks; + advance(); + } + + @Override + public void advance() { + if (nextIndex >= deviceTasks.size()) { + currentDeviceTask = null; + return; + } + currentDeviceTask = deviceTasks.get(nextIndex++); + } + + @Override + public boolean hasCurrentDeviceTask() { + return currentDeviceTask != null; + } + + @Override + public DeviceTask getCurrentDeviceTask() { + return currentDeviceTask; + } + + @Override + public void close() { + currentDeviceTask = null; + } + } + + private class DeviceCollector implements Closeable { + + private final Map deviceIteratorMap = new HashMap<>(); + + private IDeviceID currentDevice; + private List currentDeviceOffsets; + + private DeviceCollector() { + try { + for (int fileIndex = 0; fileIndex < tsFilePaths.size(); fileIndex++) { + TsFileSequenceReader reader = + FileReaderManager.getInstance() + .get(tsFilePaths.get(fileIndex), null, true, null, true); + deviceIteratorMap.put(fileIndex, new LazyTsFileDeviceIterator(reader, tableName, null)); + } + } catch (IOException e) { + close(); + throw new RuntimeException( + DataNodeQueryMessages.FAILED_TO_CREATE_EXTERNAL_TSFILE_DEVICE_COLLECTOR, e); + } + } + + private boolean hasNextDevice() { + for (LazyTsFileDeviceIterator deviceIterator : deviceIteratorMap.values()) { + if (deviceIterator.hasNext() + || (deviceIterator.hasCurrent() + && !deviceIterator.getCurrentDeviceID().equals(currentDevice))) { + return true; + } + } + return false; + } + + private IDeviceID nextDevice() { + IDeviceID minDevice = null; + Iterator> iterator = + deviceIteratorMap.entrySet().iterator(); + while (iterator.hasNext()) { + Map.Entry entry = iterator.next(); + LazyTsFileDeviceIterator deviceIterator = entry.getValue(); + IDeviceID currentFileDevice = null; + if (deviceIterator.hasCurrent()) { + currentFileDevice = deviceIterator.getCurrentDeviceID(); + } + if (currentFileDevice == null || currentFileDevice.equals(currentDevice)) { + if (deviceIterator.hasNext()) { + currentFileDevice = deviceIterator.next(); + } else { + iterator.remove(); + continue; + } + } + if (minDevice == null || minDevice.compareTo(currentFileDevice) > 0) { + minDevice = currentFileDevice; + } + } + currentDevice = minDevice; + collectCurrentDeviceOffsets(); + return currentDevice; + } + + private void collectCurrentDeviceOffsets() { + List deviceOffsets = new ArrayList<>(); + for (Map.Entry entry : deviceIteratorMap.entrySet()) { + LazyTsFileDeviceIterator deviceIterator = entry.getValue(); + if (currentDevice != null + && deviceIterator.hasCurrent() + && currentDevice.equals(deviceIterator.getCurrentDeviceID())) { + deviceOffsets.add( + new DeviceOffset( + entry.getKey(), + deviceIterator.getCurrentDeviceMeasurementNodeOffset()[0], + deviceIterator.getCurrentDeviceMeasurementNodeOffset()[1])); + } + } + currentDeviceOffsets = deviceOffsets; + } + + private List getCurrentDeviceOffsets() { + return currentDeviceOffsets; + } + + @Override + public void close() { + deviceIteratorMap.clear(); + currentDeviceOffsets = Collections.emptyList(); + } + } + + static class DeviceTask implements Accountable { + + private static final long INSTANCE_SIZE = + RamUsageEstimator.shallowSizeOfInstance(DeviceTask.class); + + private final int deviceEntryIndex; + private final List deviceOffsets; + + DeviceTask(int deviceEntryIndex, List deviceOffsets) { + this.deviceEntryIndex = deviceEntryIndex; + this.deviceOffsets = deviceOffsets; + } + + private void serialize(DataOutputStream outputStream) throws IOException { + ReadWriteIOUtils.write(deviceEntryIndex, outputStream); + ReadWriteIOUtils.write(deviceOffsets.size(), outputStream); + for (DeviceOffset offset : deviceOffsets) { + ReadWriteIOUtils.write(offset.fileIndex, outputStream); + ReadWriteIOUtils.write(offset.startOffset, outputStream); + ReadWriteIOUtils.write(offset.endOffset, outputStream); + } + } + + private static DeviceTask deserialize(DataInputStream inputStream) throws IOException { + int deviceEntryIndex = ReadWriteIOUtils.readInt(inputStream); + int offsetSize = ReadWriteIOUtils.readInt(inputStream); + List offsets = new ArrayList<>(offsetSize); + for (int i = 0; i < offsetSize; i++) { + int fileIndex = ReadWriteIOUtils.readInt(inputStream); + long startOffset = inputStream.readLong(); + long endOffset = inputStream.readLong(); + offsets.add(new DeviceOffset(fileIndex, startOffset, endOffset)); + } + return new DeviceTask(deviceEntryIndex, offsets); + } + + @Override + public long ramBytesUsed() { + return INSTANCE_SIZE + + RamUsageEstimator.shallowSizeOfInstance(ArrayList.class) + + RamUsageEstimator.NUM_BYTES_ARRAY_HEADER + + (long) RamUsageEstimator.NUM_BYTES_OBJECT_REF * deviceOffsets.size() + + deviceOffsets.size() * DeviceOffset.INSTANCE_SIZE; + } + } + + public static class DeviceOffset { + + private static final long INSTANCE_SIZE = + RamUsageEstimator.shallowSizeOfInstance(DeviceOffset.class); + + private final int fileIndex; + private final long startOffset; + private final long endOffset; + + DeviceOffset(int fileIndex, long startOffset, long endOffset) { + this.fileIndex = fileIndex; + this.startOffset = startOffset; + this.endOffset = endOffset; + } + + public int getFileIndex() { + return fileIndex; + } + + public long getStartOffset() { + return startOffset; + } + + public long getEndOffset() { + return endOffset; + } + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/read_tsfile/ReadTsFileTableFunction.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/read_tsfile/ReadTsFileTableFunction.java new file mode 100644 index 0000000000000..d141d0112c895 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/read_tsfile/ReadTsFileTableFunction.java @@ -0,0 +1,305 @@ +/* + * 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.queryengine.plan.relational.function.tvf.read_tsfile; + +import org.apache.iotdb.commons.schema.table.column.TsTableColumnCategory; +import org.apache.iotdb.commons.udf.utils.UDFDataTypeTransformer; +import org.apache.iotdb.db.conf.IoTDBDescriptor; +import org.apache.iotdb.db.i18n.DataNodeQueryMessages; +import org.apache.iotdb.db.queryengine.common.MPPQueryContext; +import org.apache.iotdb.udf.api.exception.UDFArgumentNotValidException; +import org.apache.iotdb.udf.api.exception.UDFException; +import org.apache.iotdb.udf.api.relational.TableFunction; +import org.apache.iotdb.udf.api.relational.table.TableFunctionAnalysis; +import org.apache.iotdb.udf.api.relational.table.TableFunctionHandle; +import org.apache.iotdb.udf.api.relational.table.TableFunctionProcessorProvider; +import org.apache.iotdb.udf.api.relational.table.argument.Argument; +import org.apache.iotdb.udf.api.relational.table.argument.DescribedSchema; +import org.apache.iotdb.udf.api.relational.table.argument.ScalarArgument; +import org.apache.iotdb.udf.api.relational.table.specification.ParameterSpecification; +import org.apache.iotdb.udf.api.relational.table.specification.ScalarParameterSpecification; +import org.apache.iotdb.udf.api.type.Type; + +import org.apache.tsfile.file.metadata.TableSchema; +import org.apache.tsfile.write.schema.IMeasurementSchema; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +/** Reads one or more TsFiles as a table function source. */ +public class ReadTsFileTableFunction implements TableFunction { + private static final String TABLE_NAME_PARAMETER_NAME = "TABLE_NAME"; + private static final String PATHS_PARAMETER_NAME = "PATHS"; + + private MPPQueryContext mppQueryContext; + + public void setMPPQueryContext(MPPQueryContext mppQueryContext) { + this.mppQueryContext = mppQueryContext; + } + + @Override + public List getArgumentsSpecifications() { + return Arrays.asList( + ScalarParameterSpecification.builder().name(PATHS_PARAMETER_NAME).type(Type.STRING).build(), + ScalarParameterSpecification.builder() + .name(TABLE_NAME_PARAMETER_NAME) + .type(Type.STRING) + .defaultValue("") + .build()); + } + + @Override + public TableFunctionAnalysis analyze(Map arguments) throws UDFException { + String tableName = getOptionalStringArgument(arguments, TABLE_NAME_PARAMETER_NAME); + List tsFilePaths = + parseTsFilePaths(getRequiredStringArgument(arguments, PATHS_PARAMETER_NAME)); + checkTsFilePathsAreOutsideDataDirs(tsFilePaths); + TsFileSchemaCollector schemaCollector = + new TsFileSchemaCollector(tableName.isEmpty() ? null : tableName, mppQueryContext); + schemaCollector.collect(tsFilePaths); + TableSchema mergedTableSchema = schemaCollector.getMergedTableSchema(); + if (mergedTableSchema == null) { + throw new UDFArgumentNotValidException( + tableName.isEmpty() + ? DataNodeQueryMessages.NO_TABLE_SCHEMA_FOUND_IN_TSFILES + : String.format( + DataNodeQueryMessages.NO_TABLE_SCHEMA_FOUND_FOR_TABLE_IN_TSFILES, tableName)); + } + DescribedSchema outputSchema = convertToDescribedSchema(mergedTableSchema); + + ReadTsFileTableFunctionHandle handle = + new ReadTsFileTableFunctionHandle( + schemaCollector.getTableName(), + schemaCollector.getTsFiles().stream() + .map(File::getAbsolutePath) + .collect(Collectors.toList()), + mergedTableSchema.getColumnTypes().stream() + .map(TsTableColumnCategory::fromTsFileColumnCategory) + .collect(Collectors.toList()), + outputSchema); + + return TableFunctionAnalysis.builder().properColumnSchema(outputSchema).handle(handle).build(); + } + + @Override + public TableFunctionHandle createTableFunctionHandle() { + return new ReadTsFileTableFunctionHandle(); + } + + @Override + public TableFunctionProcessorProvider getProcessorProvider( + TableFunctionHandle tableFunctionHandle) { + throw new UnsupportedOperationException( + DataNodeQueryMessages.READ_TSFILE_MUST_BE_PLANNED_AS_EXTERNAL_TSFILE_SCAN_NODE); + } + + private static String getRequiredStringArgument(Map arguments, String name) { + Argument argument = arguments.get(name); + if (!(argument instanceof ScalarArgument)) { + throw new UDFArgumentNotValidException(DataNodeQueryMessages.MISSING_SCALAR_ARGUMENT + name); + } + Object value = ((ScalarArgument) argument).getValue(); + if (!(value instanceof String) || ((String) value).trim().isEmpty()) { + throw new UDFArgumentNotValidException( + String.format(DataNodeQueryMessages.ARGUMENT_SHOULD_NOT_BE_EMPTY, name)); + } + return ((String) value).trim(); + } + + private static String getOptionalStringArgument(Map arguments, String name) { + Argument argument = arguments.get(name); + if (argument == null) { + return ""; + } + if (!(argument instanceof ScalarArgument)) { + throw new UDFArgumentNotValidException(DataNodeQueryMessages.INVALID_SCALAR_ARGUMENT + name); + } + Object value = ((ScalarArgument) argument).getValue(); + if (!(value instanceof String)) { + throw new UDFArgumentNotValidException( + String.format(DataNodeQueryMessages.ARGUMENT_SHOULD_BE_A_STRING, name)); + } + return ((String) value).trim(); + } + + private static List parseTsFilePaths(String tsFilePaths) { + List paths = + Arrays.stream(tsFilePaths.split(",")) + .map(String::trim) + .filter(path -> !path.isEmpty()) + .collect(Collectors.toList()); + if (paths.isEmpty()) { + throw new UDFArgumentNotValidException( + String.format( + DataNodeQueryMessages.ARGUMENT_SHOULD_CONTAIN_AT_LEAST_ONE_PATH, + PATHS_PARAMETER_NAME)); + } + return paths; + } + + private static void checkTsFilePathsAreOutsideDataDirs(List tsFilePaths) { + List dataDirs = + Arrays.stream(IoTDBDescriptor.getInstance().getConfig().getDataDirs()) + .map(ReadTsFileTableFunction::normalizePath) + .collect(Collectors.toList()); + for (String tsFilePath : tsFilePaths) { + Path normalizedTsFilePath = normalizePath(tsFilePath); + for (Path dataDir : dataDirs) { + if (normalizedTsFilePath.startsWith(dataDir) || dataDir.startsWith(normalizedTsFilePath)) { + throw new UDFArgumentNotValidException( + String.format( + DataNodeQueryMessages.READ_TSFILE_PATH_IS_NOT_ALLOWED, tsFilePath, dataDir)); + } + } + } + } + + private static Path normalizePath(String path) { + Path normalizedPath = Paths.get(path).toAbsolutePath().normalize(); + try { + return normalizedPath.toRealPath(); + } catch (IOException e) { + return normalizedPath; + } + } + + private static DescribedSchema convertToDescribedSchema(TableSchema tableSchema) { + DescribedSchema.Builder builder = DescribedSchema.builder(); + for (IMeasurementSchema columnSchema : tableSchema.getColumnSchemas()) { + builder.addField( + columnSchema.getMeasurementName(), + UDFDataTypeTransformer.transformToUDFDataType(columnSchema.getType())); + } + return builder.build(); + } + + public static class ReadTsFileTableFunctionHandle implements TableFunctionHandle { + private String tableName; + private List tsFilePaths; + private List outputColumnNames; + private List outputColumnTypes; + private List outputColumnCategories; + + public ReadTsFileTableFunctionHandle() { + this( + "", + Collections.emptyList(), + Collections.emptyList(), + Collections.emptyList(), + Collections.emptyList()); + } + + public ReadTsFileTableFunctionHandle( + String tableName, + List tsFilePaths, + List outputColumnCategories, + DescribedSchema outputSchema) { + this( + tableName, + tsFilePaths, + outputSchema.getFields().stream() + .map(field -> field.getName().orElse("")) + .collect(Collectors.toList()), + outputSchema.getFields().stream() + .map(DescribedSchema.Field::getType) + .collect(Collectors.toList()), + outputColumnCategories); + } + + private ReadTsFileTableFunctionHandle( + String tableName, + List tsFilePaths, + List outputColumnNames, + List outputColumnTypes, + List outputColumnCategories) { + if (outputColumnNames.size() != outputColumnTypes.size()) { + throw new IllegalArgumentException( + DataNodeQueryMessages.OUTPUT_COLUMN_NAMES_AND_TYPES_SIZE_MISMATCH); + } + if (outputColumnNames.size() != outputColumnCategories.size()) { + throw new IllegalArgumentException( + DataNodeQueryMessages.OUTPUT_COLUMN_NAMES_AND_CATEGORIES_SIZE_MISMATCH); + } + this.tableName = tableName; + this.tsFilePaths = Collections.unmodifiableList(new ArrayList<>(tsFilePaths)); + this.outputColumnNames = Collections.unmodifiableList(new ArrayList<>(outputColumnNames)); + this.outputColumnTypes = Collections.unmodifiableList(new ArrayList<>(outputColumnTypes)); + this.outputColumnCategories = + Collections.unmodifiableList(new ArrayList<>(outputColumnCategories)); + } + + public String getTableName() { + return tableName; + } + + public List getTsFilePaths() { + return tsFilePaths; + } + + public List getOutputColumnNames() { + return outputColumnNames; + } + + public List getOutputColumnTypes() { + return outputColumnTypes; + } + + public List getOutputColumnCategories() { + return outputColumnCategories; + } + + @Override + public byte[] serialize() { + throw new UnsupportedOperationException( + DataNodeQueryMessages.READ_TSFILE_TABLE_FUNCTION_HANDLE_DOES_NOT_SUPPORT_SERIALIZATION); + } + + @Override + public void deserialize(byte[] bytes) { + throw new UnsupportedOperationException( + DataNodeQueryMessages.READ_TSFILE_TABLE_FUNCTION_HANDLE_DOES_NOT_SUPPORT_DESERIALIZATION); + } + + @Override + public String toString() { + return "ReadTsFileTableFunctionHandle{" + + "tableName='" + + tableName + + '\'' + + ", tsFilePaths=" + + tsFilePaths + + ", outputColumnNames=" + + outputColumnNames + + ", outputColumnTypes=" + + outputColumnTypes + + ", outputColumnCategories=" + + outputColumnCategories + + '}'; + } + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/read_tsfile/TsFileSchemaCollector.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/read_tsfile/TsFileSchemaCollector.java new file mode 100644 index 0000000000000..609581ebbd6f0 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/read_tsfile/TsFileSchemaCollector.java @@ -0,0 +1,348 @@ +/* + * 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.queryengine.plan.relational.function.tvf.read_tsfile; + +import org.apache.iotdb.db.i18n.DataNodeQueryMessages; +import org.apache.iotdb.db.queryengine.common.MPPQueryContext; +import org.apache.iotdb.udf.api.exception.UDFArgumentNotValidException; + +import org.apache.tsfile.common.conf.TSFileConfig; +import org.apache.tsfile.common.constant.TsFileConstant; +import org.apache.tsfile.enums.ColumnCategory; +import org.apache.tsfile.enums.TSDataType; +import org.apache.tsfile.file.metadata.TableSchema; +import org.apache.tsfile.read.TsFileSequenceReader; +import org.apache.tsfile.write.schema.IMeasurementSchema; +import org.apache.tsfile.write.schema.MeasurementSchema; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Locale; +import java.util.Map; +import java.util.stream.Stream; + +import static org.apache.iotdb.commons.schema.table.TsTable.TIME_COLUMN_NAME; + +final class TsFileSchemaCollector { + private final String specifiedTableName; + private final MPPQueryContext mppQueryContext; + private String tableName; + private final List tsFiles = new ArrayList<>(); + private MergedTableSchemaBuilder schemaBuilder; + private TableSchema mergedTableSchema; + + TsFileSchemaCollector(String specifiedTableName, MPPQueryContext mppQueryContext) { + this.specifiedTableName = + specifiedTableName == null ? null : specifiedTableName.toLowerCase(Locale.ENGLISH); + this.mppQueryContext = mppQueryContext; + this.tableName = this.specifiedTableName; + } + + void collect(List tsFilePaths) { + for (String tsFilePath : tsFilePaths) { + Path path = new File(tsFilePath).toPath(); + if (!Files.exists(path)) { + throw new UDFArgumentNotValidException( + DataNodeQueryMessages.TSFILE_PATH_DOES_NOT_EXIST + tsFilePath); + } + if (Files.isRegularFile(path)) { + checkTimeOutIfNeeded(); + TableSchema tableSchema = readTableSchema(specifiedTableName, path.toFile(), true); + collect(path.toFile(), tableSchema); + continue; + } + if (!Files.isDirectory(path)) { + throw new UDFArgumentNotValidException( + DataNodeQueryMessages.TSFILE_PATH_IS_NEITHER_A_FILE_NOR_A_DIRECTORY + tsFilePath); + } + collectFromDirectory(tsFilePath, path); + } + if (tsFiles.isEmpty()) { + throw new UDFArgumentNotValidException(DataNodeQueryMessages.NO_VALID_TSFILES_FOUND); + } + } + + String getTableName() { + return tableName; + } + + List getTsFiles() { + return tsFiles; + } + + TableSchema getMergedTableSchema() { + return mergedTableSchema; + } + + private void collectFromDirectory(String tsFilePath, Path path) { + try (Stream walkedPaths = Files.walk(path)) { + Iterator iterator = + walkedPaths.filter(Files::isRegularFile).filter(this::hasTsFileSuffix).iterator(); + while (iterator.hasNext()) { + Path filePath = iterator.next(); + checkTimeOutIfNeeded(); + TableSchema tableSchema = readTableSchema(specifiedTableName, filePath.toFile(), false); + collect(filePath.toFile(), tableSchema); + } + } catch (IOException e) { + throw new UDFArgumentNotValidException( + DataNodeQueryMessages.FAILED_TO_SCAN_TSFILE_PATH + tsFilePath); + } + } + + private boolean hasTsFileSuffix(Path filePath) { + Path fileName = filePath.getFileName(); + return fileName != null && fileName.toString().endsWith(TsFileConstant.TSFILE_SUFFIX); + } + + private void checkTimeOutIfNeeded() { + if (mppQueryContext != null) { + mppQueryContext.checkTimeOut(); + } + } + + private void collect(File tsFile, TableSchema tableSchema) { + if (tableSchema == null) { + return; + } + String currentTableName = tableSchema.getTableName().toLowerCase(Locale.ENGLISH); + if (tableName == null) { + tableName = currentTableName; + } else if (!tableName.equals(currentTableName)) { + throw new UDFArgumentNotValidException( + String.format( + DataNodeQueryMessages.CANNOT_INFER_TABLE_NAME_FROM_TSFILES_MULTIPLE_TABLES, + tableName, + currentTableName)); + } + tsFiles.add(tsFile); + if (schemaBuilder == null) { + schemaBuilder = new MergedTableSchemaBuilder(tableName, tableSchema); + } else { + schemaBuilder.merge(tableSchema); + } + mergedTableSchema = schemaBuilder.build(); + } + + private TableSchema readTableSchema( + String specifiedTableName, File tsFile, boolean failOnInvalidTsFile) { + if (!tsFile.canRead()) { + if (failOnInvalidTsFile) { + throw invalidTsFileException(tsFile); + } + return null; + } + try (TsFileSequenceReader reader = new TsFileSequenceReader(tsFile.getAbsolutePath())) { + if (!isValidTsFile(reader)) { + if (failOnInvalidTsFile) { + throw invalidTsFileException(tsFile); + } + return null; + } + return readTableSchemaFromValidTsFile(specifiedTableName, tsFile, reader); + } catch (UDFArgumentNotValidException e) { + throw e; + } catch (Exception e) { + if (failOnInvalidTsFile) { + throw invalidTsFileException(tsFile); + } + return null; + } + } + + private TableSchema readTableSchemaFromValidTsFile( + String specifiedTableName, File tsFile, TsFileSequenceReader reader) { + try { + return selectTableSchema(specifiedTableName, tsFile, reader.getTableSchemaMap()); + } catch (UDFArgumentNotValidException e) { + throw e; + } catch (Exception e) { + throw failedToReadTableSchemaException(tsFile); + } + } + + private TableSchema selectTableSchema( + String specifiedTableName, File tsFile, Map tableSchemaMap) { + try { + if (specifiedTableName != null) { + return tableSchemaMap.get(specifiedTableName.toLowerCase(Locale.ENGLISH)); + } + if (tableSchemaMap.isEmpty()) { + throw new UDFArgumentNotValidException( + DataNodeQueryMessages.CANNOT_INFER_TABLE_NAME_FROM_TSFILE_NO_TABLE_SCHEMA + + tsFile.getAbsolutePath()); + } + if (tableSchemaMap.size() > 1) { + throw new UDFArgumentNotValidException( + DataNodeQueryMessages.CANNOT_INFER_TABLE_NAME_FROM_TSFILE_MULTIPLE_TABLES + + tsFile.getAbsolutePath()); + } + return tableSchemaMap.values().iterator().next(); + } catch (UDFArgumentNotValidException e) { + throw e; + } catch (Exception e) { + throw failedToReadTableSchemaException(tsFile); + } + } + + private boolean isValidTsFile(TsFileSequenceReader reader) throws IOException { + return TSFileConfig.MAGIC_STRING.equals(reader.readHeadMagic()) + && reader.readVersionNumber() == TSFileConfig.VERSION_NUMBER + && TSFileConfig.MAGIC_STRING.equals(reader.readTailMagic()); + } + + private UDFArgumentNotValidException invalidTsFileException(File tsFile) { + return new UDFArgumentNotValidException( + DataNodeQueryMessages.FILE_IS_NOT_A_VALID_TSFILE + tsFile.getAbsolutePath()); + } + + private UDFArgumentNotValidException failedToReadTableSchemaException(File tsFile) { + return new UDFArgumentNotValidException( + DataNodeQueryMessages.FAILED_TO_READ_TABLE_SCHEMA_FROM_TSFILE + tsFile.getAbsolutePath()); + } + + private static class MergedTableSchemaBuilder { + private final String tableName; + private IMeasurementSchema timeColumnSchema; + private final List tagColumnSchemas = new ArrayList<>(); + private final Map fieldColumnSchemaMap = new LinkedHashMap<>(); + private final Map columnCategoryMap = new LinkedHashMap<>(); + + private MergedTableSchemaBuilder(String tableName, TableSchema tableSchema) { + this.tableName = tableName.toLowerCase(Locale.ENGLISH); + merge(tableSchema); + } + + private void merge(TableSchema tableSchema) { + IMeasurementSchema currentTimeColumn = null; + List currentTagColumns = new ArrayList<>(); + List currentFieldColumns = new ArrayList<>(); + List columnSchemas = tableSchema.getColumnSchemas(); + List columnCategories = tableSchema.getColumnTypes(); + + for (int i = 0; i < columnCategories.size(); i++) { + ColumnCategory currentCategory = columnCategories.get(i); + if (currentCategory == ColumnCategory.TIME) { + if (currentTimeColumn != null) { + throw new UDFArgumentNotValidException( + DataNodeQueryMessages.MULTIPLE_TIME_COLUMNS_FOUND_WHEN_MERGING_TABLE_SCHEMA + + tableName); + } + currentTimeColumn = columnSchemas.get(i); + } else if (currentCategory == ColumnCategory.TAG) { + checkAndRecordColumnCategory(columnSchemas.get(i), currentCategory); + currentTagColumns.add(columnSchemas.get(i)); + } else if (currentCategory == ColumnCategory.FIELD) { + checkAndRecordColumnCategory(columnSchemas.get(i), currentCategory); + currentFieldColumns.add(columnSchemas.get(i)); + } + } + + mergeTimeColumn(currentTimeColumn); + mergeTagColumns(currentTagColumns); + mergeFieldColumns(currentFieldColumns); + } + + private void mergeTimeColumn(IMeasurementSchema currentTimeColumn) { + if (currentTimeColumn == null) { + return; + } + if (timeColumnSchema == null) { + timeColumnSchema = currentTimeColumn; + return; + } + if (!timeColumnSchema.getMeasurementName().equals(currentTimeColumn.getMeasurementName()) + || currentTimeColumn.getType() != TSDataType.TIMESTAMP) { + throw new UDFArgumentNotValidException( + DataNodeQueryMessages.TIME_COLUMN_CONFLICTS_WHEN_MERGING_TABLE_SCHEMA + tableName); + } + } + + private void mergeTagColumns(List currentTagColumns) { + int prefixLength = Math.min(tagColumnSchemas.size(), currentTagColumns.size()); + for (int i = 0; i < prefixLength; i++) { + if (!tagColumnSchemas + .get(i) + .getMeasurementName() + .equals(currentTagColumns.get(i).getMeasurementName())) { + throw new UDFArgumentNotValidException( + DataNodeQueryMessages.TAG_COLUMNS_CONFLICT_WHEN_MERGING_TABLE_SCHEMA + tableName); + } + } + tagColumnSchemas.addAll(currentTagColumns.subList(prefixLength, currentTagColumns.size())); + } + + private void mergeFieldColumns(List currentFieldColumns) { + for (IMeasurementSchema fieldColumn : currentFieldColumns) { + String fieldName = fieldColumn.getMeasurementName().toLowerCase(Locale.ENGLISH); + IMeasurementSchema existingColumn = fieldColumnSchemaMap.get(fieldName); + if (existingColumn != null + && !existingColumn.getType().isCompatible(fieldColumn.getType())) { + throw new UDFArgumentNotValidException( + String.format( + DataNodeQueryMessages + .FIELD_COLUMN_HAS_CONFLICTING_DATA_TYPES_WHEN_MERGING_TABLE_SCHEMA, + fieldColumn.getMeasurementName(), + tableName)); + } + fieldColumnSchemaMap.putIfAbsent(fieldName, fieldColumn); + } + } + + private void checkAndRecordColumnCategory( + IMeasurementSchema columnSchema, ColumnCategory currentCategory) { + String columnName = columnSchema.getMeasurementName().toLowerCase(Locale.ENGLISH); + ColumnCategory existingCategory = columnCategoryMap.get(columnName); + if (existingCategory != null && existingCategory != currentCategory) { + throw new UDFArgumentNotValidException( + String.format( + DataNodeQueryMessages.COLUMN_HAS_CONFLICTING_CATEGORIES_WHEN_MERGING_TABLE_SCHEMA, + columnSchema.getMeasurementName(), + tableName)); + } + columnCategoryMap.putIfAbsent(columnName, currentCategory); + } + + private TableSchema build() { + List columnSchemas = new ArrayList<>(); + List columnCategories = new ArrayList<>(); + columnSchemas.add( + timeColumnSchema == null + ? new MeasurementSchema(TIME_COLUMN_NAME, TSDataType.TIMESTAMP) + : timeColumnSchema); + columnCategories.add(ColumnCategory.TIME); + for (IMeasurementSchema tagColumnSchema : tagColumnSchemas) { + columnSchemas.add(tagColumnSchema); + columnCategories.add(ColumnCategory.TAG); + } + for (IMeasurementSchema fieldColumnSchema : fieldColumnSchemaMap.values()) { + columnSchemas.add(fieldColumnSchema); + columnCategories.add(ColumnCategory.FIELD); + } + return new TableSchema(tableName, columnSchemas, columnCategories); + } + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/metadata/TableMetadataImpl.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/metadata/TableMetadataImpl.java index 2f0f12b9f8c32..4c2ed8397cd94 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/metadata/TableMetadataImpl.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/metadata/TableMetadataImpl.java @@ -27,6 +27,7 @@ import org.apache.iotdb.commons.partition.SchemaPartition; import org.apache.iotdb.commons.queryengine.common.SessionInfo; import org.apache.iotdb.commons.queryengine.plan.relational.function.OperatorType; +import org.apache.iotdb.commons.queryengine.plan.relational.function.TableFunctionFactory; import org.apache.iotdb.commons.queryengine.plan.relational.function.arithmetic.AdditionResolver; import org.apache.iotdb.commons.queryengine.plan.relational.function.arithmetic.DivisionResolver; import org.apache.iotdb.commons.queryengine.plan.relational.function.arithmetic.ModulusResolver; @@ -52,6 +53,7 @@ import org.apache.iotdb.db.queryengine.common.MPPQueryContext; import org.apache.iotdb.db.queryengine.plan.analyze.ClusterPartitionFetcher; import org.apache.iotdb.db.queryengine.plan.analyze.IPartitionFetcher; +import org.apache.iotdb.db.queryengine.plan.relational.function.DataNodeTableBuiltinTableFunction; import org.apache.iotdb.db.queryengine.plan.relational.metadata.fetcher.TableDeviceSchemaFetcher; import org.apache.iotdb.db.queryengine.plan.relational.metadata.fetcher.TableDeviceSchemaValidator; import org.apache.iotdb.db.queryengine.plan.relational.metadata.fetcher.TableHeaderSchemaValidator; @@ -62,6 +64,7 @@ import org.apache.iotdb.udf.api.customizer.parameter.FunctionArguments; import org.apache.iotdb.udf.api.relational.AggregateFunction; import org.apache.iotdb.udf.api.relational.ScalarFunction; +import org.apache.iotdb.udf.api.relational.TableFunction; import org.apache.tsfile.file.metadata.IDeviceID; import org.apache.tsfile.read.common.type.ObjectType; @@ -95,6 +98,14 @@ public class TableMetadataImpl implements Metadata { private final DataNodeTableCache tableCache = DataNodeTableCache.getInstance(); + @Override + public TableFunction getTableFunction(String functionName) { + if (DataNodeTableBuiltinTableFunction.isBuiltInTableFunction(functionName)) { + return DataNodeTableBuiltinTableFunction.getBuiltinTableFunction(functionName); + } + return TableFunctionFactory.getTableFunction(functionName); + } + @Override public boolean tableExists(final QualifiedObjectName name) { return tableCache.getTable(name.getDatabaseName(), name.getObjectName(), false) != null; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/RelationPlanner.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/RelationPlanner.java index 6acc8ca55cce2..bf14e37d6f3f9 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/RelationPlanner.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/RelationPlanner.java @@ -115,12 +115,15 @@ import org.apache.iotdb.db.queryengine.plan.relational.analyzer.Scope; import org.apache.iotdb.db.queryengine.plan.relational.analyzer.tablefunction.TableArgumentAnalysis; import org.apache.iotdb.db.queryengine.plan.relational.analyzer.tablefunction.TableFunctionInvocationAnalysis; +import org.apache.iotdb.db.queryengine.plan.relational.function.DataNodeTableBuiltinTableFunction; +import org.apache.iotdb.db.queryengine.plan.relational.function.tvf.read_tsfile.ReadTsFileTableFunction; import org.apache.iotdb.db.queryengine.plan.relational.metadata.TableMetadataImpl; import org.apache.iotdb.db.queryengine.plan.relational.metadata.TreeDeviceViewSchema; import org.apache.iotdb.db.queryengine.plan.relational.planner.ir.IrUtils; import org.apache.iotdb.db.queryengine.plan.relational.planner.ir.PredicateWithUncorrelatedScalarSubqueryReconstructor; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.CteScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.DeviceTableScanNode; +import org.apache.iotdb.db.queryengine.plan.relational.planner.node.ExternalTsFileScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.InformationSchemaTableScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.TreeDeviceViewScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.rowpattern.rowpattern.RowPatternToIrRewriter; @@ -151,6 +154,7 @@ import java.util.HashMap; import java.util.LinkedHashMap; import java.util.List; +import java.util.Locale; import java.util.Map; import java.util.Optional; import java.util.Set; @@ -1459,6 +1463,11 @@ public RelationPlan visitDelete(final Delete node, final Void context) { @Override public RelationPlan visitTableFunctionInvocation(TableFunctionInvocation node, Void context) { TableFunctionInvocationAnalysis functionAnalysis = analysis.getTableFunctionAnalysis(node); + if (DataNodeTableBuiltinTableFunction.READ_TSFILE + .getFunctionName() + .equalsIgnoreCase(functionAnalysis.getFunctionName())) { + return planExternalTsFileScan(node, functionAnalysis); + } ImmutableList.Builder sources = ImmutableList.builder(); ImmutableList.Builder sourceProperties = @@ -1590,6 +1599,66 @@ public RelationPlan visitTableFunctionInvocation(TableFunctionInvocation node, V return new RelationPlan(root, analysis.getScope(node), outputSymbols.build(), outerContext); } + private RelationPlan planExternalTsFileScan( + TableFunctionInvocation node, TableFunctionInvocationAnalysis functionAnalysis) { + if (!(functionAnalysis.getTableFunctionHandle() + instanceof ReadTsFileTableFunction.ReadTsFileTableFunctionHandle)) { + throw new IllegalStateException( + DataNodeQueryMessages.READ_TSFILE_TABLE_FUNCTION_HANDLE_IS_INVALID); + } + + ReadTsFileTableFunction.ReadTsFileTableFunctionHandle handle = + (ReadTsFileTableFunction.ReadTsFileTableFunctionHandle) + functionAnalysis.getTableFunctionHandle(); + Scope scope = analysis.getScope(node); + RelationType relationType = scope.getRelationType(); + + ImmutableList.Builder outputSymbolsBuilder = ImmutableList.builder(); + ImmutableMap.Builder assignmentsBuilder = ImmutableMap.builder(); + Map tagAndAttributeIndexMap = new HashMap<>(); + int tagIndex = 0; + for (int i = 0; i < relationType.getAllFieldCount(); i++) { + Field field = relationType.getFieldByIndex(i); + Symbol symbol = symbolAllocator.newSymbol(field); + TsTableColumnCategory columnCategory = handle.getOutputColumnCategories().get(i); + outputSymbolsBuilder.add(symbol); + assignmentsBuilder.put( + symbol, + new ColumnSchema( + field.getName().orElse(null), field.getType(), field.isHidden(), columnCategory)); + if (columnCategory == TsTableColumnCategory.TAG) { + tagAndAttributeIndexMap.put(symbol, tagIndex++); + } + } + + List outputSymbols = outputSymbolsBuilder.build(); + Map assignments = assignmentsBuilder.build(); + QualifiedObjectName qualifiedObjectName = + createExternalTsFileQualifiedObjectName(handle.getTableName()); + + ExternalTsFileScanNode scanNode = + new ExternalTsFileScanNode( + idAllocator.genPlanNodeId(), + qualifiedObjectName, + outputSymbols, + assignments, + tagAndAttributeIndexMap, + queryContext.createExternalTsFileQueryResource( + handle.getTableName(), handle.getTsFilePaths(), assignments)); + + return new RelationPlan(scanNode, scope, outputSymbols, outerContext); + } + + private QualifiedObjectName createExternalTsFileQualifiedObjectName(String tableName) { + String normalizedTableName = tableName.toLowerCase(Locale.ENGLISH); + if (normalizedTableName.indexOf('.') >= 0) { + return QualifiedObjectName.valueOf(normalizedTableName); + } + String databaseName = + sessionInfo.getDatabaseName().orElse("external").toLowerCase(Locale.ENGLISH); + return new QualifiedObjectName(databaseName, normalizedTableName); + } + private static void stayConsistent( String[] measurements, MeasurementSchema[] measurementSchemas) { int minLength = Math.min(measurements.length, measurementSchemas.length); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/distribute/TableDistributedPlanGenerator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/distribute/TableDistributedPlanGenerator.java index d31996aceba89..1f55f9bc4972c 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/distribute/TableDistributedPlanGenerator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/distribute/TableDistributedPlanGenerator.java @@ -87,6 +87,7 @@ import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanVisitor; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.WritePlanNode; import org.apache.iotdb.db.queryengine.plan.relational.analyzer.Analysis; +import org.apache.iotdb.db.queryengine.plan.relational.function.tvf.read_tsfile.ExternalTsFileQueryResource.DeviceTaskPartition; import org.apache.iotdb.db.queryengine.plan.relational.metadata.AlignedDeviceEntry; import org.apache.iotdb.db.queryengine.plan.relational.metadata.DeviceEntry; import org.apache.iotdb.db.queryengine.plan.relational.planner.SymbolAllocator; @@ -96,6 +97,8 @@ import org.apache.iotdb.db.queryengine.plan.relational.planner.node.CopyToNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.DeviceTableScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.ExplainAnalyzeNode; +import org.apache.iotdb.db.queryengine.plan.relational.planner.node.ExternalTsFileAggregationScanNode; +import org.apache.iotdb.db.queryengine.plan.relational.planner.node.ExternalTsFileScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.InformationSchemaTableScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.IntoNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.NonAlignedAggregationTreeDeviceViewScanNode; @@ -727,6 +730,124 @@ public List visitDeviceTableScan( } } + @Override + public List visitExternalTsFileScan( + final ExternalTsFileScanNode node, final PlanContext context) { + TRegionReplicaSet localRegionReplicaSet = + new TRegionReplicaSet(null, ImmutableList.of(DataNodeEndPoints.getLocalDataNodeLocation())); + node.setRegionReplicaSet(localRegionReplicaSet); + context.mostUsedRegion = node.getRegionReplicaSet(); + Optional sortPropertyContext = + context.hasSortProperty ? analyzeSortProperty(node, context) : Optional.empty(); + int partitionCount = IoTDBDescriptor.getInstance().getConfig().getDegreeOfParallelism(); + node.getExternalTsFileQueryResource() + .collectDeviceEntries( + node.getSchemaFilter(), + sortPropertyContext.map(propertyContext -> propertyContext.comparator).orElse(null), + partitionCount); + + List partitions = + node.getExternalTsFileQueryResource().getDeviceTaskPartitions(); + if (partitions.isEmpty()) { + return Collections.singletonList(node); + } + List result = new ArrayList<>(partitions.size()); + for (DeviceTaskPartition partition : partitions) { + ExternalTsFileScanNode splitNode = + new ExternalTsFileScanNode( + partition.getPlanNodeId(), + node.getQualifiedObjectName(), + node.getOutputSymbols(), + node.getAssignments(), + node.getPushDownPredicate(), + node.getPushDownLimit(), + node.getPushDownOffset(), + node.getTimePredicate().orElse(null), + node.getScanOrder(), + node.isPushLimitToEachDevice(), + node.getTagAndAttributeIndexMap(), + node.getExternalTsFileQueryResource(), + partition.getDeviceEntryIndexes(), + partition.getPartitionIndex(), + node.getSchemaFilter()); + splitNode.setRegionReplicaSet(localRegionReplicaSet); + result.add(splitNode); + } + sortPropertyContext.ifPresent( + propertyContext -> applySortProperty(node, result, propertyContext, false)); + return result; + } + + @Override + public List visitExternalTsFileAggregationScan( + ExternalTsFileAggregationScanNode node, PlanContext context) { + TRegionReplicaSet localRegionReplicaSet = + new TRegionReplicaSet(null, ImmutableList.of(DataNodeEndPoints.getLocalDataNodeLocation())); + node.setRegionReplicaSet(localRegionReplicaSet); + context.mostUsedRegion = node.getRegionReplicaSet(); + Optional sortPropertyContext = + context.hasSortProperty ? analyzeSortProperty(node, context) : Optional.empty(); + int partitionCount = IoTDBDescriptor.getInstance().getConfig().getDegreeOfParallelism(); + node.getExternalTsFileQueryResource() + .collectDeviceEntries( + node.getSchemaFilter(), + sortPropertyContext.map(propertyContext -> propertyContext.comparator).orElse(null), + partitionCount); + + List partitions = + node.getExternalTsFileQueryResource().getDeviceTaskPartitions(); + if (partitions.isEmpty()) { + return Collections.singletonList(node); + } + boolean needFinalAggregation = + node.getStep() == SINGLE && node.getGroupingKeys().isEmpty() && partitions.size() > 1; + AggregationNode finalAggregation = null; + AggregationTableScanNode partialTemplateNode = node; + if (needFinalAggregation) { + Pair splitResult = + split(node, symbolAllocator, queryId); + finalAggregation = splitResult.left; + partialTemplateNode = splitResult.right; + } + List result = new ArrayList<>(partitions.size()); + for (DeviceTaskPartition partition : partitions) { + ExternalTsFileAggregationScanNode splitNode = + new ExternalTsFileAggregationScanNode( + partition.getPlanNodeId(), + node.getQualifiedObjectName(), + partialTemplateNode.getOutputSymbols(), + node.getAssignments(), + node.getTagAndAttributeIndexMap(), + node.getScanOrder(), + node.getTimePredicate().orElse(null), + node.getPushDownPredicate(), + node.getPushDownLimit(), + node.getPushDownOffset(), + node.isPushLimitToEachDevice(), + node.containsNonAlignedDevice(), + node.getProjection(), + partialTemplateNode.getAggregations(), + partialTemplateNode.getGroupingSets(), + partialTemplateNode.getPreGroupedSymbols(), + partialTemplateNode.getStep(), + partialTemplateNode.getGroupIdSymbol(), + node.getExternalTsFileQueryResource(), + partition.getDeviceEntryIndexes(), + partition.getPartitionIndex(), + node.getSchemaFilter()); + splitNode.setRegionReplicaSet(localRegionReplicaSet); + result.add(splitNode); + } + sortPropertyContext.ifPresent( + propertyContext -> applySortProperty(node, result, propertyContext, false)); + if (needFinalAggregation) { + OrderingScheme childOrdering = nodeOrderingMap.get(result.get(0).getPlanNodeId()); + finalAggregation.setChild(mergeChildrenViaCollectOrMergeSort(childOrdering, result)); + return Collections.singletonList(finalAggregation); + } + return result; + } + private List constructDeviceTableScanByTags( final DeviceTableScanNode node, final PlanContext context) { DataPartition dataPartition = analysis.getDataPartitionInfo(); @@ -1802,18 +1923,26 @@ private void processSortProperty( final DeviceTableScanNode deviceTableScanNode, final List resultTableScanNodeList, final PlanContext context) { + Optional sortPropertyContext = + analyzeSortProperty(deviceTableScanNode, context); + if (!sortPropertyContext.isPresent()) { + return; + } + applySortProperty( + deviceTableScanNode, resultTableScanNodeList, sortPropertyContext.get(), true); + } + + private Optional analyzeSortProperty( + DeviceTableScanNode deviceTableScanNode, PlanContext context) { final List newOrderingSymbols = new ArrayList<>(); final List newSortOrders = new ArrayList<>(); final OrderingScheme expectedOrderingScheme = context.expectedOrderingScheme; boolean lastIsTimeRelated = false; + boolean scanOrderDesc = false; for (final Symbol symbol : expectedOrderingScheme.getOrderBy()) { if (timeRelatedSymbol(symbol, deviceTableScanNode)) { - if (!expectedOrderingScheme.getOrderings().get(symbol).isAscending()) { - // TODO(beyyes) move scan order judgement into logical plan optimizer - resultTableScanNodeList.forEach( - node -> ((DeviceTableScanNode) node).setScanOrder(Ordering.DESC)); - } + scanOrderDesc = !expectedOrderingScheme.getOrderings().get(symbol).isAscending(); newOrderingSymbols.add(symbol); newSortOrders.add(expectedOrderingScheme.getOrdering(symbol)); lastIsTimeRelated = true; @@ -1828,9 +1957,22 @@ private void processSortProperty( // no sort property can be pushed down into DeviceTableScanNode if (newOrderingSymbols.isEmpty()) { - return; + return Optional.empty(); } + return Optional.of( + new SortPropertyContext( + newOrderingSymbols, + newSortOrders, + createDeviceEntryComparator(deviceTableScanNode, newOrderingSymbols, newSortOrders), + lastIsTimeRelated, + scanOrderDesc)); + } + + private Comparator createDeviceEntryComparator( + DeviceTableScanNode deviceTableScanNode, + List newOrderingSymbols, + List newSortOrders) { Optional extractor = createTreeDeviceIdColumnValueExtractor(deviceTableScanNode); final List> orderingRules = new ArrayList<>(); @@ -1905,21 +2047,48 @@ private void processSortProperty( comparator = comparator.thenComparing(thenComparator); } } + return comparator; + } + private void applySortProperty( + final DeviceTableScanNode deviceTableScanNode, + final List resultTableScanNodeList, + final SortPropertyContext sortPropertyContext, + final boolean sortDeviceEntries) { + final Map tableColumnSchema; + if (deviceTableScanNode instanceof ExternalTsFileScanNode) { + tableColumnSchema = + ((ExternalTsFileScanNode) deviceTableScanNode) + .getExternalTsFileQueryResource() + .getTableColumnSchema(); + } else if (deviceTableScanNode instanceof ExternalTsFileAggregationScanNode) { + tableColumnSchema = + ((ExternalTsFileAggregationScanNode) deviceTableScanNode) + .getExternalTsFileQueryResource() + .getTableColumnSchema(); + } else { + tableColumnSchema = + analysis.getTableColumnSchema(deviceTableScanNode.getQualifiedObjectName()); + } final Optional newOrderingScheme = tableScanOrderingSchema( - analysis.getTableColumnSchema(deviceTableScanNode.getQualifiedObjectName()), + tableColumnSchema, deviceTableScanNode.getAssignments(), - newOrderingSymbols, - newSortOrders, - lastIsTimeRelated, - deviceTableScanNode.getDeviceEntries().size() == 1); + sortPropertyContext.orderingSymbols, + sortPropertyContext.sortOrders, + sortPropertyContext.lastIsTimeRelated, + resultTableScanNodeList.size() == 1 + && ((DeviceTableScanNode) resultTableScanNodeList.get(0)).getDeviceEntries().size() + == 1); for (final PlanNode planNode : resultTableScanNodeList) { final DeviceTableScanNode scanNode = (DeviceTableScanNode) planNode; + if (sortPropertyContext.scanOrderDesc) { + scanNode.setScanOrder(Ordering.DESC); + } newOrderingScheme.ifPresent( orderingScheme -> nodeOrderingMap.put(scanNode.getPlanNodeId(), orderingScheme)); - if (comparator != null) { - scanNode.getDeviceEntries().sort(comparator); + if (sortDeviceEntries && sortPropertyContext.comparator != null) { + scanNode.getDeviceEntries().sort(sortPropertyContext.comparator); } } } @@ -2239,6 +2408,27 @@ public List visitUnion(UnionNode node, PlanContext context) { node.getOutputSymbols())); } + private static class SortPropertyContext { + final List orderingSymbols; + final List sortOrders; + final Comparator comparator; + final boolean lastIsTimeRelated; + final boolean scanOrderDesc; + + private SortPropertyContext( + List orderingSymbols, + List sortOrders, + Comparator comparator, + boolean lastIsTimeRelated, + boolean scanOrderDesc) { + this.orderingSymbols = orderingSymbols; + this.sortOrders = sortOrders; + this.comparator = comparator; + this.lastIsTimeRelated = lastIsTimeRelated; + this.scanOrderDesc = scanOrderDesc; + } + } + public static class PlanContext { final Map nodeDistributionMap; boolean hasExchangeNode = false; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/iterative/rule/PruneTableScanColumns.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/iterative/rule/PruneTableScanColumns.java index bf02d62f546e8..22b89d24a5d9f 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/iterative/rule/PruneTableScanColumns.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/iterative/rule/PruneTableScanColumns.java @@ -27,6 +27,7 @@ import org.apache.iotdb.db.queryengine.plan.relational.planner.SymbolsExtractor; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.AggregationTableScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.DeviceTableScanNode; +import org.apache.iotdb.db.queryengine.plan.relational.planner.node.ExternalTsFileScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.InformationSchemaTableScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.TreeAlignedDeviceViewScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.TreeDeviceViewScanNode; @@ -157,6 +158,26 @@ public static Optional pruneColumns(TableScanNode node, Set re deviceTableScanNode.containsNonAlignedDevice(), treeDeviceViewScanNode.getTreeDBName(), treeDeviceViewScanNode.getMeasurementColumnNameMap())); + } else if (node instanceof ExternalTsFileScanNode externalTsFileScanNode) { + ExternalTsFileScanNode prunedNode = + new ExternalTsFileScanNode( + deviceTableScanNode.getPlanNodeId(), + deviceTableScanNode.getQualifiedObjectName(), + newOutputs, + newAssignments, + deviceTableScanNode.getPushDownPredicate(), + deviceTableScanNode.getPushDownLimit(), + deviceTableScanNode.getPushDownOffset(), + deviceTableScanNode.getTimePredicate().orElse(null), + deviceTableScanNode.getScanOrder(), + deviceTableScanNode.isPushLimitToEachDevice(), + deviceTableScanNode.getTagAndAttributeIndexMap(), + externalTsFileScanNode.getExternalTsFileQueryResource(), + externalTsFileScanNode.getDeviceEntryIndexes(), + externalTsFileScanNode.getDeviceTaskPartitionIndex(), + externalTsFileScanNode.getSchemaFilter()); + prunedNode.setRegionReplicaSet(deviceTableScanNode.getRegionReplicaSet()); + return Optional.of(prunedNode); } else { DeviceTableScanNode prunedNode = new DeviceTableScanNode( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/AggregationTableScanNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/AggregationTableScanNode.java index e5298d01cd34d..9402e837575fd 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/AggregationTableScanNode.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/AggregationTableScanNode.java @@ -295,6 +295,34 @@ public static AggregationTableScanNode combineAggregationAndTableScan( AggregationNode aggregationNode, ProjectNode projectNode, DeviceTableScanNode tableScanNode) { + if (tableScanNode instanceof ExternalTsFileScanNode) { + ExternalTsFileScanNode externalTsFileScanNode = (ExternalTsFileScanNode) tableScanNode; + ExternalTsFileAggregationScanNode scanNode = + new ExternalTsFileAggregationScanNode( + id, + tableScanNode.getQualifiedObjectName(), + tableScanNode.getOutputSymbols(), + tableScanNode.getAssignments(), + tableScanNode.getTagAndAttributeIndexMap(), + tableScanNode.getScanOrder(), + tableScanNode.getTimePredicate().orElse(null), + tableScanNode.getPushDownPredicate(), + tableScanNode.getPushDownLimit(), + tableScanNode.getPushDownOffset(), + tableScanNode.isPushLimitToEachDevice(), + tableScanNode.containsNonAlignedDevice(), + projectNode == null ? null : projectNode.getAssignments(), + aggregationNode.getAggregations(), + aggregationNode.getGroupingSets(), + aggregationNode.getPreGroupedSymbols(), + aggregationNode.getStep(), + aggregationNode.getGroupIdSymbol(), + externalTsFileScanNode.getExternalTsFileQueryResource(), + externalTsFileScanNode.getDeviceEntryIndexes(), + externalTsFileScanNode.getDeviceTaskPartitionIndex(), + externalTsFileScanNode.getSchemaFilter()); + return scanNode; + } if (tableScanNode instanceof TreeDeviceViewScanNode) { TreeDeviceViewScanNode treeDeviceViewScanNode = (TreeDeviceViewScanNode) tableScanNode; return new AggregationTreeDeviceViewScanNode( @@ -349,6 +377,32 @@ public static AggregationTableScanNode combineAggregationAndTableScan( ProjectNode projectNode, DeviceTableScanNode tableScanNode, AggregationNode.Step step) { + if (tableScanNode instanceof ExternalTsFileScanNode) { + ExternalTsFileScanNode externalTsFileScanNode = (ExternalTsFileScanNode) tableScanNode; + return new ExternalTsFileAggregationScanNode( + id, + tableScanNode.getQualifiedObjectName(), + tableScanNode.getOutputSymbols(), + tableScanNode.getAssignments(), + tableScanNode.getTagAndAttributeIndexMap(), + tableScanNode.getScanOrder(), + tableScanNode.getTimePredicate().orElse(null), + tableScanNode.getPushDownPredicate(), + tableScanNode.getPushDownLimit(), + tableScanNode.getPushDownOffset(), + tableScanNode.isPushLimitToEachDevice(), + tableScanNode.containsNonAlignedDevice(), + projectNode == null ? null : projectNode.getAssignments(), + aggregationNode.getAggregations(), + aggregationNode.getGroupingSets(), + aggregationNode.getPreGroupedSymbols(), + step, + aggregationNode.getGroupIdSymbol(), + externalTsFileScanNode.getExternalTsFileQueryResource(), + externalTsFileScanNode.getDeviceEntryIndexes(), + externalTsFileScanNode.getDeviceTaskPartitionIndex(), + externalTsFileScanNode.getSchemaFilter()); + } if (tableScanNode instanceof TreeDeviceViewScanNode) { TreeDeviceViewScanNode treeDeviceViewScanNode = (TreeDeviceViewScanNode) tableScanNode; return new AggregationTreeDeviceViewScanNode( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/ExternalTsFileAggregationScanNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/ExternalTsFileAggregationScanNode.java new file mode 100644 index 0000000000000..4934b7f3b3c23 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/ExternalTsFileAggregationScanNode.java @@ -0,0 +1,181 @@ +/* + * 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.queryengine.plan.relational.planner.node; + +import org.apache.iotdb.commons.queryengine.plan.planner.plan.node.IPlanVisitor; +import org.apache.iotdb.commons.queryengine.plan.planner.plan.node.PlanNodeId; +import org.apache.iotdb.commons.queryengine.plan.relational.metadata.ColumnSchema; +import org.apache.iotdb.commons.queryengine.plan.relational.metadata.QualifiedObjectName; +import org.apache.iotdb.commons.queryengine.plan.relational.planner.Assignments; +import org.apache.iotdb.commons.queryengine.plan.relational.planner.Symbol; +import org.apache.iotdb.commons.queryengine.plan.relational.planner.node.AggregationNode; +import org.apache.iotdb.commons.queryengine.plan.relational.sql.ast.Expression; +import org.apache.iotdb.commons.schema.filter.SchemaFilter; +import org.apache.iotdb.db.i18n.DataNodeQueryMessages; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanVisitor; +import org.apache.iotdb.db.queryengine.plan.relational.function.tvf.read_tsfile.ExternalTsFileQueryResource; +import org.apache.iotdb.db.queryengine.plan.relational.metadata.DeviceEntry; +import org.apache.iotdb.db.queryengine.plan.statement.component.Ordering; + +import com.google.common.collect.Lists; + +import java.io.DataOutputStream; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.List; +import java.util.Map; +import java.util.Optional; + +public class ExternalTsFileAggregationScanNode extends AggregationTableScanNode { + private final ExternalTsFileQueryResource externalTsFileQueryResource; + private List deviceEntryIndexes; + private int deviceTaskPartitionIndex = -1; + private SchemaFilter schemaFilter; + + public ExternalTsFileAggregationScanNode( + PlanNodeId id, + QualifiedObjectName qualifiedObjectName, + List outputSymbols, + Map assignments, + Map tagAndAttributeIndexMap, + Ordering scanOrder, + Expression timePredicate, + Expression pushDownPredicate, + long pushDownLimit, + long pushDownOffset, + boolean pushLimitToEachDevice, + boolean containsNonAlignedDevice, + Assignments projection, + Map aggregations, + AggregationNode.GroupingSetDescriptor groupingSets, + List preGroupedSymbols, + AggregationNode.Step step, + Optional groupIdSymbol, + ExternalTsFileQueryResource externalTsFileQueryResource, + List deviceEntryIndexes, + int deviceTaskPartitionIndex, + SchemaFilter schemaFilter) { + super( + id, + qualifiedObjectName, + outputSymbols, + assignments, + Lists.transform( + deviceEntryIndexes, externalTsFileQueryResource.getSharedDeviceEntries()::get), + tagAndAttributeIndexMap, + scanOrder, + timePredicate, + pushDownPredicate, + pushDownLimit, + pushDownOffset, + pushLimitToEachDevice, + containsNonAlignedDevice, + projection, + aggregations, + groupingSets, + preGroupedSymbols, + step, + groupIdSymbol); + this.externalTsFileQueryResource = externalTsFileQueryResource; + this.deviceEntryIndexes = deviceEntryIndexes; + this.deviceTaskPartitionIndex = deviceTaskPartitionIndex; + this.schemaFilter = schemaFilter; + } + + @Override + public R accept(IPlanVisitor visitor, C context) { + return ((PlanVisitor) visitor).visitExternalTsFileAggregationScan(this, context); + } + + @Override + public ExternalTsFileAggregationScanNode clone() { + return new ExternalTsFileAggregationScanNode( + id, + qualifiedObjectName, + outputSymbols, + assignments, + tagAndAttributeIndexMap, + scanOrder, + timePredicate, + pushDownPredicate, + pushDownLimit, + pushDownOffset, + pushLimitToEachDevice, + containsNonAlignedDevice, + projection, + aggregations, + groupingSets, + preGroupedSymbols, + step, + groupIdSymbol, + externalTsFileQueryResource, + deviceEntryIndexes, + deviceTaskPartitionIndex, + schemaFilter); + } + + public List getTsFilePaths() { + return externalTsFileQueryResource.getTsFilePaths(); + } + + public ExternalTsFileQueryResource getExternalTsFileQueryResource() { + return externalTsFileQueryResource; + } + + public List getDeviceEntryIndexes() { + return deviceEntryIndexes; + } + + public int getDeviceTaskPartitionIndex() { + return deviceTaskPartitionIndex; + } + + @Override + public void setDeviceEntries(List deviceEntries) { + throw new UnsupportedOperationException( + DataNodeQueryMessages + .EXTERNAL_TSFILE_AGGREGATION_SCAN_NODE_DEVICE_ENTRIES_MUST_BE_SET_BY_DEVICE_ENTRY_INDEXES); + } + + public SchemaFilter getSchemaFilter() { + return schemaFilter; + } + + public void setSchemaFilter(SchemaFilter schemaFilter) { + this.schemaFilter = schemaFilter; + } + + @Override + protected void serializeAttributes(ByteBuffer byteBuffer) { + throw new UnsupportedOperationException( + DataNodeQueryMessages.EXTERNAL_TSFILE_AGGREGATION_SCAN_NODE_CANNOT_BE_SERIALIZED); + } + + @Override + protected void serializeAttributes(DataOutputStream stream) throws IOException { + throw new UnsupportedOperationException( + DataNodeQueryMessages.EXTERNAL_TSFILE_AGGREGATION_SCAN_NODE_CANNOT_BE_SERIALIZED); + } + + @Override + public String toString() { + return "ExternalTsFileAggregationScanNode-" + this.getPlanNodeId(); + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/ExternalTsFileScanNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/ExternalTsFileScanNode.java new file mode 100644 index 0000000000000..160d54351e651 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/ExternalTsFileScanNode.java @@ -0,0 +1,172 @@ +/* + * 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.queryengine.plan.relational.planner.node; + +import org.apache.iotdb.commons.queryengine.plan.planner.plan.node.IPlanVisitor; +import org.apache.iotdb.commons.queryengine.plan.planner.plan.node.PlanNodeId; +import org.apache.iotdb.commons.queryengine.plan.relational.metadata.ColumnSchema; +import org.apache.iotdb.commons.queryengine.plan.relational.metadata.QualifiedObjectName; +import org.apache.iotdb.commons.queryengine.plan.relational.planner.Symbol; +import org.apache.iotdb.commons.queryengine.plan.relational.sql.ast.Expression; +import org.apache.iotdb.commons.schema.filter.SchemaFilter; +import org.apache.iotdb.db.i18n.DataNodeQueryMessages; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanVisitor; +import org.apache.iotdb.db.queryengine.plan.relational.function.tvf.read_tsfile.ExternalTsFileQueryResource; +import org.apache.iotdb.db.queryengine.plan.relational.metadata.DeviceEntry; +import org.apache.iotdb.db.queryengine.plan.statement.component.Ordering; + +import com.google.common.collect.Lists; + +import java.io.DataOutputStream; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Collections; +import java.util.List; +import java.util.Map; + +public class ExternalTsFileScanNode extends DeviceTableScanNode { + private ExternalTsFileQueryResource externalTsFileQueryResource; + private List deviceEntryIndexes = Collections.emptyList(); + private int deviceTaskPartitionIndex = -1; + private SchemaFilter schemaFilter; + + protected ExternalTsFileScanNode() {} + + public ExternalTsFileScanNode( + PlanNodeId id, + QualifiedObjectName qualifiedObjectName, + List outputSymbols, + Map assignments, + Map tagAndAttributeIndexMap, + ExternalTsFileQueryResource externalTsFileQueryResource) { + super(id, qualifiedObjectName, outputSymbols, assignments, tagAndAttributeIndexMap); + this.externalTsFileQueryResource = externalTsFileQueryResource; + } + + public ExternalTsFileScanNode( + PlanNodeId id, + QualifiedObjectName qualifiedObjectName, + List outputSymbols, + Map assignments, + Expression pushDownPredicate, + long pushDownLimit, + long pushDownOffset, + Expression timePredicate, + Ordering scanOrder, + boolean pushLimitToEachDevice, + Map tagAndAttributeIndexMap, + ExternalTsFileQueryResource externalTsFileQueryResource, + List deviceEntryIndexes, + int deviceTaskPartitionIndex, + SchemaFilter schemaFilter) { + super( + id, + qualifiedObjectName, + outputSymbols, + assignments, + Lists.transform( + deviceEntryIndexes, externalTsFileQueryResource.getSharedDeviceEntries()::get), + tagAndAttributeIndexMap, + scanOrder, + timePredicate, + pushDownPredicate, + pushDownLimit, + pushDownOffset, + pushLimitToEachDevice, + false); + this.externalTsFileQueryResource = externalTsFileQueryResource; + this.deviceEntryIndexes = deviceEntryIndexes; + this.deviceTaskPartitionIndex = deviceTaskPartitionIndex; + this.schemaFilter = schemaFilter; + } + + @Override + public R accept(IPlanVisitor visitor, C context) { + return ((PlanVisitor) visitor).visitExternalTsFileScan(this, context); + } + + @Override + public ExternalTsFileScanNode clone() { + return new ExternalTsFileScanNode( + getPlanNodeId(), + qualifiedObjectName, + outputSymbols, + assignments, + pushDownPredicate, + pushDownLimit, + pushDownOffset, + timePredicate, + scanOrder, + pushLimitToEachDevice, + tagAndAttributeIndexMap, + externalTsFileQueryResource, + deviceEntryIndexes, + deviceTaskPartitionIndex, + schemaFilter); + } + + public List getTsFilePaths() { + return externalTsFileQueryResource.getTsFilePaths(); + } + + public ExternalTsFileQueryResource getExternalTsFileQueryResource() { + return externalTsFileQueryResource; + } + + public List getDeviceEntryIndexes() { + return deviceEntryIndexes; + } + + public int getDeviceTaskPartitionIndex() { + return deviceTaskPartitionIndex; + } + + @Override + public void setDeviceEntries(List deviceEntries) { + throw new UnsupportedOperationException( + DataNodeQueryMessages + .EXTERNAL_TSFILE_SCAN_NODE_DEVICE_ENTRIES_MUST_BE_SET_BY_DEVICE_ENTRY_INDEXES); + } + + public SchemaFilter getSchemaFilter() { + return schemaFilter; + } + + public void setSchemaFilter(SchemaFilter schemaFilter) { + this.schemaFilter = schemaFilter; + } + + @Override + protected void serializeAttributes(ByteBuffer byteBuffer) { + throw new UnsupportedOperationException( + DataNodeQueryMessages.EXTERNAL_TSFILE_SCAN_NODE_CANNOT_BE_SERIALIZED); + } + + @Override + protected void serializeAttributes(DataOutputStream stream) throws IOException { + throw new UnsupportedOperationException( + DataNodeQueryMessages.EXTERNAL_TSFILE_SCAN_NODE_CANNOT_BE_SERIALIZED); + } + + @Override + public String toString() { + return "ExternalTsFileScanNode-" + this.getPlanNodeId(); + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/ParallelizeGrouping.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/ParallelizeGrouping.java index 304f930c68db4..66c4eff6e3e03 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/ParallelizeGrouping.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/ParallelizeGrouping.java @@ -38,6 +38,8 @@ import org.apache.iotdb.db.queryengine.plan.relational.analyzer.Analysis; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.AggregationTableScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.DeviceTableScanNode; +import org.apache.iotdb.db.queryengine.plan.relational.planner.node.ExternalTsFileAggregationScanNode; +import org.apache.iotdb.db.queryengine.plan.relational.planner.node.ExternalTsFileScanNode; import java.util.List; import java.util.Map; @@ -227,8 +229,20 @@ public PlanNode visitPatternRecognition(PatternRecognitionNode node, Context con public PlanNode visitDeviceTableScan(DeviceTableScanNode node, Context context) { if (!context.canSkip()) { OrderingScheme sortKey = context.sortKey; - Map tableColumnSchema = - analysis.getTableColumnSchema(node.getQualifiedObjectName()); + Map tableColumnSchema; + if (node instanceof ExternalTsFileScanNode) { + tableColumnSchema = + ((ExternalTsFileScanNode) node) + .getExternalTsFileQueryResource() + .getTableColumnSchema(); + } else if (node instanceof ExternalTsFileAggregationScanNode) { + tableColumnSchema = + ((ExternalTsFileAggregationScanNode) node) + .getExternalTsFileQueryResource() + .getTableColumnSchema(); + } else { + tableColumnSchema = analysis.getTableColumnSchema(node.getQualifiedObjectName()); + } // check there are no field in sortKey and all tags in sortKey Set tagSymbols = tableColumnSchema.entrySet().stream() diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/PushAggregationIntoTableScan.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/PushAggregationIntoTableScan.java index 30ed96dd7a973..d44ad05431da3 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/PushAggregationIntoTableScan.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/PushAggregationIntoTableScan.java @@ -21,6 +21,7 @@ import org.apache.iotdb.commons.queryengine.common.SessionInfo; import org.apache.iotdb.commons.queryengine.plan.planner.plan.node.PlanNode; +import org.apache.iotdb.commons.queryengine.plan.relational.metadata.ColumnSchema; import org.apache.iotdb.commons.queryengine.plan.relational.planner.Symbol; import org.apache.iotdb.commons.queryengine.plan.relational.planner.node.AggregationNode; import org.apache.iotdb.commons.queryengine.plan.relational.planner.node.ProjectNode; @@ -30,10 +31,13 @@ import org.apache.iotdb.commons.udf.builtin.relational.TableBuiltinScalarFunction; import org.apache.iotdb.db.queryengine.common.QueryId; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanVisitor; +import org.apache.iotdb.db.queryengine.plan.relational.analyzer.Analysis; import org.apache.iotdb.db.queryengine.plan.relational.metadata.Metadata; import org.apache.iotdb.db.queryengine.plan.relational.planner.SymbolAllocator; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.AggregationTableScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.DeviceTableScanNode; +import org.apache.iotdb.db.queryengine.plan.relational.planner.node.ExternalTsFileAggregationScanNode; +import org.apache.iotdb.db.queryengine.plan.relational.planner.node.ExternalTsFileScanNode; import com.google.common.collect.ImmutableSet; import org.apache.tsfile.utils.Pair; @@ -69,6 +73,7 @@ public PlanNode optimize(PlanNode plan, PlanOptimizer.Context context) { new Rewriter(), new Context( context.getQueryContext().getQueryId(), + context.getAnalysis(), context.getMetadata(), context.sessionInfo(), context.getSymbolAllocator())); @@ -162,7 +167,7 @@ private PushDownLevel calculatePushDownLevel( } // calculate DataSet part - boolean singleDeviceEntry = tableScanNode.getDeviceEntries().size() < 2; + boolean singleDeviceEntry = isSingleDeviceEntry(tableScanNode); if (groupingKeys.isEmpty()) { // GlobalAggregation if (singleDeviceEntry) { @@ -202,12 +207,38 @@ private PushDownLevel calculatePushDownLevel( } } + private boolean isSingleDeviceEntry(DeviceTableScanNode tableScanNode) { + if (tableScanNode instanceof ExternalTsFileScanNode + || tableScanNode instanceof ExternalTsFileAggregationScanNode) { + // External scan nodes collect device entries in distributed planning, so the logical + // optimizer cannot safely use the single-device shortcut here. + return false; + } + return tableScanNode.getDeviceEntries().size() < 2; + } + private List getTagColumnsInTableStore( DeviceTableScanNode tableScanNode, Metadata metadata, SessionInfo session) { - return Objects.requireNonNull( - metadata.getTableSchema(session, tableScanNode.getQualifiedObjectName()).orElse(null)) - .getColumns() - .stream() + Collection columnSchemas; + if (tableScanNode instanceof ExternalTsFileScanNode externalTsFileScanNode) { + columnSchemas = + externalTsFileScanNode.getExternalTsFileQueryResource().getTableColumnSchema().values(); + } else if (tableScanNode + instanceof ExternalTsFileAggregationScanNode externalTsFileAggregationScanNode) { + columnSchemas = + externalTsFileAggregationScanNode + .getExternalTsFileQueryResource() + .getTableColumnSchema() + .values(); + } else { + columnSchemas = + Objects.requireNonNull( + metadata + .getTableSchema(session, tableScanNode.getQualifiedObjectName()) + .orElse(null)) + .getColumns(); + } + return columnSchemas.stream() .filter(columnSchema -> columnSchema.getColumnCategory() == TAG) .map(columnSchema -> Symbol.of(columnSchema.getName())) .collect(Collectors.toList()); @@ -240,13 +271,19 @@ private enum PushDownLevel { private static class Context { private final QueryId queryId; + private final Analysis analysis; private final Metadata metadata; private final SessionInfo session; private final SymbolAllocator symbolAllocator; public Context( - QueryId queryId, Metadata metadata, SessionInfo session, SymbolAllocator symbolAllocator) { + QueryId queryId, + Analysis analysis, + Metadata metadata, + SessionInfo session, + SymbolAllocator symbolAllocator) { this.queryId = queryId; + this.analysis = analysis; this.metadata = metadata; this.session = session; this.symbolAllocator = symbolAllocator; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/PushLimitOffsetIntoTableScan.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/PushLimitOffsetIntoTableScan.java index 298e0616f3178..87218d9c6bb7b 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/PushLimitOffsetIntoTableScan.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/PushLimitOffsetIntoTableScan.java @@ -45,6 +45,8 @@ import org.apache.iotdb.db.queryengine.plan.relational.analyzer.Analysis; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.CteScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.DeviceTableScanNode; +import org.apache.iotdb.db.queryengine.plan.relational.planner.node.ExternalTsFileAggregationScanNode; +import org.apache.iotdb.db.queryengine.plan.relational.planner.node.ExternalTsFileScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.InformationSchemaTableScanNode; import java.util.HashSet; @@ -200,8 +202,20 @@ public PlanNode visitSort(SortNode node, Context context) { return node; } OrderingScheme orderingScheme = node.getOrderingScheme(); - Map tableColumnSchema = - analysis.getTableColumnSchema(tableScanNode.getQualifiedObjectName()); + Map tableColumnSchema; + if (tableScanNode instanceof ExternalTsFileScanNode) { + tableColumnSchema = + ((ExternalTsFileScanNode) tableScanNode) + .getExternalTsFileQueryResource() + .getTableColumnSchema(); + } else if (tableScanNode instanceof ExternalTsFileAggregationScanNode) { + tableColumnSchema = + ((ExternalTsFileAggregationScanNode) tableScanNode) + .getExternalTsFileQueryResource() + .getTableColumnSchema(); + } else { + tableColumnSchema = analysis.getTableColumnSchema(tableScanNode.getQualifiedObjectName()); + } Set sortSymbols = new HashSet<>(); for (Symbol orderBy : orderingScheme.getOrderBy()) { if (tableScanNode.isTimeColumn(orderBy)) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/PushPredicateIntoTableScan.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/PushPredicateIntoTableScan.java index aff3a1445843a..8440bb24c7b46 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/PushPredicateIntoTableScan.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/PushPredicateIntoTableScan.java @@ -47,10 +47,15 @@ import org.apache.iotdb.commons.queryengine.plan.relational.sql.ast.NullLiteral; import org.apache.iotdb.commons.queryengine.plan.relational.sql.ast.StringLiteral; import org.apache.iotdb.commons.queryengine.plan.relational.sql.ast.SymbolReference; +import org.apache.iotdb.commons.queryengine.plan.relational.type.InternalTypeManager; import org.apache.iotdb.commons.queryengine.utils.TimestampPrecisionUtils; +import org.apache.iotdb.commons.schema.filter.SchemaFilter; import org.apache.iotdb.commons.schema.table.InformationSchema; +import org.apache.iotdb.commons.schema.table.TsTable; +import org.apache.iotdb.commons.schema.table.column.TagColumnSchema; import org.apache.iotdb.db.conf.IoTDBConfig; import org.apache.iotdb.db.conf.IoTDBDescriptor; +import org.apache.iotdb.db.i18n.DataNodeQueryMessages; import org.apache.iotdb.db.queryengine.common.MPPQueryContext; import org.apache.iotdb.db.queryengine.common.QueryId; import org.apache.iotdb.db.queryengine.metric.QueryPlanCostMetricSet; @@ -61,6 +66,7 @@ import org.apache.iotdb.db.queryengine.plan.relational.analyzer.predicate.ConvertPredicateToTimeFilterVisitor; import org.apache.iotdb.db.queryengine.plan.relational.analyzer.predicate.PredicateCombineIntoTableScanChecker; import org.apache.iotdb.db.queryengine.plan.relational.analyzer.predicate.PredicatePushIntoMetadataChecker; +import org.apache.iotdb.db.queryengine.plan.relational.analyzer.predicate.schema.ConvertSchemaPredicateToFilterVisitor; import org.apache.iotdb.db.queryengine.plan.relational.metadata.DeviceEntry; import org.apache.iotdb.db.queryengine.plan.relational.metadata.Metadata; import org.apache.iotdb.db.queryengine.plan.relational.metadata.NonAlignedDeviceEntry; @@ -72,6 +78,7 @@ import org.apache.iotdb.db.queryengine.plan.relational.planner.SymbolsExtractor; import org.apache.iotdb.db.queryengine.plan.relational.planner.ir.ReplaceSymbolInExpression; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.DeviceTableScanNode; +import org.apache.iotdb.db.queryengine.plan.relational.planner.node.ExternalTsFileScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.InformationSchemaTableScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.TreeDeviceViewScanNode; @@ -117,6 +124,7 @@ import static org.apache.iotdb.commons.schema.table.InformationSchema.CURRENT_QUERIES; import static org.apache.iotdb.commons.schema.table.column.TsTableColumnCategory.ATTRIBUTE; import static org.apache.iotdb.commons.schema.table.column.TsTableColumnCategory.FIELD; +import static org.apache.iotdb.commons.schema.table.column.TsTableColumnCategory.TAG; import static org.apache.iotdb.commons.schema.table.column.TsTableColumnCategory.TIME; import static org.apache.iotdb.db.queryengine.metric.QueryPlanCostMetricSet.PARTITION_FETCHER; import static org.apache.iotdb.db.queryengine.metric.QueryPlanCostMetricSet.SCHEMA_FETCHER; @@ -471,12 +479,22 @@ public PlanNode visitDeviceTableScan( return combineFilterAndScan(tableScanNode, context.inheritedPredicate); } + @Override + public PlanNode visitExternalTsFileScan( + ExternalTsFileScanNode tableScanNode, RewriteContext context) { + if (TRUE_LITERAL.equals(context.inheritedPredicate)) { + return tableScanNode; + } + + return combineFilterAndScan(tableScanNode, context.inheritedPredicate); + } + public PlanNode combineFilterAndScan(TableScanNode tableScanNode, Expression predicate) { SplitExpression splitExpression = tableScanNode instanceof InformationSchemaTableScanNode ? splitPredicateForInformationSchemaTable( (InformationSchemaTableScanNode) tableScanNode, predicate) - : splitPredicate((DeviceTableScanNode) tableScanNode, predicate); + : splitPredicate(tableScanNode, predicate); // exist expressions can push down to scan operator if (!splitExpression.getExpressionsCanPushDown().isEmpty()) { @@ -490,7 +508,9 @@ public PlanNode combineFilterAndScan(TableScanNode tableScanNode, Expression pre Pair resultPair = extractGlobalTimeFilter(pushDownPredicate, splitExpression.getTimeColumnName()); Boolean hasValueFilter = resultPair.getRight(); - if (tableScanNode instanceof DeviceTableScanNode && resultPair.left != null) { + if (tableScanNode instanceof ExternalTsFileScanNode && resultPair.left != null) { + ((ExternalTsFileScanNode) tableScanNode).setTimePredicate(resultPair.left); + } else if (tableScanNode instanceof DeviceTableScanNode && resultPair.left != null) { ((DeviceTableScanNode) tableScanNode).setTimePredicate(resultPair.left); } if (Boolean.TRUE.equals(hasValueFilter)) { @@ -507,7 +527,11 @@ public PlanNode combineFilterAndScan(TableScanNode tableScanNode, Expression pre } // do index scan after expressionCanPushDown is processed - if (tableScanNode instanceof DeviceTableScanNode) { + if (tableScanNode instanceof ExternalTsFileScanNode externalTsFileScanNode) { + externalTsFileScanNode.setSchemaFilter( + constructExternalTsFileDeviceFilter( + externalTsFileScanNode, splitExpression.getMetadataExpressions())); + } else if (tableScanNode instanceof DeviceTableScanNode) { getDeviceEntriesWithDataPartitions( (DeviceTableScanNode) tableScanNode, splitExpression.getMetadataExpressions()); } @@ -526,6 +550,36 @@ public PlanNode combineFilterAndScan(TableScanNode tableScanNode, Expression pre return tableScanNode; } + private SchemaFilter constructExternalTsFileDeviceFilter( + ExternalTsFileScanNode tableScanNode, List metadataExpressions) { + if (metadataExpressions.isEmpty()) { + return null; + } + TsTable table = new TsTable(tableScanNode.getQualifiedObjectName().getObjectName()); + for (Map.Entry entry : tableScanNode.getAssignments().entrySet()) { + ColumnSchema columnSchema = entry.getValue(); + if (TAG.equals(columnSchema.getColumnCategory())) { + table.addColumnSchema( + new TagColumnSchema( + entry.getKey().getName(), + InternalTypeManager.getTSDataType(columnSchema.getType()))); + } + } + Expression predicate = + metadataExpressions.size() == 1 + ? metadataExpressions.get(0) + : new LogicalExpression(LogicalExpression.Operator.AND, metadataExpressions); + SchemaFilter deviceFilter = + predicate.accept( + new ConvertSchemaPredicateToFilterVisitor(), + new ConvertSchemaPredicateToFilterVisitor.Context(table)); + if (deviceFilter == null) { + throw new UnsupportedOperationException( + DataNodeQueryMessages.UNSUPPORTED_EXTERNAL_TSFILE_DEVICE_FILTER + predicate); + } + return deviceFilter; + } + interface InformationSchemaTablePredicatePushDownChecker { boolean canPushDown(Expression expression); } @@ -621,7 +675,7 @@ private SplitExpression splitPredicateForInformationSchemaTable( Collections.emptyList(), expressionsCanPushDown, expressionsCannotPushDown, null); } - private SplitExpression splitPredicate(DeviceTableScanNode node, Expression predicate) { + private SplitExpression splitPredicate(TableScanNode node, Expression predicate) { Set idOrAttributeColumnNames = new HashSet<>(node.getAssignments().size()); Set timeOrMeasurementColumnNames = new HashSet<>(node.getAssignments().size()); String timeColumnName = null; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/TransformSortToStreamSort.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/TransformSortToStreamSort.java index 4b4866aa1faab..2c40c73f72048 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/TransformSortToStreamSort.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/TransformSortToStreamSort.java @@ -35,11 +35,13 @@ import org.apache.iotdb.db.queryengine.plan.relational.planner.node.AggregationTableScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.CteScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.DeviceTableScanNode; +import org.apache.iotdb.db.queryengine.plan.relational.planner.node.ExternalTsFileAggregationScanNode; +import org.apache.iotdb.db.queryengine.plan.relational.planner.node.ExternalTsFileScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.InformationSchemaTableScanNode; import java.util.Map; -import static org.apache.iotdb.db.queryengine.plan.relational.planner.node.DeviceTableScanNode.isTimeColumn; +import static org.apache.iotdb.commons.queryengine.plan.planner.plan.node.TableScanNode.isTimeColumn; /** * Optimization phase: Logical plan planning. @@ -92,8 +94,21 @@ public PlanNode visitSort(SortNode node, Context context) { context.setCanTransform(false); DeviceTableScanNode deviceTableScanNode = context.getTableScanNode(); - Map tableColumnSchema = - analysis.getTableColumnSchema(deviceTableScanNode.getQualifiedObjectName()); + Map tableColumnSchema; + if (deviceTableScanNode instanceof ExternalTsFileScanNode) { + tableColumnSchema = + ((ExternalTsFileScanNode) deviceTableScanNode) + .getExternalTsFileQueryResource() + .getTableColumnSchema(); + } else if (deviceTableScanNode instanceof ExternalTsFileAggregationScanNode) { + tableColumnSchema = + ((ExternalTsFileAggregationScanNode) deviceTableScanNode) + .getExternalTsFileQueryResource() + .getTableColumnSchema(); + } else { + tableColumnSchema = + analysis.getTableColumnSchema(deviceTableScanNode.getQualifiedObjectName()); + } OrderingScheme orderingScheme = node.getOrderingScheme(); int streamSortIndex = -1; @@ -144,6 +159,12 @@ public PlanNode visitDeviceTableScan(DeviceTableScanNode node, Context context) return node; } + @Override + public PlanNode visitExternalTsFileScan(ExternalTsFileScanNode node, Context context) { + context.setTableScanNode(node); + return node; + } + @Override public PlanNode visitInformationSchemaTableScan( InformationSchemaTableScanNode node, Context context) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/UnaliasSymbolReferences.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/UnaliasSymbolReferences.java index 58fd2dbd61349..e3f8ffa3ea2b5 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/UnaliasSymbolReferences.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/UnaliasSymbolReferences.java @@ -68,6 +68,7 @@ import org.apache.iotdb.db.queryengine.plan.relational.planner.node.CteScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.DeviceTableScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.ExplainAnalyzeNode; +import org.apache.iotdb.db.queryengine.plan.relational.planner.node.ExternalTsFileScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.InformationSchemaTableScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.IntoNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.TreeDeviceViewScanNode; @@ -272,6 +273,43 @@ public PlanAndMappings visitInformationSchemaTableScan( mapping); } + @Override + public PlanAndMappings visitExternalTsFileScan( + ExternalTsFileScanNode node, UnaliasContext context) { + Map mapping = new HashMap<>(context.getCorrelationMapping()); + SymbolMapper mapper = symbolMapper(mapping); + + List newOutputs = mapper.map(node.getOutputSymbols()); + + Map newAssignments = new HashMap<>(); + node.getAssignments() + .forEach( + (symbol, handle) -> { + Symbol newSymbol = mapper.map(symbol); + newAssignments.put(newSymbol, handle); + }); + + ExternalTsFileScanNode rewrittenNode = + new ExternalTsFileScanNode( + node.getPlanNodeId(), + node.getQualifiedObjectName(), + newOutputs, + newAssignments, + node.getPushDownPredicate() == null ? null : mapper.map(node.getPushDownPredicate()), + node.getPushDownLimit(), + node.getPushDownOffset(), + node.getTimePredicate().map(mapper::map).orElse(null), + node.getScanOrder(), + node.isPushLimitToEachDevice(), + node.getTagAndAttributeIndexMap(), + node.getExternalTsFileQueryResource(), + node.getDeviceEntryIndexes(), + node.getDeviceTaskPartitionIndex(), + node.getSchemaFilter()); + rewrittenNode.setRegionReplicaSet(node.getRegionReplicaSet()); + return new PlanAndMappings(rewrittenNode, mapping); + } + @Override public PlanAndMappings visitCteScan(CteScanNode node, UnaliasContext context) { Map mapping = new HashMap<>(context.getCorrelationMapping()); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/buffer/BloomFilterCache.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/buffer/BloomFilterCache.java index 380583efbdb9f..eb0ae5554a57c 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/buffer/BloomFilterCache.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/buffer/BloomFilterCache.java @@ -101,9 +101,20 @@ public BloomFilter get( LongConsumer cacheHitAdder, LongConsumer cacheMissAdder) throws IOException { - BloomFilterLoader loader = new BloomFilterLoader(ioSizeRecorder); + return get(key, debug, ioSizeRecorder, cacheHitAdder, cacheMissAdder, false); + } + + public BloomFilter get( + BloomFilterCacheKey key, + boolean debug, + LongConsumer ioSizeRecorder, + LongConsumer cacheHitAdder, + LongConsumer cacheMissAdder, + boolean externalTsFile) + throws IOException { + BloomFilterLoader loader = new BloomFilterLoader(ioSizeRecorder, externalTsFile); try { - if (!CACHE_ENABLE) { + if (!CACHE_ENABLE || externalTsFile) { return loader.apply(key); } @@ -203,9 +214,15 @@ private static class BloomFilterLoader implements Function { private boolean cacheMiss = false; private final LongConsumer ioSizeRecorder; + private final boolean externalTsFile; - private ChunkLoader(LongConsumer ioSizeRecorder) { + private ChunkLoader(LongConsumer ioSizeRecorder, boolean externalTsFile) { this.ioSizeRecorder = ioSizeRecorder; + this.externalTsFile = externalTsFile; } @Override @@ -310,7 +315,7 @@ public Chunk apply(ChunkCacheKey key) { cacheMiss = true; TsFileSequenceReader reader = FileReaderManager.getInstance() - .get(key.getFilePath(), key.tsFileID, key.closed, ioSizeRecorder); + .get(key.getFilePath(), key.tsFileID, key.closed, ioSizeRecorder, externalTsFile); Chunk chunk = reader.readMemChunk(key.offsetOfChunkHeader, ioSizeRecorder); // to save memory footprint, we don't save measurementId in ChunkHeader of Chunk chunk.getHeader().setMeasurementID(null); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/buffer/TimeSeriesMetadataCache.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/buffer/TimeSeriesMetadataCache.java index 3ee21d911d00b..255cce7c738d2 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/buffer/TimeSeriesMetadataCache.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/buffer/TimeSeriesMetadataCache.java @@ -145,6 +145,19 @@ public TimeseriesMetadata get( boolean debug, QueryContext queryContext) throws IOException { + return get(filePath, key, allSensors, ignoreNotExists, debug, queryContext, null); + } + + @SuppressWarnings({"squid:S1860", "squid:S6541", "squid:S3776"}) // Suppress synchronize warning + public TimeseriesMetadata get( + String filePath, + TimeSeriesMetadataCacheKey key, + Set allSensors, + boolean ignoreNotExists, + boolean debug, + QueryContext queryContext, + long[] deviceMetadataIndexNodeOffset) + throws IOException { long startTime = System.nanoTime(); long loadBloomFilterTime = 0; LongConsumer timeSeriesMetadataIoSizeRecorder = @@ -152,28 +165,35 @@ public TimeseriesMetadata get( LongConsumer bloomFilterIoSizeRecorder = queryContext.getQueryStatistics().getLoadBloomFilterActualIOSize()::addAndGet; boolean cacheHit = true; + boolean externalTsFile = queryContext.isExternalTsFileScan(); try { - if (!CACHE_ENABLE) { + if (!CACHE_ENABLE || externalTsFile) { String deviceStringFormat = key.device.toString(); cacheHit = false; // bloom filter part TsFileSequenceReader reader = FileReaderManager.getInstance() - .get(filePath, key.tsFileID, true, bloomFilterIoSizeRecorder); - BloomFilter bloomFilter = reader.readBloomFilter(bloomFilterIoSizeRecorder); - queryContext.getQueryStatistics().getLoadBloomFilterFromDiskCount().incrementAndGet(); - if (bloomFilter != null - && !bloomFilter.contains( - deviceStringFormat + IoTDBConstant.PATH_SEPARATOR + key.measurement)) { + .get(filePath, key.tsFileID, true, bloomFilterIoSizeRecorder, externalTsFile); + if (deviceMetadataIndexNodeOffset == null) { + BloomFilter bloomFilter = reader.readBloomFilter(bloomFilterIoSizeRecorder); + queryContext.getQueryStatistics().getLoadBloomFilterFromDiskCount().incrementAndGet(); + if (bloomFilter != null + && !bloomFilter.contains( + deviceStringFormat + IoTDBConstant.PATH_SEPARATOR + key.measurement)) { + loadBloomFilterTime = System.nanoTime() - startTime; + return null; + } loadBloomFilterTime = System.nanoTime() - startTime; - return null; } - loadBloomFilterTime = System.nanoTime() - startTime; TimeseriesMetadata timeseriesMetadata = reader.readTimeseriesMetadata( - key.device, key.measurement, ignoreNotExists, timeSeriesMetadataIoSizeRecorder); + key.device, + deviceMetadataIndexNodeOffset, + key.measurement, + ignoreNotExists, + timeSeriesMetadataIoSizeRecorder); return (timeseriesMetadata == null || timeseriesMetadata.getStatistics().getCount() == 0) ? null : timeseriesMetadata; @@ -197,35 +217,40 @@ public TimeseriesMetadata get( if (timeseriesMetadata == null) { cacheHit = false; - long loadBloomFilterStartTime = System.nanoTime(); - // bloom filter part - BloomFilter bloomFilter = - BloomFilterCache.getInstance() - .get( - new BloomFilterCache.BloomFilterCacheKey(filePath, key.tsFileID), - debug, - bloomFilterIoSizeRecorder, - queryContext.getQueryStatistics().getLoadBloomFilterFromCacheCount() - ::addAndGet, - queryContext.getQueryStatistics().getLoadBloomFilterFromDiskCount() - ::addAndGet); - if (bloomFilter != null - && !bloomFilter.contains( - deviceStringFormat + TsFileConstant.PATH_SEPARATOR + key.measurement)) { - if (debug) { - DEBUG_LOGGER.info(StorageEngineMessages.TS_METADATA_FILTERED_BY_BLOOM_FILTER, key); + if (deviceMetadataIndexNodeOffset == null) { + long loadBloomFilterStartTime = System.nanoTime(); + // bloom filter part + BloomFilter bloomFilter = + BloomFilterCache.getInstance() + .get( + new BloomFilterCache.BloomFilterCacheKey(filePath, key.tsFileID), + debug, + bloomFilterIoSizeRecorder, + queryContext.getQueryStatistics().getLoadBloomFilterFromCacheCount() + ::addAndGet, + queryContext.getQueryStatistics().getLoadBloomFilterFromDiskCount() + ::addAndGet, + false); + if (bloomFilter != null + && !bloomFilter.contains( + deviceStringFormat + TsFileConstant.PATH_SEPARATOR + key.measurement)) { + if (debug) { + DEBUG_LOGGER.info( + StorageEngineMessages.TS_METADATA_FILTERED_BY_BLOOM_FILTER, key); + } + loadBloomFilterTime = System.nanoTime() - loadBloomFilterStartTime; + return null; } + loadBloomFilterTime = System.nanoTime() - loadBloomFilterStartTime; - return null; } - - loadBloomFilterTime = System.nanoTime() - loadBloomFilterStartTime; TsFileSequenceReader reader = FileReaderManager.getInstance() - .get(filePath, key.tsFileID, true, timeSeriesMetadataIoSizeRecorder); + .get(filePath, key.tsFileID, true, timeSeriesMetadataIoSizeRecorder, false); List timeSeriesMetadataList = reader.readTimeseriesMetadata( key.device, + deviceMetadataIndexNodeOffset, key.measurement, allSensors, ignoreNotExists, diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/QueryDataSourceType.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/QueryDataSourceType.java index d80ce27ad7a1e..1214fd2b89ca0 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/QueryDataSourceType.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/QueryDataSourceType.java @@ -21,5 +21,6 @@ public enum QueryDataSourceType { SERIES_SCAN, DEVICE_REGION_SCAN, - TIME_SERIES_REGION_SCAN + TIME_SERIES_REGION_SCAN, + EXTERNAL_TSFILE_SCAN } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/control/FileReaderManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/control/FileReaderManager.java index 3952c7480a59d..d2dac0b775bc6 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/control/FileReaderManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/control/FileReaderManager.java @@ -80,11 +80,19 @@ public class FileReaderManager { */ private Map unclosedReferenceMap; + /** External TsFile readers. The key is the file path. */ + private Map externalFileReaderMap; + + /** Reference count of external TsFile readers. */ + private Map externalReferenceMap; + private FileReaderManager() { closedFileReaderMap = new ConcurrentHashMap<>(); unclosedFileReaderMap = new ConcurrentHashMap<>(); closedReferenceMap = new ConcurrentHashMap<>(); unclosedReferenceMap = new ConcurrentHashMap<>(); + externalFileReaderMap = new ConcurrentHashMap<>(); + externalReferenceMap = new ConcurrentHashMap<>(); } public static FileReaderManager getInstance() { @@ -137,6 +145,20 @@ public synchronized TsFileSequenceReader get(String filePath, TsFileID tsFileID, public synchronized TsFileSequenceReader get( String filePath, TsFileID tsFileID, boolean isClosed, LongConsumer ioSizeRecorder) throws IOException { + return get(filePath, tsFileID, isClosed, ioSizeRecorder, false); + } + + @SuppressWarnings("squid:S2095") + public synchronized TsFileSequenceReader get( + String filePath, + TsFileID tsFileID, + boolean isClosed, + LongConsumer ioSizeRecorder, + boolean isExternalTsFile) + throws IOException { + if (isExternalTsFile) { + return getExternalTsFileReader(filePath, ioSizeRecorder); + } Map readerMap = !isClosed ? unclosedFileReaderMap : closedFileReaderMap; @@ -170,6 +192,25 @@ public synchronized TsFileSequenceReader get( return readerMap.get(tsFileID); } + private TsFileSequenceReader getExternalTsFileReader(String filePath, LongConsumer ioSizeRecorder) + throws IOException { + TsFileSequenceReader reader = externalFileReaderMap.get(filePath); + if (reader == null) { + int currentOpenedReaderCount = externalFileReaderMap.size(); + if (currentOpenedReaderCount >= MAX_CACHED_FILE_SIZE + && (currentOpenedReaderCount % PRINT_INTERVAL == 0)) { + logger.warn(StorageEngineMessages.QUERY_OPENED_FILES, externalFileReaderMap.size()); + } + reader = + new TsFileSequenceReader( + filePath, + ioSizeRecorder, + EncryptDBUtils.getFirstEncryptParamFromTSFilePath(filePath)); + externalFileReaderMap.put(filePath, reader); + } + return reader; + } + /** * Increase the reference count of the reader specified by filePath. Only when the reference count * of a reader equals zero, the reader can be closed and removed. @@ -189,6 +230,10 @@ public void increaseFileReaderReference(TsFileResource tsFile, boolean isClosed) } } + public synchronized void increaseExternalFileReaderReference(String filePath) { + externalReferenceMap.computeIfAbsent(filePath, k -> new AtomicInteger()).getAndIncrement(); + } + /** * Decrease the reference count of the reader specified by filePath. This method is latch-free. * Only when the reference count of a reader equals zero, the reader can be closed and removed. @@ -207,6 +252,38 @@ public void decreaseFileReaderReference(TsFileResource tsFile, boolean isClosed) tsFile.readUnlock(); } + public synchronized void decreaseExternalFileReaderReference(String filePath) { + AtomicInteger reference = externalReferenceMap.get(filePath); + if (reference != null && reference.decrementAndGet() == 0) { + closeUnUsedExternalReaderAndRemoveRef(filePath); + } + } + + private void closeUnUsedExternalReaderAndRemoveRef(String readerKey) { + synchronized (this) { + AtomicInteger reference = externalReferenceMap.get(readerKey); + if (reference != null && reference.get() != 0) { + return; + } + + TsFileSequenceReader reader = externalFileReaderMap.get(readerKey); + if (reader != null) { + try { + reader.close(); + } catch (IOException e) { + logger.error( + StorageEngineMessages.CANNOT_CLOSE_TSFILE_SEQUENCE_READER, reader.getFileName(), e); + } + } + externalFileReaderMap.remove(readerKey); + externalReferenceMap.remove(readerKey); + if (resourceLogger.isDebugEnabled()) { + resourceLogger.debug( + "{} externalTsFileReader is closed because of no reference.", readerKey); + } + } + } + private void closeUnUsedReaderAndRemoveRef( String tsFilePath, TsFileID tsFileID, boolean isClosed) { Map readerMap = @@ -263,6 +340,17 @@ public synchronized void closeAndRemoveAllOpenedReaders() throws IOException { unclosedReferenceMap.remove(entry.getKey()); iterator.remove(); } + Iterator> externalIterator = + externalFileReaderMap.entrySet().iterator(); + while (externalIterator.hasNext()) { + Map.Entry entry = externalIterator.next(); + entry.getValue().close(); + if (resourceLogger.isDebugEnabled()) { + resourceLogger.debug("{} externalTsFileReader is closed.", entry.getKey()); + } + externalReferenceMap.remove(entry.getKey()); + externalIterator.remove(); + } } /** This method is only for unit tests. */ diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/read_tsfile/ExternalTsFileQueryResourceTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/read_tsfile/ExternalTsFileQueryResourceTest.java new file mode 100644 index 0000000000000..e1ba2ae05b824 --- /dev/null +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/function/tvf/read_tsfile/ExternalTsFileQueryResourceTest.java @@ -0,0 +1,204 @@ +/* + * 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.queryengine.plan.relational.function.tvf.read_tsfile; + +import org.apache.iotdb.db.queryengine.common.MPPQueryContext; +import org.apache.iotdb.db.queryengine.common.QueryId; +import org.apache.iotdb.db.queryengine.plan.relational.function.tvf.read_tsfile.ExternalTsFileQueryResource.DeviceOffset; +import org.apache.iotdb.db.queryengine.plan.relational.function.tvf.read_tsfile.ExternalTsFileQueryResource.DeviceTask; +import org.apache.iotdb.db.queryengine.plan.relational.function.tvf.read_tsfile.ExternalTsFileQueryResource.DeviceTaskPartition; +import org.apache.iotdb.db.queryengine.plan.relational.function.tvf.read_tsfile.ExternalTsFileQueryResource.DeviceTaskRunReader; +import org.apache.iotdb.db.queryengine.plan.relational.metadata.AlignedDeviceEntry; +import org.apache.iotdb.db.queryengine.plan.relational.metadata.DeviceEntry; +import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; + +import org.apache.tsfile.file.metadata.IDeviceID; +import org.apache.tsfile.utils.Binary; +import org.junit.After; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import java.io.File; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.Comparator; +import java.util.List; +import java.util.Map; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +public class ExternalTsFileQueryResourceTest { + + @Rule public TemporaryFolder temporaryFolder = new TemporaryFolder(); + + private ExternalTsFileQueryResource resource; + + @After + public void tearDown() { + if (resource != null) { + resource.closeByQueryExecution(); + } + } + + @Test + public void testDeviceTaskRunReaderMergesMultipleRunsWithComparator() throws Exception { + resource = newResource("merge_comparator", Arrays.asList("file-0.tsfile", "file-1.tsfile")); + addDevices("d1", "d2", "d3", "d4", "d5"); + resource.setDeviceEntryComparator(Comparator.comparing(DeviceEntry::getDeviceID)); + DeviceTaskPartition partition = newPartition(); + + partition.add(task(2, offset(0, 30, 39))); + partition.add(task(4, offset(1, 50, 59))); + partition.flush(); + partition.add(task(0, offset(0, 10, 19))); + partition.add(task(3, offset(1, 40, 49))); + partition.flush(); + partition.add(task(1, offset(0, 20, 29))); + partition.flush(); + + try (DeviceTaskRunReader reader = newReader(partition)) { + assertDeviceOrder(reader, "d1", "d2", "d3", "d4", "d5"); + } + } + + @Test + public void testDeviceTaskRunReaderReadsRunsInFifoOrderWithoutComparator() throws Exception { + resource = newResource("fifo", Collections.singletonList("file-0.tsfile")); + addDevices("d1", "d2", "d3", "d4"); + DeviceTaskPartition partition = newPartition(); + + partition.add(task(2, offset(0, 30, 39))); + partition.flush(); + partition.add(task(0, offset(0, 10, 19))); + partition.add(task(1, offset(0, 20, 29))); + partition.flush(); + partition.add(task(3, offset(0, 40, 49))); + partition.flush(); + + try (DeviceTaskRunReader reader = newReader(partition)) { + assertDeviceOrder(reader, "d3", "d1", "d2", "d4"); + } + } + + @Test + public void testDeviceTaskRunReaderReadsDiskRunBeforePendingMemoryTasksWithoutComparator() + throws Exception { + resource = newResource("disk_memory", Collections.singletonList("file-0.tsfile")); + addDevices("d1", "d2", "d3"); + DeviceTaskPartition partition = newPartition(); + + partition.add(task(1, offset(0, 20, 29))); + partition.flush(); + partition.add(task(0, offset(0, 10, 19))); + partition.add(task(2, offset(0, 30, 39))); + partition.finish(); + + try (DeviceTaskRunReader reader = newReader(partition)) { + assertDeviceOrder(reader, "d2", "d1", "d3"); + } + } + + @Test + public void testDeviceTaskRunReaderUsesSharedTsFileResourceAsOffsetMapKey() throws Exception { + resource = newResource("offset_map", Arrays.asList("file-0.tsfile", "file-1.tsfile")); + addDevices("d1"); + DeviceTaskPartition partition = newPartition(); + partition.add(task(0, offset(0, 11, 22), offset(1, 33, 44))); + partition.finish(); + + try (DeviceTaskRunReader reader = newReader(partition)) { + assertTrue(reader.nextDevice()); + Map offsetMap = reader.getCurrentDeviceOffsetMap(); + List sharedResources = resource.getSharedTsFileResources(); + + assertEquals(2, offsetMap.size()); + assertTrue(offsetMap.containsKey(sharedResources.get(0))); + assertTrue(offsetMap.containsKey(sharedResources.get(1))); + assertOffset(offsetMap.get(sharedResources.get(0)), 0, 11, 22); + assertOffset(offsetMap.get(sharedResources.get(1)), 1, 33, 44); + assertEquals(sharedResources, reader.getCurrentDeviceQueryDataSource().getUnseqResources()); + assertFalse(reader.nextDevice()); + } + } + + private ExternalTsFileQueryResource newResource(String queryId, List fileNames) + throws Exception { + File root = temporaryFolder.newFolder(queryId); + List tsFilePaths = new ArrayList<>(fileNames.size()); + for (String fileName : fileNames) { + tsFilePaths.add(new File(root, fileName).getAbsolutePath()); + } + MPPQueryContext queryContext = new MPPQueryContext(new QueryId(queryId)); + queryContext.setStartTime(System.currentTimeMillis()); + queryContext.setTimeOut(Long.MAX_VALUE); + return new ExternalTsFileQueryResource( + queryContext, root.toPath().resolve("tmp"), "table1", tsFilePaths, Collections.emptyMap()); + } + + private DeviceTaskRunReader newReader(DeviceTaskPartition partition) { + resource.getDeviceTaskPartitions().add(partition); + return resource.getDeviceTaskRunReader(partition.getPartitionIndex()); + } + + private DeviceTaskPartition newPartition() { + return resource.new DeviceTaskPartition(0); + } + + private void addDevices(String... deviceNames) { + for (String deviceName : deviceNames) { + resource + .getSharedDeviceEntries() + .add( + new AlignedDeviceEntry( + IDeviceID.Factory.DEFAULT_FACTORY.create(deviceName), new Binary[0])); + } + } + + private DeviceTask task(int deviceEntryIndex, DeviceOffset... offsets) { + return new DeviceTask(deviceEntryIndex, Arrays.asList(offsets)); + } + + private DeviceOffset offset(int fileIndex, long startOffset, long endOffset) { + return new DeviceOffset(fileIndex, startOffset, endOffset); + } + + private void assertDeviceOrder(DeviceTaskRunReader reader, String... expectedDeviceNames) + throws Exception { + for (String expectedDeviceName : expectedDeviceNames) { + assertTrue(reader.nextDevice()); + assertEquals(expectedDeviceName, reader.getCurrentDevice().getDeviceID().toString()); + } + assertFalse(reader.nextDevice()); + } + + private void assertOffset( + DeviceOffset offset, + int expectedFileIndex, + long expectedStartOffset, + long expectedEndOffset) { + assertEquals(expectedFileIndex, offset.getFileIndex()); + assertEquals(expectedStartOffset, offset.getStartOffset()); + assertEquals(expectedEndOffset, offset.getEndOffset()); + } +} diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/buffer/TimeSeriesMetadataCacheTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/buffer/TimeSeriesMetadataCacheTest.java index 0c368b41cca94..751eae090aec6 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/buffer/TimeSeriesMetadataCacheTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/buffer/TimeSeriesMetadataCacheTest.java @@ -243,6 +243,7 @@ public void testIgnoreNotExistsResultIsNotCached() throws Exception { Mockito.when( reader.readTimeseriesMetadata( Mockito.eq(deviceID), + Mockito.isNull(), Mockito.eq(measurement), Mockito.eq(allSensors), Mockito.eq(true), @@ -251,6 +252,7 @@ public void testIgnoreNotExistsResultIsNotCached() throws Exception { Mockito.when( reader.readTimeseriesMetadata( Mockito.eq(deviceID), + Mockito.isNull(), Mockito.eq(measurement), Mockito.eq(allSensors), Mockito.eq(false), @@ -264,6 +266,7 @@ public void testIgnoreNotExistsResultIsNotCached() throws Exception { Mockito.verify(reader) .readTimeseriesMetadata( Mockito.eq(deviceID), + Mockito.isNull(), Mockito.eq(measurement), Mockito.eq(allSensors), Mockito.eq(false), diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/queryengine/execution/MemoryEstimationHelper.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/queryengine/execution/MemoryEstimationHelper.java index 95796129011c6..e1436a1153816 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/queryengine/execution/MemoryEstimationHelper.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/queryengine/execution/MemoryEstimationHelper.java @@ -44,7 +44,7 @@ public class MemoryEstimationHelper { private static final long MEASUREMENT_PATH_INSTANCE_SIZE = RamUsageEstimator.shallowSizeOfInstance(AlignedPath.class); - private static final long ARRAY_LIST_INSTANCE_SIZE = + public static final long ARRAY_LIST_INSTANCE_SIZE = RamUsageEstimator.shallowSizeOfInstance(ArrayList.class); private static final long INTEGER_INSTANCE_SIZE = RamUsageEstimator.shallowSizeOfInstance(Integer.class); diff --git a/pom.xml b/pom.xml index c577243cb0ac2..096536c706a02 100644 --- a/pom.xml +++ b/pom.xml @@ -151,7 +151,7 @@ 0.14.1 1.9 1.5.6-3 - 2.3.2-260608-SNAPSHOT + 2.3.2-260616-SNAPSHOT en